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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index 7b1a91d..8287ced 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -111,10 +111,10 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * @param cacheName Cache name, if {@code null}, then default cache instance is used.
      * @param keys Cache keys to map to nodes.
      * @return Map of nodes to cache keys or empty map if there are no alive nodes for this cache.
-     * @throws IgniteCheckedException If failed to map cache keys.
+     * @throws IgniteException If failed to map cache keys.
      */
     public <K> Map<ClusterNode, Collection<K>> mapKeysToNodes(@Nullable String cacheName,
-        @Nullable Collection<? extends K> keys) throws IgniteCheckedException;
+        @Nullable Collection<? extends K> keys) throws IgniteException;
 
     /**
      * This method provides ability to detect which cache keys are mapped to which nodes
@@ -135,9 +135,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * @param key Cache key to map to a node.
      * @return Primary node for the key or {@code null} if cache with given name
      *      is not present in the grid.
-     * @throws IgniteCheckedException If failed to map key.
+     * @throws IgniteException If failed to map key.
      */
-    @Nullable public <K> ClusterNode mapKeyToNode(@Nullable String cacheName, K key) throws IgniteCheckedException;
+    @Nullable public <K> ClusterNode mapKeyToNode(@Nullable String cacheName, K key) throws IgniteException;
 
     /**
      * Starts one or more nodes on remote host(s).
@@ -164,10 +164,10 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * @param maxConn Number of parallel SSH connections to one host.
      * @return Collection of tuples, each containing host name, result (success of failure)
      *      and error message (if any).
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
     public Collection<GridTuple3<String, Boolean, String>> startNodes(File file, boolean restart,
-        int timeout, int maxConn) throws IgniteCheckedException;
+        int timeout, int maxConn) throws IgniteException;
 
     /**
      * Starts one or more nodes on remote host(s).
@@ -265,10 +265,10 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * @param maxConn Number of parallel SSH connections to one host.
      * @return Collection of tuples, each containing host name, result (success of failure)
      *      and error message (if any).
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
     public Collection<GridTuple3<String, Boolean, String>> startNodes(Collection<Map<String, Object>> hosts,
-        @Nullable Map<String, Object> dflts, boolean restart, int timeout, int maxConn) throws IgniteCheckedException;
+        @Nullable Map<String, Object> dflts, boolean restart, int timeout, int maxConn) throws IgniteException;
 
     /**
      * Stops nodes satisfying optional set of predicates.
@@ -277,9 +277,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * stopping node. If you have other applications running in the same JVM along with GridGain,
      * those applications will be stopped as well.
      *
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public void stopNodes() throws IgniteCheckedException;
+    public void stopNodes() throws IgniteException;
 
     /**
      * Stops nodes defined by provided IDs.
@@ -289,9 +289,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * those applications will be stopped as well.
      *
      * @param ids IDs defining nodes to stop.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public void stopNodes(Collection<UUID> ids) throws IgniteCheckedException;
+    public void stopNodes(Collection<UUID> ids) throws IgniteException;
 
     /**
      * Restarts nodes satisfying optional set of predicates.
@@ -299,9 +299,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * <b>NOTE:</b> this command only works for grid nodes started with GridGain
      * {@code ggstart.sh} or {@code ggstart.bat} scripts.
      *
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public void restartNodes() throws IgniteCheckedException;
+    public void restartNodes() throws IgniteException;
 
     /**
      * Restarts nodes defined by provided IDs.
@@ -310,9 +310,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * {@code ggstart.sh} or {@code ggstart.bat} scripts.
      *
      * @param ids IDs defining nodes to restart.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public void restartNodes(Collection<UUID> ids) throws IgniteCheckedException;
+    public void restartNodes(Collection<UUID> ids) throws IgniteException;
 
     /**
      * Resets local I/O, job, and task execution metrics.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
index 613a521..e3f6d86 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
@@ -114,9 +114,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param job Job which will be co-located on the node with given affinity key.
      * @see org.apache.ignite.compute.ComputeJobContext#cacheName()
      * @see org.apache.ignite.compute.ComputeJobContext#affinityKey()
-     * @throws IgniteCheckedException If job failed.
+     * @throws IgniteException If job failed.
      */
-    public void affinityRun(@Nullable String cacheName, Object affKey, Runnable job) throws IgniteCheckedException;
+    public void affinityRun(@Nullable String cacheName, Object affKey, Runnable job) throws IgniteException;
 
     /**
      * Executes given job on the node where data for provided affinity key is located
@@ -128,11 +128,11 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param affKey Affinity key.
      * @param job Job which will be co-located on the node with given affinity key.
      * @return Job result.
-     * @throws IgniteCheckedException If job failed.
+     * @throws IgniteException If job failed.
      * @see org.apache.ignite.compute.ComputeJobContext#cacheName()
      * @see org.apache.ignite.compute.ComputeJobContext#affinityKey()
      */
-    public <R> R affinityCall(@Nullable String cacheName, Object affKey, Callable<R> job) throws IgniteCheckedException;
+    public <R> R affinityCall(@Nullable String cacheName, Object affKey, Callable<R> job) throws IgniteException;
 
     /**
      * Executes given task on the grid projection. For step-by-step explanation of task execution process
@@ -145,9 +145,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      *      class name is used as task name.
      * @param arg Optional argument of task execution, can be {@code null}.
      * @return Task result.
-     * @throws IgniteCheckedException If task failed.
+     * @throws IgniteException If task failed.
      */
-    public <T, R> R execute(Class<? extends ComputeTask<T, R>> taskCls, @Nullable T arg) throws IgniteCheckedException;
+    public <T, R> R execute(Class<? extends ComputeTask<T, R>> taskCls, @Nullable T arg) throws IgniteException;
 
     /**
      * Executes given task on this grid projection. For step-by-step explanation of task execution process
@@ -160,9 +160,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      *      class name is used as task name.
      * @param arg Optional argument of task execution, can be {@code null}.
      * @return Task result.
-     * @throws IgniteCheckedException If task failed.
+     * @throws IgniteException If task failed.
      */
-    public <T, R> R execute(ComputeTask<T, R> task, @Nullable T arg) throws IgniteCheckedException;
+    public <T, R> R execute(ComputeTask<T, R> task, @Nullable T arg) throws IgniteException;
 
     /**
      * Executes given task on this grid projection. For step-by-step explanation of task execution process
@@ -176,10 +176,10 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param taskName Name of the task to execute.
      * @param arg Optional argument of task execution, can be {@code null}.
      * @return Task result.
-     * @throws IgniteCheckedException If task failed.
+     * @throws IgniteException If task failed.
      * @see org.apache.ignite.compute.ComputeTask for information about task execution.
      */
-    public <T, R> R execute(String taskName, @Nullable T arg) throws IgniteCheckedException;
+    public <T, R> R execute(String taskName, @Nullable T arg) throws IgniteException;
 
     /**
      * Broadcasts given job to all nodes in grid projection.
@@ -187,9 +187,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param job Job to broadcast to all projection nodes.
-     * @throws IgniteCheckedException If job failed.
+     * @throws IgniteException If job failed.
      */
-    public void broadcast(Runnable job) throws IgniteCheckedException;
+    public void broadcast(Runnable job) throws IgniteException;
 
     /**
      * Broadcasts given job to all nodes in grid projection. Every participating node will return a
@@ -199,9 +199,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      *
      * @param job Job to broadcast to all projection nodes.
      * @return Collection of results for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public <R> Collection<R> broadcast(Callable<R> job) throws IgniteCheckedException;
+    public <R> Collection<R> broadcast(Callable<R> job) throws IgniteException;
 
     /**
      * Broadcasts given closure job with passed in argument to all nodes in grid projection.
@@ -213,9 +213,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param job Job to broadcast to all projection nodes.
      * @param arg Job closure argument.
      * @return Collection of results for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public <R, T> Collection<R> broadcast(IgniteClosure<T, R> job, @Nullable T arg) throws IgniteCheckedException;
+    public <R, T> Collection<R> broadcast(IgniteClosure<T, R> job, @Nullable T arg) throws IgniteException;
 
     /**
      * Executes provided job on a node in this grid projection.
@@ -223,9 +223,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param job Job closure to execute.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public void run(Runnable job) throws IgniteCheckedException;
+    public void run(Runnable job) throws IgniteException;
 
     /**
      * Executes collection of jobs on grid nodes within this grid projection.
@@ -233,9 +233,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param jobs Collection of jobs to execute.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public void run(Collection<? extends Runnable> jobs) throws IgniteCheckedException;
+    public void run(Collection<? extends Runnable> jobs) throws IgniteException;
 
     /**
      * Executes provided job on a node in this grid projection. The result of the
@@ -245,9 +245,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      *
      * @param job Job to execute.
      * @return Job result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public <R> R call(Callable<R> job) throws IgniteCheckedException;
+    public <R> R call(Callable<R> job) throws IgniteException;
 
     /**
      * Executes collection of jobs on nodes within this grid projection.
@@ -257,9 +257,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      *
      * @param jobs Collection of jobs to execute.
      * @return Collection of job results for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public <R> Collection<R> call(Collection<? extends Callable<R>> jobs) throws IgniteCheckedException;
+    public <R> Collection<R> call(Collection<? extends Callable<R>> jobs) throws IgniteException;
 
     /**
      * Executes collection of jobs on nodes within this grid projection. The returned
@@ -270,9 +270,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param jobs Collection of jobs to execute.
      * @param rdc Reducer to reduce all job results into one individual return value.
      * @return Future with reduced job result for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public <R1, R2> R2 call(Collection<? extends Callable<R1>> jobs, IgniteReducer<R1, R2> rdc) throws IgniteCheckedException;
+    public <R1, R2> R2 call(Collection<? extends Callable<R1>> jobs, IgniteReducer<R1, R2> rdc) throws IgniteException;
 
     /**
      * Executes provided closure job on a node in this grid projection. This method is different
@@ -284,9 +284,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param job Job to run.
      * @param arg Job argument.
      * @return Job result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public <R, T> R apply(IgniteClosure<T, R> job, @Nullable T arg) throws IgniteCheckedException;
+    public <R, T> R apply(IgniteClosure<T, R> job, @Nullable T arg) throws IgniteException;
 
     /**
      * Executes provided closure job on nodes within this grid projection. A new job is executed for
@@ -298,9 +298,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param job Job to run.
      * @param args Job arguments.
      * @return Collection of job results.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
-    public <T, R> Collection<R> apply(IgniteClosure<T, R> job, Collection<? extends T> args) throws IgniteCheckedException;
+    public <T, R> Collection<R> apply(IgniteClosure<T, R> job, Collection<? extends T> args) throws IgniteException;
 
     /**
      * Executes provided closure job on nodes within this grid projection. A new job is executed for
@@ -314,10 +314,10 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param args Job arguments.
      * @param rdc Reducer to reduce all job results into one individual return value.
      * @return Future with reduced job result for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     public <R1, R2, T> R2 apply(IgniteClosure<T, R1> job, Collection<? extends T> args,
-        IgniteReducer<R1, R2> rdc) throws IgniteCheckedException;
+        IgniteReducer<R1, R2> rdc) throws IgniteException;
 
     /**
      * Gets tasks future for active tasks started on local node.
@@ -396,9 +396,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      *      class name will be used as task's name.
      * @param clsLdr Task class loader. This class loader is in charge
      *      of loading all necessary resources for task execution.
-     * @throws IgniteCheckedException If task is invalid and cannot be deployed.
+     * @throws IgniteException If task is invalid and cannot be deployed.
      */
-    public void localDeployTask(Class<? extends ComputeTask> taskCls, ClassLoader clsLdr) throws IgniteCheckedException;
+    public void localDeployTask(Class<? extends ComputeTask> taskCls, ClassLoader clsLdr) throws IgniteException;
 
     /**
      * Gets map of all locally deployed tasks keyed by their task name .
@@ -413,9 +413,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * undeployed on every node.
      *
      * @param taskName Name of the task to undeploy.
-     * @throws IgniteCheckedException Thrown if undeploy failed.
+     * @throws IgniteException Thrown if undeploy failed.
      */
-    public void undeployTask(String taskName) throws IgniteCheckedException;
+    public void undeployTask(String taskName) throws IgniteException;
 
     /** {@inheritDoc} */
     @Override public <R> ComputeTaskFuture<R> future();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
index ecedb0f..0793e69 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
@@ -109,9 +109,9 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      * Default is {@code false}.
      *
      * @param isolated Flag value.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
-    public void isolated(boolean isolated) throws IgniteCheckedException;
+    public void isolated(boolean isolated) throws IgniteException;
 
     /**
      * Gets size of per node key-value pairs buffer.
@@ -213,12 +213,12 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      *
      * @param key Key.
      * @return Future fo this operation.
-     * @throws IgniteCheckedException If failed to map key to node.
+     * @throws IgniteException If failed to map key to node.
      * @throws GridInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> removeData(K key)  throws IgniteCheckedException, GridInterruptedException, IllegalStateException;
+    public IgniteFuture<?> removeData(K key)  throws IgniteException, GridInterruptedException, IllegalStateException;
 
     /**
      * Adds data for loading on remote node. This method can be called from multiple
@@ -231,12 +231,12 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      * @param key Key.
      * @param val Value or {@code null} if respective entry must be removed from cache.
      * @return Future fo this operation.
-     * @throws IgniteCheckedException If failed to map key to node.
+     * @throws IgniteException If failed to map key to node.
      * @throws GridInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> addData(K key, @Nullable V val) throws IgniteCheckedException, GridInterruptedException,
+    public IgniteFuture<?> addData(K key, @Nullable V val) throws IgniteException, GridInterruptedException,
         IllegalStateException;
 
     /**
@@ -249,12 +249,12 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      *
      * @param entry Entry.
      * @return Future fo this operation.
-     * @throws IgniteCheckedException If failed to map key to node.
+     * @throws IgniteException If failed to map key to node.
      * @throws GridInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, GridInterruptedException,
+    public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteException, GridInterruptedException,
         IllegalStateException;
 
     /**
@@ -296,34 +296,34 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      * another thread to complete flush and exit. If you don't want to wait in this case,
      * use {@link #tryFlush()} method.
      *
-     * @throws IgniteCheckedException If failed to map key to node.
+     * @throws IgniteException If failed to map key to node.
      * @throws GridInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      * @see #tryFlush()
      */
-    public void flush() throws IgniteCheckedException, GridInterruptedException, IllegalStateException;
+    public void flush() throws IgniteException, GridInterruptedException, IllegalStateException;
 
     /**
      * Makes an attempt to load remaining data. This method is mostly similar to {@link #flush},
      * with the difference that it won't wait and will exit immediately.
      *
-     * @throws IgniteCheckedException If failed to map key to node.
+     * @throws IgniteException If failed to map key to node.
      * @throws GridInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      * @see #flush()
      */
-    public void tryFlush() throws IgniteCheckedException, GridInterruptedException, IllegalStateException;
+    public void tryFlush() throws IgniteException, GridInterruptedException, IllegalStateException;
 
     /**
      * Loads any remaining data and closes this loader.
      *
      * @param cancel {@code True} to cancel ongoing loading operations.
-     * @throws IgniteCheckedException If failed to map key to node.
+     * @throws IgniteException If failed to map key to node.
      * @throws GridInterruptedException If thread has been interrupted.
      */
-    public void close(boolean cancel) throws IgniteCheckedException, GridInterruptedException;
+    public void close(boolean cancel) throws IgniteException, GridInterruptedException;
 
     /**
      * Closes data loader. This method is identical to calling {@link #close(boolean) close(false)} method.
@@ -331,8 +331,8 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      * The method is invoked automatically on objects managed by the
      * {@code try-with-resources} statement.
      *
-     * @throws IgniteCheckedException If failed to close data loader.
+     * @throws IgniteException If failed to close data loader.
      * @throws GridInterruptedException If thread has been interrupted.
      */
-    @Override public void close() throws IgniteCheckedException, GridInterruptedException;
+    @Override public void close() throws IgniteException, GridInterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java b/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
index 728320e..e7c7b8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
@@ -62,10 +62,10 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      * @param timeout Maximum time to wait for result, {@code 0} to wait forever.
      * @param types Event types to be queried.
      * @return Collection of grid events returned from specified nodes.
-     * @throws IgniteCheckedException If query failed.
+     * @throws IgniteException If query failed.
      */
     public <T extends IgniteEvent> List<T> remoteQuery(IgnitePredicate<T> p, long timeout, @Nullable int... types)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Adds event listener for specified events to all nodes in the projection (possibly including
@@ -89,10 +89,10 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      *      provided remote filter will be sent to local node.
      * @param <T> Type of the event.
      * @return {@code Operation ID} that can be passed to {@link #stopRemoteListen(UUID)} method to stop listening.
-     * @throws IgniteCheckedException If failed to add listener.
+     * @throws IgniteException If failed to add listener.
      */
     public <T extends IgniteEvent> UUID remoteListen(@Nullable IgniteBiPredicate<UUID, T> locLsnr,
-        @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) throws IgniteCheckedException;
+        @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) throws IgniteException;
 
     /**
      * Adds event listener for specified events to all nodes in the projection (possibly including
@@ -127,11 +127,11 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      * @param <T> Type of the event.
      * @return {@code Operation ID} that can be passed to {@link #stopRemoteListen(UUID)} method to stop listening.
      * @see #stopRemoteListen(UUID)
-     * @throws IgniteCheckedException If failed to add listener.
+     * @throws IgniteException If failed to add listener.
      */
     public <T extends IgniteEvent> UUID remoteListen(int bufSize, long interval,
         boolean autoUnsubscribe, @Nullable IgniteBiPredicate<UUID, T> locLsnr, @Nullable IgnitePredicate<T> rmtFilter,
-        @Nullable int... types) throws IgniteCheckedException;
+        @Nullable int... types) throws IgniteException;
 
     /**
      * Stops listening to remote events. This will unregister all listeners identified with provided
@@ -142,9 +142,9 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      * @param opId Operation ID that was returned from
      *      {@link #remoteListen(org.apache.ignite.lang.IgniteBiPredicate, org.apache.ignite.lang.IgnitePredicate, int...)} method.
      * @see #remoteListen(org.apache.ignite.lang.IgniteBiPredicate, org.apache.ignite.lang.IgnitePredicate, int...)
-     * @throws IgniteCheckedException If failed to stop listeners.
+     * @throws IgniteException If failed to stop listeners.
      */
-    public void stopRemoteListen(UUID opId) throws IgniteCheckedException;
+    public void stopRemoteListen(UUID opId) throws IgniteException;
 
     /**
      * Waits for the specified events.
@@ -155,10 +155,10 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      *      end the wait.
      * @param types Types of the events to wait for. If not provided, all events will be passed to the filter.
      * @return Grid event.
-     * @throws IgniteCheckedException If wait was interrupted.
+     * @throws IgniteException If wait was interrupted.
      */
     public <T extends IgniteEvent> T waitForLocal(@Nullable IgnitePredicate<T> filter, @Nullable int... types)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Queries local node for events using passed-in predicate filter for event selection.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteFs.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFs.java b/modules/core/src/main/java/org/apache/ignite/IgniteFs.java
index 279ac37..e586477 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteFs.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteFs.java
@@ -76,19 +76,19 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param path Path to get information for.
      * @return Summary object.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path is not found.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
-    public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteCheckedException;
+    public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteException;
 
     /**
      * Opens a file for reading.
      *
      * @param path File path to read.
      * @return File input stream to read data from.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path doesn't exist.
      */
-    public IgniteFsInputStream open(IgniteFsPath path) throws IgniteCheckedException;
+    public IgniteFsInputStream open(IgniteFsPath path) throws IgniteException;
 
     /**
      * Opens a file for reading.
@@ -96,10 +96,10 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param path File path to read.
      * @param bufSize Read buffer size (bytes) or {@code zero} to use default value.
      * @return File input stream to read data from.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path doesn't exist.
      */
-    @Override public IgniteFsInputStream open(IgniteFsPath path, int bufSize) throws IgniteCheckedException;
+    @Override public IgniteFsInputStream open(IgniteFsPath path, int bufSize) throws IgniteException;
 
     /**
      * Opens a file for reading.
@@ -108,10 +108,10 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param bufSize Read buffer size (bytes) or {@code zero} to use default value.
      * @param seqReadsBeforePrefetch Amount of sequential reads before prefetch is started.
      * @return File input stream to read data from.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path doesn't exist.
      */
-    public IgniteFsInputStream open(IgniteFsPath path, int bufSize, int seqReadsBeforePrefetch) throws IgniteCheckedException;
+    public IgniteFsInputStream open(IgniteFsPath path, int bufSize, int seqReadsBeforePrefetch) throws IgniteException;
 
     /**
      * Creates a file and opens it for writing.
@@ -119,9 +119,9 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param path File path to create.
      * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
      * @return File output stream to write data to.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteCheckedException;
+    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteException;
 
     /**
      * Creates a file and opens it for writing.
@@ -133,10 +133,10 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param blockSize Block size.
      * @param props File properties to set.
      * @return File output stream to write data to.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
     @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite, int replication,
-        long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException;
+        long blockSize, @Nullable Map<String, String> props) throws IgniteException;
 
     /**
      * Creates a file and opens it for writing.
@@ -150,11 +150,11 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param blockSize Block size.
      * @param props File properties to set.
      * @return File output stream to write data to.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
     public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite,
         @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Opens an output stream to an existing file for appending data.
@@ -162,10 +162,10 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param path File path to append.
      * @param create Create file if it doesn't exist yet.
      * @return File output stream to append data to.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path doesn't exist and create flag is {@code false}.
      */
-    public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteCheckedException;
+    public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteException;
 
     /**
      * Opens an output stream to an existing file for appending data.
@@ -175,11 +175,11 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param create Create file if it doesn't exist yet.
      * @param props File properties to set only in case it file was just created.
      * @return File output stream to append data to.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path doesn't exist and create flag is {@code false}.
      */
     @Override public IgniteFsOutputStream append(IgniteFsPath path, int bufSize, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException;
+        @Nullable Map<String, String> props) throws IgniteException;
 
     /**
      * Sets last access time and last modification time for a given path. If argument is {@code null},
@@ -190,9 +190,9 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param modificationTime Optional last modification time to set. Value {@code -1} does not update
      *      modification time.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If target was not found.
-     * @throws IgniteCheckedException If error occurred.
+     * @throws IgniteException If error occurred.
      */
-    public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws IgniteCheckedException;
+    public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws IgniteException;
 
     /**
      * Gets affinity block locations for data blocks of the file, i.e. the nodes, on which the blocks
@@ -202,10 +202,10 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param start Position in the file to start affinity resolution from.
      * @param len Size of data in the file to resolve affinity for.
      * @return Affinity block locations.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path doesn't exist.
      */
-    public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len) throws IgniteCheckedException;
+    public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len) throws IgniteException;
 
     /**
      * Get affinity block locations for data blocks of the file. In case {@code maxLen} parameter is set and
@@ -217,26 +217,26 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param len Size of data in the file to resolve affinity for.
      * @param maxLen Maximum length of a single returned block location length.
      * @return Affinity block locations.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      * @throws org.apache.ignite.fs.IgniteFsFileNotFoundException If path doesn't exist.
      */
     public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len, long maxLen)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Gets metrics snapshot for this file system.
      *
      * @return Metrics.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public IgniteFsMetrics metrics() throws IgniteCheckedException;
+    public IgniteFsMetrics metrics() throws IgniteException;
 
     /**
      * Resets metrics for this file system.
      *
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public void resetMetrics() throws IgniteCheckedException;
+    public void resetMetrics() throws IgniteException;
 
     /**
      * Determines size of the file denoted by provided path. In case if path is a directory, then
@@ -244,18 +244,18 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      *
      * @param path File system path.
      * @return Total size.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public long size(IgniteFsPath path) throws IgniteCheckedException;
+    public long size(IgniteFsPath path) throws IgniteException;
 
     /**
      * Formats the file system removing all existing entries from it.
      * <p>
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
-     * @throws IgniteCheckedException In case format has failed.
+     * @throws IgniteException In case format has failed.
      */
-    public void format() throws IgniteCheckedException;
+    public void format() throws IgniteException;
 
     /**
      * Executes GGFS task.
@@ -267,10 +267,10 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param paths Collection of paths to be processed within this task.
      * @param arg Optional task argument.
      * @return Task result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
-        Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException;
+        Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException;
 
     /**
      * Executes GGFS task with overridden maximum range length (see
@@ -287,11 +287,11 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      *      GGFS blocks will be included.
      * @param arg Optional task argument.
      * @return Task result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
         Collection<IgniteFsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Executes GGFS task.
@@ -303,10 +303,10 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param paths Collection of paths to be processed within this task.
      * @param arg Optional task argument.
      * @return Task result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls,
-        @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException;
+        @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException;
 
     /**
      * Executes GGFS task with overridden maximum range length (see
@@ -322,11 +322,11 @@ public interface IgniteFs extends IgniteFsFileSystem, IgniteAsyncSupport {
      * @param maxRangeLen Maximum range length.
      * @param arg Optional task argument.
      * @return Task result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls,
         @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles,
-        long maxRangeLen, @Nullable T arg) throws IgniteCheckedException;
+        long maxRangeLen, @Nullable T arg) throws IgniteException;
 
     /** {@inheritDoc} */
     @Override public IgniteFs enableAsync();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java b/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
index 9aaa644..3886db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
@@ -141,9 +141,9 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      *
      * @param name Service name.
      * @param svc Service instance.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
-    public void deployClusterSingleton(String name, ManagedService svc) throws IgniteCheckedException;
+    public void deployClusterSingleton(String name, ManagedService svc) throws IgniteException;
 
     /**
      * Deploys a per-node singleton service. GridGain will guarantee that there is always
@@ -158,9 +158,9 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      *
      * @param name Service name.
      * @param svc Service instance.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
-    public void deployNodeSingleton(String name, ManagedService svc) throws IgniteCheckedException;
+    public void deployNodeSingleton(String name, ManagedService svc) throws IgniteException;
 
     /**
      * Deploys one instance of this service on the primary node for a given affinity key.
@@ -193,10 +193,10 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      * @param cacheName Name of the cache on which affinity for key should be calculated, {@code null} for
      *      default cache.
      * @param affKey Affinity cache key.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
     public void deployKeyAffinitySingleton(String name, ManagedService svc, @Nullable String cacheName, Object affKey)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Deploys multiple instances of the service on the grid. GridGain will deploy a
@@ -228,9 +228,9 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      * @param svc Service instance.
      * @param totalCnt Maximum number of deployed services in the grid, {@code 0} for unlimited.
      * @param maxPerNodeCnt Maximum number of deployed services on each node, {@code 0} for unlimited.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
-    public void deployMultiple(String name, ManagedService svc, int totalCnt, int maxPerNodeCnt) throws IgniteCheckedException;
+    public void deployMultiple(String name, ManagedService svc, int totalCnt, int maxPerNodeCnt) throws IgniteException;
 
     /**
      * Deploys multiple instances of the service on the grid according to provided
@@ -268,9 +268,9 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      * </pre>
      *
      * @param cfg Service configuration.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
-    public void deploy(ManagedServiceConfiguration cfg) throws IgniteCheckedException;
+    public void deploy(ManagedServiceConfiguration cfg) throws IgniteException;
 
     /**
      * Cancels service deployment. If a service with specified name was deployed on the grid,
@@ -283,9 +283,9 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param name Name of service to cancel.
-     * @throws IgniteCheckedException If failed to cancel service.
+     * @throws IgniteException If failed to cancel service.
      */
-    public void cancel(String name) throws IgniteCheckedException;
+    public void cancel(String name) throws IgniteException;
 
     /**
      * Cancels all deployed services.
@@ -295,9 +295,9 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      * <p>
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
-     * @throws IgniteCheckedException If failed to cancel services.
+     * @throws IgniteException If failed to cancel services.
      */
-    public void cancelAll() throws IgniteCheckedException;
+    public void cancelAll() throws IgniteException;
 
     /**
      * Gets metadata about all deployed services.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java b/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
index f11b3a6..f1b8e6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
@@ -57,10 +57,10 @@ public interface IgniteMessaging extends IgniteAsyncSupport {
      *
      * @param topic Topic to send to, {@code null} for default topic.
      * @param msg Message to send.
-     * @throws IgniteCheckedException If failed to send a message to any of the nodes.
+     * @throws IgniteException If failed to send a message to any of the nodes.
      * @throws org.apache.ignite.cluster.ClusterGroupEmptyException Thrown in case when this projection is empty.
      */
-    public void send(@Nullable Object topic, Object msg) throws IgniteCheckedException;
+    public void send(@Nullable Object topic, Object msg) throws IgniteException;
 
     /**
      * Sends given messages with specified topic to the nodes in this projection.
@@ -68,10 +68,10 @@ public interface IgniteMessaging extends IgniteAsyncSupport {
      * @param topic Topic to send to, {@code null} for default topic.
      * @param msgs Messages to send. Order of the sending is undefined. If the method produces
      *      the exception none or some messages could have been sent already.
-     * @throws IgniteCheckedException If failed to send a message to any of the nodes.
+     * @throws IgniteException If failed to send a message to any of the nodes.
      * @throws org.apache.ignite.cluster.ClusterGroupEmptyException Thrown in case when this projection is empty.
      */
-    public void send(@Nullable Object topic, Collection<?> msgs) throws IgniteCheckedException;
+    public void send(@Nullable Object topic, Collection<?> msgs) throws IgniteException;
 
     /**
      * Sends given message with specified topic to the nodes in this projection. Messages sent with
@@ -87,10 +87,10 @@ public interface IgniteMessaging extends IgniteAsyncSupport {
      * @param msg Message to send.
      * @param timeout Message timeout in milliseconds, {@code 0} for default
      *      which is {@link org.apache.ignite.configuration.IgniteConfiguration#getNetworkTimeout()}.
-     * @throws IgniteCheckedException If failed to send a message to any of the nodes.
+     * @throws IgniteException If failed to send a message to any of the nodes.
      * @throws org.apache.ignite.cluster.ClusterGroupEmptyException Thrown in case when this projection is empty.
      */
-    public void sendOrdered(@Nullable Object topic, Object msg, long timeout) throws IgniteCheckedException;
+    public void sendOrdered(@Nullable Object topic, Object msg, long timeout) throws IgniteException;
 
     /**
      * Adds local listener for given topic on local node only. This listener will be notified whenever any
@@ -123,9 +123,9 @@ public interface IgniteMessaging extends IgniteAsyncSupport {
      * @param p Predicate that is called on each node for each received message. If predicate returns {@code false},
      *      then it will be unsubscribed from any further notifications.
      * @return {@code Operation ID} that can be passed to {@link #stopRemoteListen(UUID)} method to stop listening.
-     * @throws IgniteCheckedException If failed to add listener.
+     * @throws IgniteException If failed to add listener.
      */
-    public UUID remoteListen(@Nullable Object topic, IgniteBiPredicate<UUID, ?> p) throws IgniteCheckedException;
+    public UUID remoteListen(@Nullable Object topic, IgniteBiPredicate<UUID, ?> p) throws IgniteException;
 
     /**
      * Unregisters all listeners identified with provided operation ID on all nodes in this projection.
@@ -133,9 +133,9 @@ public interface IgniteMessaging extends IgniteAsyncSupport {
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param opId Listen ID that was returned from {@link #remoteListen(Object, org.apache.ignite.lang.IgniteBiPredicate)} method.
-     * @throws IgniteCheckedException If failed to unregister listeners.
+     * @throws IgniteException If failed to unregister listeners.
      */
-    public void stopRemoteListen(UUID opId) throws IgniteCheckedException;
+    public void stopRemoteListen(UUID opId) throws IgniteException;
 
     /** {@inheritDoc} */
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/IgniteStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteStreamer.java b/modules/core/src/main/java/org/apache/ignite/IgniteStreamer.java
index 418c7bb..f878cf0 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteStreamer.java
@@ -60,9 +60,9 @@ public interface IgniteStreamer {
      *
      * @param evt Event to add.
      * @param evts Optional events to add.
-     * @throws IgniteCheckedException If event submission failed.
+     * @throws IgniteException If event submission failed.
      */
-    public void addEvent(Object evt, Object... evts) throws IgniteCheckedException;
+    public void addEvent(Object evt, Object... evts) throws IgniteException;
 
     /**
      * Submits group of events to streamer. Events will be processed from a stage with specified name.
@@ -70,27 +70,27 @@ public interface IgniteStreamer {
      * @param stageName Stage name to start with.
      * @param evt Event tp process.
      * @param evts Optional events.
-     * @throws IgniteCheckedException If event submission failed.
+     * @throws IgniteException If event submission failed.
      */
-    public void addEventToStage(String stageName, Object evt, Object... evts) throws IgniteCheckedException;
+    public void addEventToStage(String stageName, Object evt, Object... evts) throws IgniteException;
 
     /**
      * Submits group of events for processing. This group of events will be processed on default stage,
      * i.e. stage that is the first in the streamer stages list.
      *
      * @param evts Events to add.
-     * @throws IgniteCheckedException If event submission failed.
+     * @throws IgniteException If event submission failed.
      */
-    public void addEvents(Collection<?> evts) throws IgniteCheckedException;
+    public void addEvents(Collection<?> evts) throws IgniteException;
 
     /**
      * Submits events to streamer. Events will be processed from a stage with specified name.
      *
      * @param stageName Stage name to start with.
      * @param evts Events to process.
-     * @throws IgniteCheckedException If event submission failed.
+     * @throws IgniteException If event submission failed.
      */
-    public void addEventsToStage(String stageName, Collection<?> evts) throws IgniteCheckedException;
+    public void addEventsToStage(String stageName, Collection<?> evts) throws IgniteException;
 
     /**
      * Gets streamer context. Streamer context provides access to streamer local space on this node, configured

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index 1a8d17f..8b883c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -13,6 +13,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.lifecycle.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.net.*;
@@ -261,11 +262,16 @@ public class Ignition {
      * configuration file. If such file is not found, then all system defaults will be used.
      *
      * @return Started grid.
-     * @throws IgniteCheckedException If default grid could not be started. This exception will be thrown
+     * @throws IgniteException If default grid could not be started. This exception will be thrown
      *      also if default grid has already been started.
      */
-    public static Ignite start() throws IgniteCheckedException {
-        return GridGainEx.start();
+    public static Ignite start() throws IgniteException {
+        try {
+            return GridGainEx.start();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
+        }
     }
 
     /**
@@ -274,11 +280,16 @@ public class Ignition {
      *
      * @param cfg Grid configuration. This cannot be {@code null}.
      * @return Started grid.
-     * @throws IgniteCheckedException If grid could not be started. This exception will be thrown
+     * @throws IgniteException If grid could not be started. This exception will be thrown
      *      also if named grid has already been started.
      */
-    public static Ignite start(IgniteConfiguration cfg) throws IgniteCheckedException {
-        return GridGainEx.start(cfg);
+    public static Ignite start(IgniteConfiguration cfg) throws IgniteException {
+        try {
+            return GridGainEx.start(cfg);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
+        }
     }
 
     /**
@@ -293,12 +304,17 @@ public class Ignition {
      * @param springCfgPath Spring XML configuration file path or URL.
      * @return Started grid. If Spring configuration contains multiple grid instances,
      *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or configuration
+     * @throws IgniteException If grid could not be started or configuration
      *      read. This exception will be thrown also if grid with given name has already
      *      been started or Spring XML configuration file is invalid.
      */
-    public static Ignite start(@Nullable String springCfgPath) throws IgniteCheckedException {
-        return GridGainEx.start(springCfgPath);
+    public static Ignite start(@Nullable String springCfgPath) throws IgniteException {
+        try {
+            return GridGainEx.start(springCfgPath);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
+        }
     }
 
     /**
@@ -313,12 +329,17 @@ public class Ignition {
      * @param springCfgUrl Spring XML configuration file URL. This cannot be {@code null}.
      * @return Started grid. If Spring configuration contains multiple grid instances,
      *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or configuration
+     * @throws IgniteException If grid could not be started or configuration
      *      read. This exception will be thrown also if grid with given name has already
      *      been started or Spring XML configuration file is invalid.
      */
-    public static Ignite start(URL springCfgUrl) throws IgniteCheckedException {
-        return GridGainEx.start(springCfgUrl);
+    public static Ignite start(URL springCfgUrl) throws IgniteException {
+        try {
+            return GridGainEx.start(springCfgUrl);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
index 91892e8..59ad03e 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
@@ -193,7 +193,7 @@ public final class QueryContinuousPredicate<K, V> extends QueryPredicate<K, V> i
      *
      * @throws IgniteCheckedException In case of error.
      */
-    @Override public void close() throws IgniteCheckedException {
+    @Override public void close() throws IgniteException {
         // TODO: implement.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
index 0035a25..91ce0e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
@@ -243,7 +243,7 @@ public interface ClusterGroup {
      * Gets a metrics snapshot for this projection.
      *
      * @return Grid projection metrics snapshot.
-     * @throws IgniteCheckedException If projection is empty.
+     * @throws IgniteException If projection is empty.
      */
-    public ClusterMetrics metrics() throws IgniteCheckedException;
+    public ClusterMetrics metrics() throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
index 879e6bb..baa13ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterTopologyException.java
@@ -15,7 +15,7 @@ import org.jetbrains.annotations.*;
 /**
  * This exception is used to indicate error with grid topology (e.g., crashed node, etc.).
  */
-public class ClusterTopologyException extends IgniteCheckedException {
+public class ClusterTopologyException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeExecutionRejectedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeExecutionRejectedException.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeExecutionRejectedException.java
index ff526b9..0ce49f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeExecutionRejectedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeExecutionRejectedException.java
@@ -18,7 +18,7 @@ import org.jetbrains.annotations.*;
  * rejects execution.
  * @see org.apache.ignite.configuration.IgniteConfiguration#getExecutorService()
  */
-public class ComputeExecutionRejectedException extends IgniteCheckedException {
+public class ComputeExecutionRejectedException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJob.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJob.java
index 48926c6..0ffff8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJob.java
@@ -151,11 +151,11 @@ public interface ComputeJob extends Serializable {
      * @return Job execution result (possibly {@code null}). This result will be returned
      *      in {@link ComputeJobResult#getData()} method passed into
      *      {@link ComputeTask#result(ComputeJobResult, List)} task method on caller node.
-     * @throws IgniteCheckedException If job execution caused an exception. This exception will be
+     * @throws IgniteException If job execution caused an exception. This exception will be
      *      returned in {@link ComputeJobResult#getException()} method passed into
      *      {@link ComputeTask#result(ComputeJobResult, List)} task method on caller node.
      *      If execution produces a {@link RuntimeException} or {@link Error}, then
-     *      it will be wrapped into {@link IgniteCheckedException}.
+     *      it will be wrapped into {@link IgniteException}.
      */
-    @Nullable public Object execute() throws IgniteCheckedException;
+    @Nullable public Object execute() throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobContinuationAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobContinuationAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobContinuationAdapter.java
index f9fff7f..80bf5a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobContinuationAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobContinuationAdapter.java
@@ -35,13 +35,13 @@ import org.jetbrains.annotations.*;
  *     private int multiplier = 3;
  *
  *     &#64;Override
- *     protected Collection&lt;? extends ComputeJob&gt; split(int gridSize, final String arg) throws IgniteCheckedException {
+ *     protected Collection&lt;? extends ComputeJob&gt; split(int gridSize, final String arg) throws IgniteException {
  *         List&lt;GridComputeJobAdapter&lt;String&gt;&gt; jobs = new ArrayList&lt;GridComputeJobAdapter&lt;String&gt;&gt;(gridSize);
  *
  *         for (int i = 0; i < gridSize; i++) {
  *             jobs.add(new GridComputeJobAdapter() {
  *                 // Job execution logic.
- *                 public Object execute() throws IgniteCheckedException {
+ *                 public Object execute() throws IgniteException {
  *                     return multiplier * arg.length();
  *                 }
  *             });
@@ -52,7 +52,7 @@ import org.jetbrains.annotations.*;
  *
  *     // Aggregate multiple job results into
  *     // one task result.
- *     public Integer reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteCheckedException {
+ *     public Integer reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteException {
  *         int sum = 0;
  *
  *         // For the sake of this example, let's sum all results.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobMasterLeaveAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobMasterLeaveAware.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobMasterLeaveAware.java
index ab4ab6a..cb4f77d 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobMasterLeaveAware.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobMasterLeaveAware.java
@@ -17,7 +17,7 @@ public interface ComputeJobMasterLeaveAware {
      * A method which is executed in case master node has left topology during job execution.
      *
      * @param ses Task session, can be used for checkpoint saving.
-     * @throws IgniteCheckedException In case of any exception.
+     * @throws IgniteException In case of any exception.
      */
-    public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException;
+    public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
index 7c9ae3e..585d109 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
@@ -49,7 +49,7 @@ public interface ComputeJobResult {
      * Gets exception produced by execution of remote job, or {@code null} if
      * remote execution finished normally and did not produce any exceptions.
      *
-     * @return {@link IgniteCheckedException} produced by execution of remote job or {@code null} if
+     * @return {@link IgniteException} produced by execution of remote job or {@code null} if
      *      no exception was produced.
      *      <p>
      *      Note that if remote job resulted in {@link RuntimeException}
@@ -62,7 +62,7 @@ public interface ComputeJobResult {
      *      If node on which job was computing failed, then {@link org.apache.ignite.cluster.ClusterTopologyException} is
      *      returned.
      */
-    public IgniteCheckedException getException();
+    public IgniteException getException();
 
     /**
      * Gets local instance of remote job returned by {@link ComputeTask#map(List, Object)} method.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobSibling.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobSibling.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobSibling.java
index 330196a..8285fc2 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobSibling.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobSibling.java
@@ -32,7 +32,7 @@ public interface ComputeJobSibling extends GridMetadataAware {
     /**
      * Sends a request to cancel this sibling.
      *
-     * @throws IgniteCheckedException If cancellation message could not be sent.
+     * @throws IgniteException If cancellation message could not be sent.
      */
-    public void cancel() throws IgniteCheckedException;
+    public void cancel() throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
index e4ee221..df8d4aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
@@ -34,7 +34,7 @@ import java.util.*;
  * <pre name="code" class="java">
  * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;String> {
  *     &#64;Override
- *     protected Collection&lt;? extends ComputeJob> split(int gridSize, String arg) throws IgniteCheckedException {
+ *     protected Collection&lt;? extends ComputeJob> split(int gridSize, String arg) throws IgniteException {
  *         List&lt;MyFooBarJob> jobs = new ArrayList&lt;MyFooBarJob>(gridSize);
  *
  *         for (int i = 0; i &lt; gridSize; i++) {
@@ -60,7 +60,7 @@ import java.util.*;
  *     GridComputeLoadBalancer balancer;
  *
  *     // Map jobs to grid nodes.
- *     public Map&lt;? extends ComputeJob, GridNode> map(List&lt;GridNode> subgrid, String arg) throws IgniteCheckedException {
+ *     public Map&lt;? extends ComputeJob, GridNode> map(List&lt;GridNode> subgrid, String arg) throws IgniteException {
  *         Map&lt;MyFooBarJob, GridNode> jobs = new HashMap&lt;MyFooBarJob, GridNode>(subgrid.size());
  *
  *         // In more complex cases, you can actually do
@@ -76,7 +76,7 @@ import java.util.*;
  *     }
  *
  *     // Aggregate results into one compound result.
- *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteCheckedException {
+ *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteException {
  *         // For the purpose of this example we simply
  *         // concatenate string representation of every
  *         // job result
@@ -101,8 +101,8 @@ public interface ComputeLoadBalancer extends GridMetadataAware {
      * @param exclNodes Optional collection of nodes that should be excluded from balanced nodes.
      *      If collection is {@code null} or empty - no nodes will be excluded.
      * @return Next balanced node.
-     * @throws IgniteCheckedException If any error occurred when finding next balanced node.
+     * @throws IgniteException If any error occurred when finding next balanced node.
      */
     @Nullable public ClusterNode getBalancedNode(ComputeJob job, @Nullable Collection<ClusterNode> exclNodes)
-        throws IgniteCheckedException;
+        throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
index b3d90f3..6d81e86 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
@@ -153,7 +153,7 @@ import java.util.*;
  *     GridComputeLoadBalancer balancer;
  *
  *     // Map jobs to grid nodes.
- *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteCheckedException {
+ *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteException {
  *         Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
  *
  *         // In more complex cases, you can actually do
@@ -167,7 +167,7 @@ import java.util.*;
  *     }
  *
  *     // Aggregate results into one compound result.
- *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteCheckedException {
+ *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteException {
  *         // For the purpose of this example we simply
  *         // concatenate string representation of every
  *         // job result
@@ -192,7 +192,7 @@ import java.util.*;
  * <pre name="code" class="java">
  * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, String&gt; {
  *     &#64;Override
- *     protected Collection&lt;? extends ComputeJob&gt; split(int gridSize, Object arg) throws IgniteCheckedException {
+ *     protected Collection&lt;? extends ComputeJob&gt; split(int gridSize, Object arg) throws IgniteException {
  *         List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
  *
  *         for (int i = 0; i &lt; gridSize; i++) {
@@ -205,7 +205,7 @@ import java.util.*;
  *     }
  *
  *     // Aggregate results into one compound result.
- *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteCheckedException {
+ *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteException {
  *         // For the purpose of this example we simply
  *         // concatenate string representation of every
  *         // job result
@@ -239,10 +239,10 @@ public interface ComputeTask<T, R> extends Serializable {
      *      over time should result into all nodes being used equally.
      * @return Map of grid jobs assigned to subgrid node. Unless {@link ComputeTaskContinuousMapper} is
      *      injected into task, if {@code null} or empty map is returned, exception will be thrown.
-     * @throws IgniteCheckedException If mapping could not complete successfully. This exception will be
+     * @throws IgniteException If mapping could not complete successfully. This exception will be
      *      thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    @Nullable public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable T arg) throws IgniteCheckedException;
+    @Nullable public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable T arg) throws IgniteException;
 
     /**
      * Asynchronous callback invoked every time a result from remote execution is
@@ -256,10 +256,10 @@ public interface ComputeTask<T, R> extends Serializable {
      *      {@link ComputeTaskNoResultCache} annotation, then this list will be empty.
      * @return Result policy that dictates how to process further upcoming
      *       job results.
-     * @throws IgniteCheckedException If handling a job result caused an error. This exception will
+     * @throws IgniteException If handling a job result caused an error. This exception will
      *      be thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteCheckedException;
+    public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException;
 
     /**
      * Reduces (or aggregates) results received so far into one compound result to be returned to
@@ -272,8 +272,8 @@ public interface ComputeTask<T, R> extends Serializable {
      * @param results Received results of broadcasted remote executions. Note that if task class has
      *      {@link ComputeTaskNoResultCache} annotation, then this list will be empty.
      * @return Grid job result constructed from results of remote executions.
-     * @throws IgniteCheckedException If reduction or results caused an error. This exception will
+     * @throws IgniteException If reduction or results caused an error. This exception will
      *      be thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    @Nullable public R reduce(List<ComputeJobResult> results) throws IgniteCheckedException;
+    @Nullable public R reduce(List<ComputeJobResult> results) throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
index bcb3715..859c85a 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
@@ -25,7 +25,7 @@ import java.util.*;
  *     GridComputeLoadBalancer balancer;
  *
  *     // Map jobs to grid nodes.
- *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteCheckedException {
+ *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteException {
  *         Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
  *
  *         // In more complex cases, you can actually do
@@ -39,7 +39,7 @@ import java.util.*;
  *     }
  *
  *     // Aggregate results into one compound result.
- *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteCheckedException {
+ *     public String reduce(List&lt;GridComputeJobResult&gt; results) throws IgniteException {
  *         // For the purpose of this example we simply
  *         // concatenate string representation of every
  *         // job result
@@ -77,11 +77,11 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * @param rcvd All previously received results.
      * @return Result policy that dictates how to process further upcoming
      *       job results.
-     * @throws IgniteCheckedException If handling a job result caused an error effectively rejecting
+     * @throws IgniteException If handling a job result caused an error effectively rejecting
      *      a failover. This exception will be thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteCheckedException {
-        IgniteCheckedException e = res.getException();
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
+        IgniteException e = res.getException();
 
         // Try to failover if result is failed.
         if (e != null) {
@@ -92,7 +92,7 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
                 e.hasCause(ComputeJobFailoverException.class))
                 return ComputeJobResultPolicy.FAILOVER;
 
-            throw new IgniteCheckedException("Remote job threw user exception (override or implement GridComputeTask.result(..) " +
+            throw new IgniteException("Remote job threw user exception (override or implement GridComputeTask.result(..) " +
                 "method if you would like to have automatic failover for this exception).", e);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskCancelledException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskCancelledException.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskCancelledException.java
index 7f79a4c..3646050 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskCancelledException.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskCancelledException.java
@@ -15,7 +15,7 @@ import org.jetbrains.annotations.*;
 /**
  * This exception indicates that grid task was cancelled.
  */
-public class ComputeTaskCancelledException extends IgniteCheckedException {
+public class ComputeTaskCancelledException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskContinuousMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskContinuousMapper.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskContinuousMapper.java
index e43c06f..f7caf07 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskContinuousMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskContinuousMapper.java
@@ -59,33 +59,33 @@ public interface ComputeTaskContinuousMapper {
      *
      * @param job Job instance to send. If {@code null} is passed, exception will be thrown.
      * @param node Grid node. If {@code null} is passed, exception will be thrown.
-     * @throws IgniteCheckedException If job can not be processed.
+     * @throws IgniteException If job can not be processed.
      */
-    public void send(ComputeJob job, ClusterNode node) throws IgniteCheckedException;
+    public void send(ComputeJob job, ClusterNode node) throws IgniteException;
 
     /**
      * Sends collection of grid jobs to assigned nodes.
      *
      * @param mappedJobs Map of grid jobs assigned to grid node. If {@code null}
      *      or empty list is passed, exception will be thrown.
-     * @throws IgniteCheckedException If job can not be processed.
+     * @throws IgniteException If job can not be processed.
      */
-    public void send(Map<? extends ComputeJob, ClusterNode> mappedJobs) throws IgniteCheckedException;
+    public void send(Map<? extends ComputeJob, ClusterNode> mappedJobs) throws IgniteException;
 
     /**
      * Sends job to a node automatically picked by the underlying load balancer.
      *
      * @param job Job instance to send. If {@code null} is passed, exception will be thrown.
-     * @throws IgniteCheckedException If job can not be processed.
+     * @throws IgniteException If job can not be processed.
      */
-    public void send(ComputeJob job) throws IgniteCheckedException;
+    public void send(ComputeJob job) throws IgniteException;
 
     /**
      * Sends collection of jobs to nodes automatically picked by the underlying load balancer.
      *
      * @param jobs Collection of grid job instances. If {@code null} or empty
      *      list is passed, exception will be thrown.
-     * @throws IgniteCheckedException If job can not be processed.
+     * @throws IgniteException If job can not be processed.
      */
-    public void send(Collection<? extends ComputeJob> jobs) throws IgniteCheckedException;
+    public void send(Collection<? extends ComputeJob> jobs) throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
index 70d4934..aa3fc2a 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
@@ -27,21 +27,21 @@ public interface ComputeTaskFuture<R> extends IgniteFuture<R> {
      *
      * @throws ComputeTaskTimeoutException If task execution timed out.
      */
-    @Override public R get() throws IgniteCheckedException;
+    @Override public R get() throws IgniteException;
 
     /**
      * {@inheritDoc}
      *
      * @throws ComputeTaskTimeoutException If task execution timed out.
      */
-    @Override public R get(long timeout) throws IgniteCheckedException;
+    @Override public R get(long timeout) throws IgniteException;
 
     /**
      * {@inheritDoc}
      *
      * @throws ComputeTaskTimeoutException If task execution timed out.
      */
-    @Override public R get(long timeout, TimeUnit unit) throws IgniteCheckedException;
+    @Override public R get(long timeout, TimeUnit unit) throws IgniteException;
 
     /**
      * Gets task session of execution grid task.