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/05 11:03:04 UTC

[03/32] incubator-ignite git commit: # Renaming

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
index 83beb7f..a0e5fe3 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
@@ -148,7 +148,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                                     sendWithRetries(nodeId, req, null);
                                 }
                             }
-                            catch (GridTopologyException ignored) {
+                            catch (ClusterTopologyException ignored) {
                                 if (log.isDebugEnabled())
                                     log.debug("Failed to send pending start request to node (is node alive?): " +
                                         nodeId);
@@ -420,7 +420,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             // Stop with exception if projection is empty.
             if (nodes.isEmpty() && !locIncluded) {
                 return new GridFinishedFuture<>(ctx,
-                    new GridTopologyException("Failed to register remote continuous listener (projection is empty)."));
+                    new ClusterTopologyException("Failed to register remote continuous listener (projection is empty)."));
             }
 
             // IDs of nodes where request will be sent.
@@ -597,7 +597,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                         try {
                             sendWithRetries(node.id(), new GridContinuousMessage(MSG_STOP_REQ, routineId, null), null);
                         }
-                        catch (GridTopologyException ignored) {
+                        catch (ClusterTopologyException ignored) {
                             U.warn(log, "Failed to send stop request (node left topology): " + node.id());
                         }
                     }
@@ -723,7 +723,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         try {
             sendWithRetries(nodeId, new GridContinuousMessage(MSG_START_ACK, routineId, err), null);
         }
-        catch (GridTopologyException ignored) {
+        catch (ClusterTopologyException ignored) {
             if (log.isDebugEnabled())
                 log.debug("Failed to send start acknowledgement to node (is node alive?): " + nodeId);
         }
@@ -784,7 +784,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         try {
             sendWithRetries(nodeId, new GridContinuousMessage(MSG_STOP_ACK, routineId, null), null);
         }
-        catch (GridTopologyException ignored) {
+        catch (ClusterTopologyException ignored) {
             if (log.isDebugEnabled())
                 log.debug("Failed to send stop acknowledgement to node (is node alive?): " + nodeId);
         }
@@ -916,7 +916,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                                 try {
                                     sendNotification(nodeId, routineId, toSnd, hnd.orderedTopic());
                                 }
-                                catch (GridTopologyException ignored) {
+                                catch (ClusterTopologyException ignored) {
                                     if (log.isDebugEnabled())
                                         log.debug("Failed to send notification to node (is node alive?): " + nodeId);
                                 }
@@ -996,7 +996,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         if (node != null)
             sendWithRetries(node, msg, orderedTopic);
         else
-            throw new GridTopologyException("Node for provided ID doesn't exist (did it leave the grid?): " + nodeId);
+            throw new ClusterTopologyException("Node for provided ID doesn't exist (did it leave the grid?): " + nodeId);
     }
 
     /**
@@ -1062,7 +1062,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 }
                 catch (GridException e) {
                     if (!ctx.discovery().alive(node.id()))
-                        throw new GridTopologyException("Node left grid while sending message to: " + node.id(), e);
+                        throw new ClusterTopologyException("Node left grid while sending message to: " + node.id(), e);
 
                     if (cnt == retryCnt)
                         throw e;
@@ -1683,7 +1683,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                             try {
                                 sendWithRetries(id, req, null);
                             }
-                            catch (GridTopologyException ignored) {
+                            catch (ClusterTopologyException ignored) {
                                 if (log.isDebugEnabled())
                                     log.debug("Failed to resend stop request to node (is node alive?): " + id);
                             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
index a53c63e..37e011a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
@@ -439,7 +439,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
             }
 
             if (node == null) {
-                resFut.onDone(new GridTopologyException("Failed to map key to node " +
+                resFut.onDone(new ClusterTopologyException("Failed to map key to node " +
                     "(no nodes with cache found in topology) [infos=" + entries.size() +
                     ", cacheName=" + cacheName + ']'));
 
@@ -509,7 +509,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                     buf.onNodeLeft();
 
                 if (f != null)
-                    f.onDone(new GridTopologyException("Failed to wait for request completion " +
+                    f.onDone(new ClusterTopologyException("Failed to wait for request completion " +
                         "(node has left): " + nodeId));
             }
         }
@@ -1011,7 +1011,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                     if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
                         ((GridFutureAdapter<Object>)fut).onDone(e);
                     else
-                        ((GridFutureAdapter<Object>)fut).onDone(new GridTopologyException("Failed to send " +
+                        ((GridFutureAdapter<Object>)fut).onDone(new ClusterTopologyException("Failed to send " +
                             "request (node has left): " + node.id()));
                 }
             }
@@ -1027,7 +1027,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
             if (log.isDebugEnabled())
                 log.debug("Forcibly completing futures (node has left): " + node.id());
 
-            Exception e = new GridTopologyException("Failed to wait for request completion " +
+            Exception e = new ClusterTopologyException("Failed to wait for request completion " +
                 "(node has left): " + node.id());
 
             for (GridFutureAdapter<Object> f : reqs.values())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
index 19b5c4e..d495693 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
@@ -184,7 +184,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                 if (ggfsCtx.ggfsNode(discoEvt.eventNode())) {
                     for (WriteCompletionFuture future : pendingWrites.values()) {
                         future.onError(discoEvt.eventNode().id(),
-                            new GridTopologyException("Node left grid before write completed: " + evt.node().id()));
+                            new ClusterTopologyException("Node left grid before write completed: " + evt.node().id()));
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
index e2c7d11..08b922a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
@@ -160,7 +160,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
             }
             catch (GridException e) {
                 if (!ggfsCtx.kernalContext().discovery().alive(nodeId))
-                    throw new GridTopologyException("Failed to send message (node left the grid) " +
+                    throw new ClusterTopologyException("Failed to send message (node left the grid) " +
                         "[nodeId=" + nodeId + ", msg=" + msg + ']');
 
                 if (i == MESSAGE_SEND_RETRY_COUNT - 1)
@@ -568,7 +568,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
                             startSync0.remove(nodeId);
                     }
                     catch (GridException e) {
-                        if (e.hasCause(GridTopologyException.class)) {
+                        if (e.hasCause(ClusterTopologyException.class)) {
                             if (log.isDebugEnabled())
                                 log.debug("Failed to send sync message to remote node (node has left the grid): " +
                                     nodeId);
@@ -644,7 +644,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
                     sendWithRetries(nodeId, msg);
                 }
                 catch (GridException e) {
-                    if (e.hasCause(GridTopologyException.class)) {
+                    if (e.hasCause(ClusterTopologyException.class)) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to send fragmentizer request to remote node (node left grid): " +
                                 nodeId);
@@ -739,7 +739,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
                             processFragmentizerRequest(fragmentizerReq);
                         }
                         catch (GridException e) {
-                            if (e.hasCause(GridTopologyException.class)) {
+                            if (e.hasCause(ClusterTopologyException.class)) {
                                 if (log.isDebugEnabled())
                                     log.debug("Failed to process fragmentizer request (remote node left the grid) " +
                                         "[req=" + req + ", err=" + e.getMessage() + ']');
@@ -785,7 +785,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
                 sendWithRetries(nodeId, msg);
             }
             catch (GridException e) {
-                if (e.hasCause(GridTopologyException.class)) {
+                if (e.hasCause(ClusterTopologyException.class)) {
                     if (log.isDebugEnabled())
                         log.debug("Failed to send sync response to GGFS fragmentizer coordinator " +
                             "(originating node left the grid): " + nodeId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
index 1f2a7b5..bd54401 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -228,7 +228,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                                 desc = new TaskDescriptor(true, f.get(), null);
                             }
                             catch (GridException e) {
-                                if (e.hasCause(GridTopologyException.class, GridEmptyProjectionException.class))
+                                if (e.hasCause(ClusterTopologyException.class, ClusterGroupEmptyException.class))
                                     U.warn(log, "Failed to execute task due to topology issues (are all mapped " +
                                         "nodes alive?) [name=" + name + ", clientId=" + req.clientId() +
                                         ", err=" + e + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessor.java
index cb4d0b0..29596ac 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessor.java
@@ -380,12 +380,12 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
                 return fut;
             }
-            catch (GridTopologyException e) {
+            catch (ClusterTopologyException e) {
                 if (log.isDebugEnabled())
                     log.debug("Topology changed while deploying service (will retry): " + e.getMessage());
             }
             catch (GridException e) {
-                if (e.hasCause(GridTopologyException.class)) {
+                if (e.hasCause(ClusterTopologyException.class)) {
                     if (log.isDebugEnabled())
                         log.debug("Topology changed while deploying service (will retry): " + e.getMessage());
 
@@ -425,7 +425,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
                 return fut;
             }
-            catch (GridTopologyException e) {
+            catch (ClusterTopologyException e) {
                 if (log.isDebugEnabled())
                     log.debug("Topology changed while deploying service (will retry): " + e.getMessage());
             }
@@ -726,7 +726,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
                 break;
             }
-            catch (GridTopologyException e) {
+            catch (ClusterTopologyException e) {
                 if (log.isDebugEnabled())
                     log.debug("Topology changed while reassigning (will retry): " + e.getMessage());
 
@@ -999,7 +999,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                     reassign(dep, topVer);
             }
             catch (GridException e) {
-                if (!(e instanceof GridTopologyException))
+                if (!(e instanceof ClusterTopologyException))
                     log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e);
 
                 long newTopVer = ctx.discovery().topologyVersion();
@@ -1079,7 +1079,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                         reassign(dep, topVer);
                                     }
                                     catch (GridException ex) {
-                                        if (!(e instanceof GridTopologyException))
+                                        if (!(e instanceof ClusterTopologyException))
                                             LT.error(log, ex, "Failed to do service reassignment (will retry): " +
                                                 dep.configuration().getName());
 
@@ -1149,7 +1149,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                         it.remove();
                     }
                     catch (GridException e) {
-                        if (!(e instanceof GridTopologyException))
+                        if (!(e instanceof ClusterTopologyException))
                             LT.error(log, e, "Failed to do service reassignment (will retry): " +
                                 dep.configuration().getName());
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProxy.java
index 4bae4d6..3c668dc 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProxy.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceProxy.java
@@ -144,7 +144,7 @@ class GridServiceProxy<T> implements Serializable {
                         ).get();
                     }
                 }
-                catch (GridServiceNotFoundException | GridTopologyException e) {
+                catch (GridServiceNotFoundException | ClusterTopologyException e) {
                     if (log.isDebugEnabled())
                         log.debug("Service was not found or topology changed (will retry): " + e.getMessage());
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/GridStreamerStageExecutionFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/GridStreamerStageExecutionFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/GridStreamerStageExecutionFuture.java
index 2cc2675..7529a1d 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/GridStreamerStageExecutionFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/GridStreamerStageExecutionFuture.java
@@ -295,7 +295,7 @@ public class GridStreamerStageExecutionFuture extends GridFutureAdapter<Object>
      */
     public void onNodeLeft(UUID leftNodeId) {
         if (execNodeIds.contains(leftNodeId))
-            onFailed(leftNodeId, new GridTopologyException("Failed to wait for streamer pipeline future completion " +
+            onFailed(leftNodeId, new ClusterTopologyException("Failed to wait for streamer pipeline future completion " +
                 "(execution node has left the grid). All running stages will be cancelled " +
                 "[fut=" + this + ", leftNodeId=" + leftNodeId + ']'));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/IgniteStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/IgniteStreamerImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/IgniteStreamerImpl.java
index f9546d9..9c12916 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/IgniteStreamerImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/streamer/IgniteStreamerImpl.java
@@ -805,7 +805,7 @@ public class IgniteStreamerImpl implements IgniteStreamerEx, Externalizable {
                     sendWithRetries(dstNodeId, new GridStreamerResponse(futId, errBytes));
                 }
                 catch (GridException e) {
-                    if (!e.hasCause(GridTopologyException.class))
+                    if (!e.hasCause(ClusterTopologyException.class))
                         log.error("Failed to complete parent stage [futId=" + futId + ", err=" + e + ']');
                 }
             }
@@ -860,7 +860,7 @@ public class IgniteStreamerImpl implements IgniteStreamerEx, Externalizable {
                 sendWithRetries(nodeId, new GridStreamerCancelRequest(cancelledFutId));
             }
             catch (GridException e) {
-                if (!e.hasCause(GridTopologyException.class))
+                if (!e.hasCause(ClusterTopologyException.class))
                     log.error("Failed to send streamer cancel request to remote node [nodeId=" + nodeId +
                         ", cancelledFutId=" + cancelledFutId + ']', e);
             }
@@ -1115,7 +1115,7 @@ public class IgniteStreamerImpl implements IgniteStreamerEx, Externalizable {
                         ", msg=" + msg + ", err=" + e + ']');
 
                 if (!ctx.discovery().alive(dstNodeId))
-                    throw new GridTopologyException("Failed to send message (destination node left grid): " +
+                    throw new ClusterTopologyException("Failed to send message (destination node left grid): " +
                         dstNodeId);
 
                 if (i == SEND_RETRY_COUNT - 1)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskWorker.java
index 38f7007..6165fbd 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskWorker.java
@@ -413,7 +413,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
 
             processDelayedResponses();
         }
-        catch (GridEmptyProjectionException e) {
+        catch (ClusterGroupEmptyException e) {
             U.warn(log, "Failed to map task jobs to nodes (topology projection is empty): " + ses);
 
             finishTask(null, e);
@@ -534,7 +534,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         int size = subgrid.size();
 
         if (size == 0)
-            throw new GridEmptyProjectionException("Topology projection is empty.");
+            throw new ClusterGroupEmptyException("Topology projection is empty.");
 
         List<ClusterNode> shuffledNodes = new ArrayList<>(size);
 
@@ -922,7 +922,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
 
             recordTaskEvent(EVT_TASK_REDUCED, "Task reduced.");
         }
-        catch (GridTopologyException e) {
+        catch (ClusterTopologyException e) {
             U.warn(log, "Failed to reduce job results for task (any nodes from task topology left grid?): " + task);
 
             userE = e;
@@ -968,7 +968,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                 if (log.isDebugEnabled())
                     log.debug(msg);
 
-                Throwable e = new GridTopologyException(msg, jobRes.getException());
+                Throwable e = new ClusterTopologyException(msg, jobRes.getException());
 
                 finishTask(null, e);
 
@@ -1094,7 +1094,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                 GridJobExecuteResponse fakeRes = new GridJobExecuteResponse(node.id(), ses.getId(),
                     res.getJobContext().getJobId(), null, null, null, null, null, null, false);
 
-                fakeRes.setFakeException(new GridTopologyException("Failed to send job due to node failure: " + node));
+                fakeRes.setFakeException(new ClusterTopologyException("Failed to send job due to node failure: " + node));
 
                 onResponse(fakeRes);
             }
@@ -1197,7 +1197,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                 res.getJobContext().getJobId(), null, null, null, null, null, null, false);
 
             if (deadNode)
-                fakeRes.setFakeException(new GridTopologyException("Failed to send job due to node failure: " +
+                fakeRes.setFakeException(new ClusterTopologyException("Failed to send job due to node failure: " +
                     node, e));
             else
                 fakeRes.setFakeException(e);
@@ -1229,7 +1229,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                         GridJobExecuteResponse fakeRes = new GridJobExecuteResponse(nodeId, ses.getId(),
                             jr.getJobContext().getJobId(), null, null, null, null, null, null, false);
 
-                        fakeRes.setFakeException(new GridTopologyException("Node has left grid: " + nodeId));
+                        fakeRes.setFakeException(new ClusterTopologyException("Node has left grid: " + nodeId));
 
                         if (resList == null)
                             resList = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeDataCollectorTask.java
index 68dc315..6939b25 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeDataCollectorTask.java
@@ -70,7 +70,7 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
                     reduceJobResult(taskResult, jobResult, nid);
                 else {
                     // Ignore nodes that left topology.
-                    if (!(unhandledEx instanceof GridEmptyProjectionException))
+                    if (!(unhandledEx instanceof ClusterGroupEmptyException))
                         taskResult.unhandledEx().put(nid, unhandledEx);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodePingTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodePingTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodePingTask.java
index e4725cf..23ffd24 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodePingTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodePingTask.java
@@ -37,9 +37,9 @@ public class VisorNodePingTask extends VisorOneNodeTask<UUID, GridTuple3<Boolean
     @Nullable @Override public GridTuple3<Boolean, Long, Long> reduce(List<ComputeJobResult> results) throws GridException {
         try {
             return super.reduce(results);
-        } catch (GridEmptyProjectionException ignored) {
+        } catch (ClusterGroupEmptyException ignored) {
             return new GridTuple3<>(false, -1L, -1L);
-        } catch (GridTopologyException ignored) {
+        } catch (ClusterTopologyException ignored) {
             return new GridTuple3<>(false, -1L, -1L);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/util/GridUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/GridUtils.java b/modules/core/src/main/java/org/gridgain/grid/util/GridUtils.java
index 6573250..a4c5a71 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/GridUtils.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/GridUtils.java
@@ -4110,8 +4110,8 @@ public abstract class GridUtils {
      *
      * @return Empty projection exception.
      */
-    public static GridEmptyProjectionException emptyTopologyException() {
-        return new GridEmptyProjectionException("Topology projection is empty. Note that predicate based " +
+    public static ClusterGroupEmptyException emptyTopologyException() {
+        return new ClusterGroupEmptyException("Topology projection is empty. Note that predicate based " +
             "projection can be empty from call to call.");
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/main/java/org/gridgain/grid/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/gridgain/grid/util/future/GridCompoundFuture.java
index d06373b..c00b8e2 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/future/GridCompoundFuture.java
@@ -330,7 +330,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
 
                 err.compareAndSet(null, e);
             }
-            catch (GridTopologyException e) {
+            catch (ClusterTopologyException e) {
                 if (log.isDebugEnabled())
                     log.debug("Topology exception [fut=" + GridCompoundFuture.this + ", err=" + e + ']');
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/ClusterMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/ClusterMetricsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/ClusterMetricsSelfTest.java
index 203bc28..bb2f52f 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/ClusterMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/ClusterMetricsSelfTest.java
@@ -65,7 +65,7 @@ public class ClusterMetricsSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (GridEmptyProjectionException e) {
+        catch (ClusterGroupEmptyException e) {
             info("Caught expected exception: " + e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageSelfTest.java
index 3ba454b..531b462 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageSelfTest.java
@@ -133,7 +133,7 @@ public class GridEventStorageSelfTest extends GridCommonAbstractTest {
         try {
             events(ignite1.cluster().forPredicate(F.<ClusterNode>alwaysFalse())).remoteQuery(filter, 0);
         }
-        catch (GridEmptyProjectionException ignored) {
+        catch (ClusterGroupEmptyException ignored) {
             // No-op
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
index 8d79468..9b6b79b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
@@ -86,7 +86,7 @@ public class GridFailoverTaskWithPredicateSelfTest extends GridCommonAbstractTes
 
             compute(ignite1.cluster().forPredicate(p)).withTimeout(10000).execute(JobFailTask.class.getName(), "1");
         }
-        catch (GridTopologyException ignored) {
+        catch (ClusterTopologyException ignored) {
             failed.set(true);
         }
         finally {
@@ -122,7 +122,7 @@ public class GridFailoverTaskWithPredicateSelfTest extends GridCommonAbstractTes
 
             assert res == 1;
         }
-        catch (GridTopologyException ignored) {
+        catch (ClusterTopologyException ignored) {
             failed.set(true);
         }
         finally {
@@ -165,7 +165,7 @@ public class GridFailoverTaskWithPredicateSelfTest extends GridCommonAbstractTes
 
             assert res == 1;
         }
-        catch (GridTopologyException ignored) {
+        catch (ClusterTopologyException ignored) {
             failed.set(true);
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskFailoverSelfTest.java
index fe1d1cf..f961795 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskFailoverSelfTest.java
@@ -49,7 +49,7 @@ public class GridTaskFailoverSelfTest extends GridCommonAbstractTest {
 
             assert false : "Should never be reached due to exception thrown.";
         }
-        catch (GridTopologyException e) {
+        catch (ClusterTopologyException e) {
             info("Received correct exception: " + e);
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index 26b20e3..e9277fc 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -300,7 +300,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         }
         catch (GridException e) {
             // It is ok to fail with topology exception.
-            if (!X.hasCause(e, GridTopologyException.class))
+            if (!X.hasCause(e, ClusterTopologyException.class))
                 throw e;
         }
     }
@@ -327,7 +327,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         }
         catch (GridException e) {
             // It is ok to fail with topology exception.
-            if (!X.hasCause(e, GridTopologyException.class))
+            if (!X.hasCause(e, ClusterTopologyException.class))
                 throw e;
             else
                 info("Failed to put values to cache due to topology exception [0," + cnt + ')');
@@ -346,7 +346,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         }
         catch (GridException e) {
             // It is ok to fail with topology exception.
-            if (!X.hasCause(e, GridTopologyException.class))
+            if (!X.hasCause(e, ClusterTopologyException.class))
                 throw e;
         }
     }
@@ -373,7 +373,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         }
         catch (GridException e) {
             // It is ok to fail with topology exception.
-            if (!X.hasCause(e, GridTopologyException.class))
+            if (!X.hasCause(e, ClusterTopologyException.class))
                 throw e;
             else
                 info("Failed to remove values from cache due to topology exception [0," + cnt + ')');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
index d3363f9..ac77e6b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
@@ -146,7 +146,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
                     break;
                 }
                 catch (Exception e) {
-                    if (X.hasCause(e, GridTopologyException.class)) {
+                    if (X.hasCause(e, ClusterTopologyException.class)) {
                         info("Got topology exception while tear down (will retry in 1000ms).");
 
                         U.sleep(1000);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
index a2c4831..1ab773e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
@@ -494,7 +494,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
                                 try {
                                     cache.put(key, Integer.toString(key));
                                 }
-                                catch (GridCacheTxRollbackException | GridTopologyException ignored) {
+                                catch (GridCacheTxRollbackException | ClusterTopologyException ignored) {
                                     // It is ok if primary node leaves grid.
                                 }
 
@@ -642,11 +642,11 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
                                         tx.commit();
                                     }
-                                    catch (GridTopologyException ignored) {
+                                    catch (ClusterTopologyException ignored) {
                                         // It is ok if primary node leaves grid.
                                     }
                                 }
-                                catch (GridTopologyException ignored) {
+                                catch (ClusterTopologyException ignored) {
                                     // It is ok if primary node leaves grid.
                                 }
 
@@ -787,7 +787,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
                                     tx.commit();
                                 }
-                                catch (GridTopologyException ignored) {
+                                catch (ClusterTopologyException ignored) {
                                     // It is ok if primary node leaves grid.
                                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
index 440dad5..26a44c6 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
@@ -171,7 +171,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
                     @Override public Object call() throws Exception {
                         return nearOnly.get(key);
                     }
-                }, GridTopologyException.class, null);
+                }, ClusterTopologyException.class, null);
             }
 
             // Test optimistic transaction.
@@ -185,7 +185,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
 
                     return null;
                 }
-            }, GridTopologyException.class, null);
+            }, ClusterTopologyException.class, null);
 
             // Test pessimistic transaction.
             GridTestUtils.assertThrows(log, new Callable<Object>() {
@@ -198,7 +198,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
 
                     return null;
                 }
-            }, GridTopologyException.class, null);
+            }, ClusterTopologyException.class, null);
 
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
index 9612585..7851577 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
@@ -46,7 +46,7 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteClosure<Throwable, Throwable> errorHandler() {
         return new IgniteClosure<Throwable, Throwable>() {
             @Override public Throwable apply(Throwable e) {
-                if (e instanceof GridException || X.hasCause(e, GridTopologyException.class)) {
+                if (e instanceof GridException || X.hasCause(e, ClusterTopologyException.class)) {
                     info("Discarding exception: " + e);
 
                     return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 8814648..409944b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -256,7 +256,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
                     return null;
                 }
             },
-            GridTopologyException.class,
+            ClusterTopologyException.class,
             null
         );
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
index b533e0d..5b2c9cd 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
@@ -971,7 +971,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
                         queue.add(F.t(idx, consumeId));
                     }
-                    catch (GridTopologyException ignored) {
+                    catch (ClusterTopologyException ignored) {
                         // No-op.
                     }
 
@@ -1004,7 +1004,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
                         stopped.add(consumeId);
                     }
-                    catch (GridTopologyException ignored) {
+                    catch (ClusterTopologyException ignored) {
                         // No-op.
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureAdapterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureAdapterSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureAdapterSelfTest.java
index 43e6ffd..678a0a6 100644
--- a/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureAdapterSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureAdapterSelfTest.java
@@ -269,14 +269,14 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
             fut = new GridFutureAdapter<>(ctx);
             chain = fut.chain(passThrough);
 
-            fut.onDone(new GridEmptyProjectionException("test exception"));
+            fut.onDone(new ClusterGroupEmptyException("test exception"));
 
             try {
                 chain.get();
 
                 fail("Expects failed with exception.");
             }
-            catch (GridEmptyProjectionException e) {
+            catch (ClusterGroupEmptyException e) {
                 info("Expected exception: " + e.getMessage());
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/scalar/src/main/scala/org/gridgain/scalar/pimps/ScalarProjectionPimp.scala
----------------------------------------------------------------------
diff --git a/modules/scalar/src/main/scala/org/gridgain/scalar/pimps/ScalarProjectionPimp.scala b/modules/scalar/src/main/scala/org/gridgain/scalar/pimps/ScalarProjectionPimp.scala
index 00a1fd4..7c8f0ef 100644
--- a/modules/scalar/src/main/scala/org/gridgain/scalar/pimps/ScalarProjectionPimp.scala
+++ b/modules/scalar/src/main/scala/org/gridgain/scalar/pimps/ScalarProjectionPimp.scala
@@ -11,7 +11,7 @@
 
 package org.gridgain.scalar.pimps
 
-import org.apache.ignite.cluster.{GridEmptyProjectionException, ClusterGroup, ClusterNode}
+import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterGroup, ClusterNode}
 import org.apache.ignite.lang.{IgniteFutureCancelledException, IgniteFuture, IgnitePredicate}
 import org.gridgain.grid._
 import org.jetbrains.annotations._
@@ -202,7 +202,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
         try
             call$(s, p)
         catch {
-            case _: GridEmptyProjectionException => dflt()
+            case _: ClusterGroupEmptyException => dflt()
         }
     }
 
@@ -246,7 +246,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
         try
             call$(Seq(s), p)
         catch {
-            case _: GridEmptyProjectionException => dflt()
+            case _: ClusterGroupEmptyException => dflt()
         }
     }
 
@@ -300,7 +300,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
             run$(s, p)
         }
         catch {
-            case _: GridEmptyProjectionException => if (dflt != null) dflt() else ()
+            case _: ClusterGroupEmptyException => if (dflt != null) dflt() else ()
         }
     }
 
@@ -342,7 +342,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
             run$(s, p)
         }
         catch {
-            case _: GridEmptyProjectionException => if (dflt != null) dflt() else ()
+            case _: ClusterGroupEmptyException => if (dflt != null) dflt() else ()
         }
     }
 
@@ -547,7 +547,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
         try
             reduceAsync$(s, r, p).get
         catch {
-            case _: GridEmptyProjectionException => dflt()
+            case _: ClusterGroupEmptyException => dflt()
         }
     }
 
@@ -589,7 +589,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      *      If `null` - this method is no-op.
      * @param p Optional filtering predicate. If `null` provided - all nodes in this projection will be used for topology.
      * @throws GridException Thrown in case of any error.
-     * @throws GridEmptyProjectionException Thrown in case when this projection is empty.
+     * @throws ClusterGroupEmptyException Thrown in case when this projection is empty.
      *      Note that in case of dynamic projection this method will take a snapshot of all the
      *      nodes at the time of this call, apply all filtering predicates, if any, and if the
      *      resulting collection of nodes is empty - the exception will be thrown.
@@ -633,7 +633,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      *      If `null` - this method is no-op.
      * @param p Optional filtering predicate. If `null` provided - all nodes in this projection will be used for topology.
      * @throws GridException Thrown in case of any error.
-     * @throws GridEmptyProjectionException Thrown in case when this projection is empty.
+     * @throws ClusterGroupEmptyException Thrown in case when this projection is empty.
      *      Note that in case of dynamic projection this method will take a snapshot of all the
      *      nodes at the time of this call, apply all filtering predicates, if any, and if the
      *      resulting collection of nodes is empty - the exception will be thrown.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/visor-console/src/main/scala/org/gridgain/visor/commands/ack/VisorAckCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/ack/VisorAckCommand.scala b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/ack/VisorAckCommand.scala
index f86b696..fc4e62e 100644
--- a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/ack/VisorAckCommand.scala
+++ b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/ack/VisorAckCommand.scala
@@ -11,7 +11,7 @@
 
 package org.gridgain.visor.commands.ack
 
-import org.apache.ignite.cluster.GridEmptyProjectionException
+import org.apache.ignite.cluster.ClusterGroupEmptyException
 import org.gridgain.grid._
 import org.gridgain.grid.kernal.visor.misc.VisorAckTask
 
@@ -109,7 +109,7 @@ class VisorAckCommand {
                     .execute(classOf[VisorAckTask], toTaskArgument(nodeIds, msg))
             }
             catch {
-                case _: GridEmptyProjectionException => scold("Topology is empty.")
+                case _: ClusterGroupEmptyException => scold("Topology is empty.")
                 case e: Exception => scold("System error: " + e.getMessage)
             }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/visor-console/src/main/scala/org/gridgain/visor/commands/gc/VisorGcCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/gc/VisorGcCommand.scala b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/gc/VisorGcCommand.scala
index d51624b..5e0ece4 100644
--- a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/gc/VisorGcCommand.scala
+++ b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/gc/VisorGcCommand.scala
@@ -11,7 +11,7 @@
 
 package org.gridgain.visor.commands.gc
 
-import org.apache.ignite.cluster.{GridEmptyProjectionException, ClusterNode}
+import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode}
 import org.gridgain.grid.kernal.visor.node.VisorNodeGcTask
 
 import java.lang.{Boolean => JavaBoolean}
@@ -157,7 +157,7 @@ class VisorGcCommand {
                 t.render()
             }
             catch {
-                case e: GridEmptyProjectionException => scold("Topology is empty.")
+                case e: ClusterGroupEmptyException => scold("Topology is empty.")
                 case e: GridException => scold(e.getMessage)
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f7f61ed/modules/visor-console/src/main/scala/org/gridgain/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/gridgain/visor/visor.scala b/modules/visor-console/src/main/scala/org/gridgain/visor/visor.scala
index c4608b5..1a64dd4 100644
--- a/modules/visor-console/src/main/scala/org/gridgain/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/gridgain/visor/visor.scala
@@ -14,7 +14,7 @@ package org.gridgain.visor
 import org.apache.ignite.lifecycle.IgniteListener
 import org.apache.ignite.thread.IgniteThreadPoolExecutor
 import org.apache.ignite.{IgniteSystemProperties, IgniteState, Ignition}
-import org.apache.ignite.cluster.{GridEmptyProjectionException, ClusterGroup, ClusterMetrics, ClusterNode}
+import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterGroup, ClusterMetrics, ClusterNode}
 import org.apache.ignite.configuration.IgniteConfiguration
 import org.apache.ignite.events.{IgniteEvent, IgniteDiscoveryEvent, IgniteEventType}
 import org.apache.ignite.lang.IgnitePredicate
@@ -2455,7 +2455,7 @@ object visor extends VisorTag {
                         }
                     }
                     catch {
-                        case _: GridEmptyProjectionException => // Ignore.
+                        case _: ClusterGroupEmptyException => // Ignore.
                         case e: Exception => logText("Failed to collect log.")
                     }
                 }
@@ -2487,7 +2487,7 @@ object visor extends VisorTag {
             try
                 drawBar(g.metrics())
             catch {
-                case e: GridEmptyProjectionException => logText("Topology is empty.")
+                case e: ClusterGroupEmptyException => logText("Topology is empty.")
                 case e: Exception => ()
             }
     }