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 2015/02/02 10:10:06 UTC

[3/3] incubator-ignite git commit: # ignite-26

# ignite-26


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

Branch: refs/heads/ignite-26
Commit: c1ac5a5e6545b4993ab12b4b4f5e987d6c3e7d8d
Parents: a88b637
Author: sboikov <sb...@gridgain.com>
Authored: Mon Feb 2 11:33:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Feb 2 11:55:31 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCluster.java   |  24 ++--
 .../ignite/internal/IgniteClusterAsyncImpl.java |  42 ++++--
 .../apache/ignite/internal/IgniteKernal.java    |  50 +++++--
 .../ignite/internal/util/IgniteUtils.java       | 141 ++++++++++++++-----
 .../fs/GridGgfsProcessorValidationSelfTest.java |   3 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |   8 +-
 .../java/org/apache/ignite/IgniteSpring.java    |   2 +-
 .../commands/alert/VisorAlertCommand.scala      |   6 +-
 .../commands/cache/VisorCacheCommand.scala      |   4 +-
 .../config/VisorConfigurationCommand.scala      |   2 +-
 .../commands/events/VisorEventsCommand.scala    |   2 +-
 .../visor/commands/gc/VisorGcCommand.scala      |   4 +-
 .../visor/commands/kill/VisorKillCommand.scala  |   4 +-
 .../commands/start/VisorStartCommand.scala      |   4 +-
 .../commands/tasks/VisorTasksCommand.scala      |  10 +-
 .../commands/top/VisorTopologyCommand.scala     |   4 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  16 +--
 17 files changed, 218 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/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 94eab2b..da5c76b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -171,14 +171,14 @@ 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.
      */
     @IgniteAsyncSupported
     public Collection<GridTuple3<String, Boolean, String>> startNodes(File file,
         boolean restart,
         int timeout,
         int maxConn)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Starts one or more nodes on remote host(s).
@@ -276,7 +276,7 @@ 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.
      */
     @IgniteAsyncSupported
     public Collection<GridTuple3<String, Boolean, String>> startNodes(Collection<Map<String, Object>> hosts,
@@ -284,7 +284,7 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
         boolean restart,
         int timeout,
         int maxConn)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Stops nodes satisfying optional set of predicates.
@@ -293,9 +293,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.
@@ -305,9 +305,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.
@@ -315,9 +315,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * <b>NOTE:</b> this command only works for grid nodes started with GridGain
      * {@code ignite.sh} or {@code ignite.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.
@@ -326,9 +326,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * {@code ignite.sh} or {@code ignite.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/c1ac5a5e/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
index 6b3b893..c684217 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -85,34 +86,51 @@ public class IgniteClusterAsyncImpl extends IgniteAsyncSupportAdapter<IgniteClus
 
     /** {@inheritDoc} */
     @Override public Collection<GridTuple3<String, Boolean, String>> startNodes(File file,
-        boolean restart, int timeout, int maxConn) throws IgniteCheckedException {
-        return saveOrGet(grid.startNodesAsync(file, restart, timeout, maxConn));
+        boolean restart,
+        int timeout,
+        int maxConn)
+    {
+        try {
+            return saveOrGet(grid.startNodesAsync(file, restart, timeout, maxConn));
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override 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 {
-        return saveOrGet(grid.startNodesAsync(hosts, dflts, restart, timeout, maxConn));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stopNodes() throws IgniteCheckedException {
+        Collection<Map<String, Object>> hosts,
+        @Nullable Map<String, Object> dflts,
+        boolean restart,
+        int timeout,
+        int maxConn)
+    {
+        try {
+            return saveOrGet(grid.startNodesAsync(hosts, dflts, restart, timeout, maxConn));
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopNodes() {
         grid.stopNodes();
     }
 
     /** {@inheritDoc} */
-    @Override public void stopNodes(Collection<UUID> ids) throws IgniteCheckedException {
+    @Override public void stopNodes(Collection<UUID> ids) {
         grid.stopNodes(ids);
     }
 
     /** {@inheritDoc} */
-    @Override public void restartNodes() throws IgniteCheckedException {
+    @Override public void restartNodes() {
         grid.restartNodes();
     }
 
     /** {@inheritDoc} */
-    @Override public void restartNodes(Collection<UUID> ids) throws IgniteCheckedException {
+    @Override public void restartNodes(Collection<UUID> ids) {
         grid.restartNodes(ids);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 841c848..ab5f843 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2613,9 +2613,16 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<GridTuple3<String, Boolean, String>> startNodes(File file, boolean restart,
-        int timeout, int maxConn) throws IgniteCheckedException {
-        return startNodesAsync(file, restart, timeout, maxConn).get();
+    @Override public Collection<GridTuple3<String, Boolean, String>> startNodes(File file,
+        boolean restart,
+        int timeout,
+        int maxConn) {
+        try {
+            return startNodesAsync(file, restart, timeout, maxConn).get();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /**
@@ -2624,17 +2631,25 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
      * @param timeout Connection timeout.
      * @param maxConn Number of parallel SSH connections to one host.
      * @return Future with results.
-     * @throws IgniteCheckedException In case of error.
      * @see {@link IgniteCluster#startNodes(java.io.File, boolean, int, int)}.
      */
-    IgniteInternalFuture<Collection<GridTuple3<String, Boolean, String>>> startNodesAsync(File file, boolean restart,                                                                                            int timeout, int maxConn) throws IgniteCheckedException {
+    IgniteInternalFuture<Collection<GridTuple3<String, Boolean, String>>> startNodesAsync(File file,
+        boolean restart,
+        int timeout,
+        int maxConn)
+    {
         A.notNull(file, "file");
         A.ensure(file.exists(), "file doesn't exist.");
         A.ensure(file.isFile(), "file is a directory.");
 
-        IgniteBiTuple<Collection<Map<String, Object>>, Map<String, Object>> t = parseFile(file);
+        try {
+            IgniteBiTuple<Collection<Map<String, Object>>, Map<String, Object>> t = parseFile(file);
 
-        return startNodesAsync(t.get1(), t.get2(), restart, timeout, maxConn);
+            return startNodesAsync(t.get1(), t.get2(), restart, timeout, maxConn);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFuture<>(ctx, e);
+        }
     }
 
     /** {@inheritDoc} */
@@ -2659,9 +2674,13 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
         boolean restart,
         int timeout,
         int maxConn)
-        throws IgniteCheckedException
     {
-        return startNodesAsync(hosts, dflts, restart, timeout, maxConn).get();
+        try {
+            return startNodesAsync(hosts, dflts, restart, timeout, maxConn).get();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /**
@@ -2671,7 +2690,6 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
      * @param timeout Connection timeout in milliseconds.
      * @param maxConn Number of parallel SSH connections to one host.
      * @return Future with results.
-     * @throws IgniteCheckedException In case of error.
      * @see {@link IgniteCluster#startNodes(java.util.Collection, java.util.Map, boolean, int, int)}.
      */
     IgniteInternalFuture<Collection<GridTuple3<String, Boolean, String>>> startNodesAsync(
@@ -2680,7 +2698,6 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
         boolean restart,
         int timeout,
         int maxConn)
-        throws IgniteCheckedException
     {
         A.notNull(hosts, "hosts");
 
@@ -2775,6 +2792,9 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
 
             return fut;
         }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFuture<>(ctx, e);
+        }
         finally {
             unguard();
         }
@@ -2837,7 +2857,7 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public void stopNodes() throws IgniteCheckedException {
+    @Override public void stopNodes() {
         guard();
 
         try {
@@ -2849,7 +2869,7 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public void stopNodes(Collection<UUID> ids) throws IgniteCheckedException {
+    @Override public void stopNodes(Collection<UUID> ids) {
         guard();
 
         try {
@@ -2861,7 +2881,7 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public void restartNodes() throws IgniteCheckedException {
+    @Override public void restartNodes() {
         guard();
 
         try {
@@ -2873,7 +2893,7 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public void restartNodes(Collection<UUID> ids) throws IgniteCheckedException {
+    @Override public void restartNodes(Collection<UUID> ids) {
         guard();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index da9fba8..c33083f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -304,6 +304,10 @@ public abstract class IgniteUtils {
     /** Mutex. */
     private static final Object mux = new Object();
 
+    /** Exception converters. */
+    private static final Map<Class<? extends IgniteCheckedException>, C1<IgniteCheckedException, IgniteException>>
+        exceptionConverters;
+
     /**
      * Initializes enterprise check.
      */
@@ -542,6 +546,108 @@ public abstract class IgniteUtils {
         PORTABLE_CLS.add(UUID[].class);
         PORTABLE_CLS.add(Date[].class);
         PORTABLE_CLS.add(Timestamp[].class);
+
+        exceptionConverters = Collections.unmodifiableMap(exceptionConverters());
+    }
+
+
+    /**
+     * Gets map with converters to convert internal checked exceptions to public API unchecked exceptions.
+     *
+     * @return Exception converters.
+     */
+    private static Map<Class<? extends IgniteCheckedException>, C1<IgniteCheckedException, IgniteException>> exceptionConverters() {
+        Map<Class<? extends IgniteCheckedException>, C1<IgniteCheckedException, IgniteException>> m = new HashMap<>();
+
+        m.put(IgniteInterruptedCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteInterruptedException(e.getMessage(), (InterruptedException)e.getCause());
+            }
+        });
+
+        m.put(IgniteFutureCancelledCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteFutureCancelledException(e.getMessage(), e);
+            }
+        });
+
+        m.put(IgniteFutureTimeoutCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteFutureTimeoutException(e.getMessage(), e);
+            }
+        });
+
+        m.put(ClusterGroupEmptyCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new ClusterGroupEmptyException(e.getMessage(), e);
+            }
+        });
+
+        m.put(ClusterTopologyCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new ClusterTopologyException(e.getMessage(), e);
+            }
+        });
+
+        m.put(IgniteDeploymentCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteDeploymentException(e.getMessage(), e);
+            }
+        });
+
+        m.put(ComputeTaskTimeoutCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new ComputeTaskTimeoutException(e.getMessage(), e);
+            }
+        });
+
+        m.put(ComputeTaskCancelledCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new ComputeTaskCancelledException(e.getMessage(), e);
+            }
+        });
+
+        m.put(IgniteTxRollbackCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteTxRollbackException(e.getMessage(), e);
+            }
+        });
+
+        m.put(IgniteTxHeuristicCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteTxHeuristicException(e.getMessage(), e);
+            }
+        });
+
+        m.put(IgniteTxTimeoutCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteTxTimeoutException(e.getMessage(), e);
+            }
+        });
+
+        m.put(IgniteTxOptimisticCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteTxOptimisticException(e.getMessage(), e);
+            }
+        });
+
+        return m;
+    }
+
+    /**
+     * @param e Ignite checked exception.
+     * @return Ignite runtime exception.
+     */
+    public static IgniteException convertException(IgniteCheckedException e) {
+        C1<IgniteCheckedException, IgniteException> converter = exceptionConverters.get(e.getClass());
+
+        if (converter != null)
+            return converter.apply(e);
+
+        if (e.getCause() instanceof IgniteException)
+            return (IgniteException)e.getCause();
+
+        return new IgniteException(e.getMessage(), e);
     }
 
     /**
@@ -9147,39 +9253,4 @@ public abstract class IgniteUtils {
 
         return sb.toString();
     }
-
-    /**
-     * @param e Ignite checked exception.
-     * @return Ignite runtime exception.
-     */
-    public static IgniteException convertException(IgniteCheckedException e) {
-        if (e instanceof IgniteInterruptedCheckedException)
-            return new IgniteInterruptedException(e.getMessage(), (InterruptedException)e.getCause());
-        else if (e instanceof IgniteFutureCancelledCheckedException)
-            return new IgniteFutureCancelledException(e.getMessage(), e.getCause());
-        else if (e instanceof IgniteFutureTimeoutCheckedException)
-            return new IgniteFutureTimeoutException(e.getMessage(), e.getCause());
-        else if (e instanceof ClusterGroupEmptyCheckedException)
-            return new ClusterGroupEmptyException(e.getMessage(), e.getCause());
-        else if (e instanceof ClusterTopologyCheckedException)
-            return new ClusterTopologyException(e.getMessage(), e.getCause());
-        else if (e instanceof IgniteDeploymentCheckedException)
-            return new IgniteDeploymentException(e.getMessage(), e.getCause());
-        else if (e instanceof ComputeTaskTimeoutCheckedException)
-            return new ComputeTaskTimeoutException(e.getMessage(), e.getCause());
-        else if (e instanceof ComputeTaskCancelledCheckedException)
-            return new ComputeTaskCancelledException(e.getMessage(), e.getCause());
-        else if (e instanceof IgniteTxRollbackCheckedException)
-            return new IgniteTxRollbackException(e.getMessage(), e.getCause());
-        else if (e instanceof IgniteTxHeuristicCheckedException)
-            return new IgniteTxHeuristicException(e.getMessage(), e.getCause());
-        else if (e instanceof IgniteTxTimeoutCheckedException)
-            return new IgniteTxTimeoutException(e.getMessage(), e.getCause());
-        else if (e instanceof IgniteTxOptimisticCheckedException)
-            return new IgniteTxOptimisticException(e.getMessage(), e.getCause());
-        else if (e.getCause() instanceof IgniteException)
-            return (IgniteException)e.getCause();
-
-        return new IgniteException(e.getMessage(), e);
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorValidationSelfTest.java
index 32d5dd1..98ee4c5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorValidationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorValidationSelfTest.java
@@ -469,7 +469,8 @@ public class GridGgfsProcessorValidationSelfTest extends GridGgfsCommonAbstractT
         catch (IgniteException e) {
             if (testLoc) {
                 if ("Failed to start processor: GridProcessorAdapter []".equals(e.getMessage()) &&
-                    e.getCause().getMessage().contains(excMsgSnippet))
+                    (e.getCause().getMessage().contains(excMsgSnippet) ||
+                     e.getCause().getCause().getMessage().contains(excMsgSnippet)))
                     return; // Expected exception.
             }
             else if (e.getMessage().contains(excMsgSnippet))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
index e37e385..74ae612 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
@@ -1064,22 +1064,22 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public void stopNodes() throws IgniteCheckedException {
+        @Override public void stopNodes() {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void stopNodes(Collection<UUID> ids) throws IgniteCheckedException {
+        @Override public void stopNodes(Collection<UUID> ids) {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void restartNodes() throws IgniteCheckedException {
+        @Override public void restartNodes() {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void restartNodes(Collection<UUID> ids) throws IgniteCheckedException {
+        @Override public void restartNodes(Collection<UUID> ids) {
             // No-op.
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
index 3014f6e..55e8017 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
@@ -26,7 +26,7 @@ import org.springframework.context.*;
 import java.net.*;
 
 /**
- * Factory methods to start GridGain with optional Spring application context, this context can be injected into
+ * Factory methods to start Ignite with optional Spring application context, this context can be injected into
  * grid tasks and grid jobs using {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource}
  * annotation.
  * <p>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
index 837b424..8b74782 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
@@ -224,7 +224,7 @@ class VisorAlertCommand {
         if (expr.isDefined)
             (n: ClusterNode) => f(n) && expr.get.apply(value(n))
         else
-            throw new IgniteCheckedException("Invalid expression: " + exprStr)
+            throw new IgniteException("Invalid expression: " + exprStr)
     }
 
     /**
@@ -243,7 +243,7 @@ class VisorAlertCommand {
         if (expr.isDefined)
             () => f() && expr.get.apply(value())
         else
-            throw new IgniteCheckedException("Invalid expression: " + exprStr)
+            throw new IgniteException("Invalid expression: " + exprStr)
     }
 
     /**
@@ -299,7 +299,7 @@ class VisorAlertCommand {
                             // Other tags.
                             case "t" if v != null => freq = v.toLong
                             case "r" => () // Skipping.
-                            case _ => throw new IgniteCheckedException("Invalid argument: " + makeArg(arg))
+                            case _ => throw new IgniteException("Invalid argument: " + makeArg(arg))
                         }
                     })
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 92a5b46..575ce82 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -481,7 +481,7 @@ class VisorCacheCommand {
                 new IgniteBiTuple(new JavaBoolean(name.isEmpty), name.orNull))).toList
         }
         catch {
-            case e: IgniteCheckedException => Nil
+            case e: IgniteException => Nil
         }
     }
 
@@ -496,7 +496,7 @@ class VisorCacheCommand {
             grid.compute(grid.forNode(node)).withNoFailover()
                 .execute(classOf[VisorNodeConfigurationCollectorTask], emptyTaskArgument(node.id()))
         catch {
-            case e: IgniteCheckedException =>
+            case e: IgniteException =>
                 scold(e.getMessage)
 
                 null

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
index 4df0c35..3a76a5a 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
@@ -212,7 +212,7 @@ class VisorConfigurationCommand {
                     .withNoFailover()
                     .execute(classOf[VisorNodeConfigurationCollectorTask], emptyTaskArgument(node.id()))
             catch {
-                case e: IgniteCheckedException =>
+                case e: IgniteException =>
                     scold(e.getMessage)
 
                     break()

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
index 79a98e5..21d02d0 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
@@ -298,7 +298,7 @@ class VisorEventsCommand {
                 grid.compute(grid.forNode(node)).execute(classOf[VisorNodeEventsCollectorTask],
                     toTaskArgument(nid, VisorNodeEventsCollectorTaskArg.createEventsArg(tpFilter, tmFilter)))
             catch {
-                case e: IgniteCheckedException =>
+                case e: IgniteException =>
                     scold(e.getMessage)
 
                     return

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
index 14e62f6..aafee99 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
@@ -166,8 +166,8 @@ class VisorGcCommand {
                 t.render()
             }
             catch {
-                case e: ClusterGroupEmptyCheckedException => scold("Topology is empty.")
-                case e: IgniteCheckedException => scold(e.getMessage)
+                case e: ClusterGroupEmptyException => scold("Topology is empty.")
+                case e: IgniteException => scold(e.getMessage)
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
index b4b90e4..12fcfa8 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
@@ -179,7 +179,7 @@ class VisorKillCommand {
                     if (restart && node != null && node.attribute[String](ATTR_RESTART_ENABLED) != "true")
                         scold("Node doesn't support restart: " + nid8(node)).^^
                 catch {
-                    case e: IgniteCheckedException => scold("Failed to restart the node. " + e.getMessage).^^
+                    case e: IgniteException => scold("Failed to restart the node. " + e.getMessage).^^
                 }
 
                 val op = if (restart) "restart" else "kill"
@@ -187,7 +187,7 @@ class VisorKillCommand {
                 try
                     killOrRestart(if (node == null) grid.nodes().map(_.id()) else Collections.singleton(node.id()), restart)
                 catch {
-                    case _: IgniteCheckedException => scold("Failed to " + op + " due to system error.").^^
+                    case _: IgniteException => scold("Failed to " + op + " due to system error.").^^
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
index d4affc1..69d68fd 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
@@ -233,7 +233,7 @@ class VisorStartCommand {
                         Result(t.get1, t.get2, t.get3)
                     }).toSeq
                 catch {
-                    case e: IgniteCheckedException => scold(e.getMessage).^^
+                    case e: IgniteException => scold(e.getMessage).^^
                     case _: RejectedExecutionException => scold("Failed due to system error.").^^
                 }
             }
@@ -288,7 +288,7 @@ class VisorStartCommand {
                     res = grid.startNodes(asJavaCollection(Seq(params)), null, restart, timeout, maxConn).
                         map(t => Result(t.get1, t.get2, t.get3)).toSeq
                 catch {
-                    case e: IgniteCheckedException => scold(e.getMessage).^^
+                    case e: IgniteException => scold(e.getMessage).^^
                     case _: RejectedExecutionException => scold("Failed due to system error.").^^
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
index 3486b95..fd5e0b3 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
@@ -785,7 +785,7 @@ class VisorTasksCommand {
                 taskFootnote()
             }
             catch {
-                case e: IgniteCheckedException =>
+                case e: IgniteException =>
                     scold(e.getMessage)
 
                     break()
@@ -978,7 +978,7 @@ class VisorTasksCommand {
                 }
             }
             catch {
-                case e: IgniteCheckedException =>
+                case e: IgniteException =>
                     scold(e.getMessage)
 
                     break()
@@ -1110,7 +1110,7 @@ class VisorTasksCommand {
                 evtsT.render()
             }
             catch {
-                case e: IgniteCheckedException =>
+                case e: IgniteException =>
                     scold(e.getMessage)
 
                     break()
@@ -1222,7 +1222,7 @@ class VisorTasksCommand {
                 taskFootnote()
             }
             catch {
-                case e: IgniteCheckedException =>
+                case e: IgniteException =>
                     scold(e.getMessage)
 
                     break()
@@ -1338,7 +1338,7 @@ class VisorTasksCommand {
                 taskFootnote()
             }
             catch {
-                case e: IgniteCheckedException =>
+                case e: IgniteException =>
                     scold(e.getMessage)
 
                     break()

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
index 9f6b637..6615ec8 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
@@ -192,7 +192,7 @@ class VisorTopologyCommand {
             }
             catch {
                 case e: NumberFormatException => scold(e.getMessage)
-                case e: IgniteCheckedException => scold(e.getMessage)
+                case e: IgniteException => scold(e.getMessage)
             }
         }
     }
@@ -213,7 +213,7 @@ class VisorTopologyCommand {
         if (expr.isDefined)
             (n: ClusterNode) => f(n) && expr.get.apply(v(n))
         else
-            throw new IgniteCheckedException("Invalid expression: " + exprStr)
+            throw new IgniteException("Invalid expression: " + exprStr)
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1ac5a5e/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index 18d4e6a..3b39821 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -235,18 +235,18 @@ object visor extends VisorTag {
      *
      * @param nid Node ID.
      * @return GridNode instance.
-     * @throws IgniteCheckedException if Visor is disconnected or node not found.
+     * @throws IgniteException if Visor is disconnected or node not found.
      */
     def node(nid: UUID): ClusterNode = {
         val g = grid
 
         if (g == null)
-            throw new IgniteCheckedException("Visor disconnected")
+            throw new IgniteException("Visor disconnected")
         else {
             val node = g.node(nid)
 
             if (node == null)
-                throw new IgniteCheckedException("Node is gone: " + nid)
+                throw new IgniteException("Node is gone: " + nid)
 
             node
         }
@@ -1478,7 +1478,7 @@ object visor extends VisorTag {
                             val url = IgniteUtils.resolveGridGainUrl(path)
 
                             if (url == null)
-                                throw new IgniteCheckedException("GridGain configuration path is invalid: " + path, e)
+                                throw new IgniteException("Ignite configuration path is invalid: " + path, e)
 
                             url
                     }
@@ -1503,10 +1503,10 @@ object visor extends VisorTag {
                     }
 
                 if (cfgs == null || cfgs.isEmpty)
-                    throw new IgniteCheckedException("Can't find grid configuration in: " + url)
+                    throw new IgniteException("Can't find grid configuration in: " + url)
 
                 if (cfgs.size > 1)
-                    throw new IgniteCheckedException("More than one grid configuration found in: " + url)
+                    throw new IgniteException("More than one grid configuration found in: " + url)
 
                 val cfg = cfgs.iterator().next()
 
@@ -1565,7 +1565,7 @@ object visor extends VisorTag {
             open(cfg, cfgPath)
         }
         catch {
-            case e: IgniteCheckedException =>
+            case e: IgniteException =>
                 warn(e.getMessage)
                 warn("Type 'help open' to see how to use this command.")
 
@@ -1608,7 +1608,7 @@ object visor extends VisorTag {
                 case _: IllegalStateException =>
                     this.cfgPath = null
 
-                    throw new IgniteCheckedException("Named grid unavailable: " + startedGridName)
+                    throw new IgniteException("Named grid unavailable: " + startedGridName)
             }
 
         assert(cfgPath != null)