# 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/7cdd0c17
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/7cdd0c17
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/7cdd0c17

Branch: refs/heads/ignite-26
Commit: 7cdd0c17f3f17afd521a3116c6129d48c4db71a5
Parents: 62cae2d
Author: sboikov <sboi...@gridgain.com>
Authored: Thu Jan 29 14:49:46 2015 +0300
Committer: sboikov <sboi...@gridgain.com>
Committed: Thu Jan 29 17:36:10 2015 +0300

----------------------------------------------------------------------
 .../examples/datagrid/CacheAffinityExample.java |  12 +--
 .../examples/messaging/MessagingExample.java    |   7 +-
 .../messaging/MessagingPingPongExample.java     |   4 +-
 .../java/org/apache/ignite/IgniteCompute.java   |  74 ++++++-------
 .../ignite/IgniteDeploymentException.java       |   2 +-
 .../java/org/apache/ignite/IgniteEvents.java    |  28 ++---
 .../java/org/apache/ignite/IgniteManaged.java   |  29 ++---
 .../java/org/apache/ignite/IgniteMessaging.java |  35 +++---
 .../java/org/apache/ignite/IgniteScheduler.java |  12 +--
 .../main/java/org/apache/ignite/Ignition.java   |  47 +++++---
 .../org/apache/ignite/cache/CacheManager.java   |   2 +-
 .../cluster/ClusterGroupEmptyException.java     |   4 +-
 .../cluster/ClusterTopologyException.java       |   2 +-
 .../apache/ignite/compute/ComputeJobResult.java |   2 +-
 .../org/apache/ignite/compute/ComputeTask.java  |   4 +-
 .../ignite/compute/ComputeTaskAdapter.java      |   6 +-
 .../ignite/compute/ComputeTaskFuture.java       |  16 +--
 .../internal/GridEventConsumeHandler.java       |   4 +-
 .../internal/GridMessageListenHandler.java      |   4 +-
 .../ignite/internal/GridTaskFutureImpl.java     |  33 +++++-
 .../ignite/internal/IgniteComputeImpl.java      |  93 +++++++++++-----
 .../IgniteDeploymentCheckedException.java       |  58 ++++++++++
 .../ignite/internal/IgniteEventsImpl.java       |  22 +++-
 .../apache/ignite/internal/IgniteKernal.java    |   3 +-
 .../ignite/internal/IgniteManagedImpl.java      |  36 +++++--
 .../ignite/internal/IgniteMessagingImpl.java    |  35 ++++--
 .../ignite/internal/IgniteSchedulerImpl.java    |  12 ++-
 .../ClusterGroupEmptyCheckedException.java      |  55 ++++++++++
 .../ClusterTopologyCheckedException.java        |  49 +++++++++
 .../managers/communication/GridIoManager.java   |   4 +-
 .../deployment/GridDeploymentLocalStore.java    |   2 +-
 .../eventstorage/GridEventStorageManager.java   |   4 +-
 .../processors/affinity/GridAffinityUtils.java  |   4 +-
 .../processors/cache/GridCacheAdapter.java      |   5 +-
 .../cache/GridCacheEvictionManager.java         |   5 +-
 .../processors/cache/GridCacheIoManager.java    |  13 +--
 .../processors/cache/GridCacheMvccManager.java  |   3 +-
 .../GridCachePartitionExchangeManager.java      |   3 +-
 .../GridCacheDataStructuresManager.java         |   9 +-
 .../GridTransactionalCacheQueueImpl.java        |  18 ++--
 ...ridCacheOptimisticCheckPreparedTxFuture.java |   5 +-
 ...dCachePessimisticCheckCommittedTxFuture.java |   7 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |  11 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   7 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  13 +--
 .../cache/distributed/dht/GridDhtTxLocal.java   |   4 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   9 +-
 .../dht/GridPartitionedGetFuture.java           |  13 +--
 .../dht/atomic/GridDhtAtomicCache.java          |   7 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   5 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   7 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  13 +--
 .../dht/preloader/GridDhtForceKeysFuture.java   |   9 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   5 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   3 +-
 .../GridDhtPartitionsExchangeFuture.java        |   3 +-
 .../dht/preloader/GridDhtPreloader.java         |   3 +-
 .../distributed/near/GridNearGetFuture.java     |  13 +--
 .../distributed/near/GridNearLockFuture.java    |  13 +--
 .../near/GridNearTxFinishFuture.java            |   7 +-
 .../near/GridNearTxPrepareFuture.java           |   7 +-
 .../query/GridCacheDistributedQueryManager.java |   3 +-
 .../cache/query/GridCacheQueryAdapter.java      |   3 +-
 .../GridCacheContinuousQueryAdapter.java        |   3 +-
 .../GridCacheContinuousQueryHandler.java        |   4 +-
 .../cache/transactions/IgniteTxHandler.java     |   7 +-
 .../continuous/GridContinuousProcessor.java     |  25 ++---
 .../dataload/IgniteDataLoaderImpl.java          |   9 +-
 .../processors/fs/GridGgfsDataManager.java      |   3 +-
 .../fs/GridGgfsFragmentizerManager.java         |  11 +-
 .../processors/job/GridJobProcessor.java        |   2 +-
 .../handlers/task/GridTaskCommandHandler.java   |   3 +-
 .../service/GridServiceProcessor.java           |  17 +--
 .../processors/service/GridServiceProxy.java    |   3 +-
 .../GridStreamerStageExecutionFuture.java       |   3 +-
 .../processors/streamer/IgniteStreamerImpl.java |   8 +-
 .../processors/task/GridTaskProcessor.java      |  24 ++---
 .../processors/task/GridTaskWorker.java         |  15 +--
 .../apache/ignite/internal/util/GridUtils.java  |  13 ++-
 .../util/future/GridCompoundFuture.java         |   4 +-
 .../internal/util/future/IgniteFutureImpl.java  |   2 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |   3 +-
 .../internal/visor/node/VisorNodePingTask.java  |   4 +-
 .../org/apache/ignite/lang/IgniteFuture.java    |  13 ++-
 .../ignite/scheduler/SchedulerFuture.java       |  48 +++------
 .../ignite/internal/ClusterMetricsSelfTest.java |   3 +-
 .../internal/GridCommunicationSelfTest.java     |   2 +-
 .../internal/GridEventStorageSelfTest.java      |   1 +
 .../internal/GridFactoryVmShutdownTest.java     |   3 -
 .../GridFailoverTaskWithPredicateSelfTest.java  |   8 +-
 .../GridMultithreadedJobStealingSelfTest.java   |   2 +-
 .../internal/GridTaskFailoverSelfTest.java      |   4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |  10 +-
 .../cache/GridCacheAbstractSelfTest.java        |   4 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |   2 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |  10 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |   7 +-
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   3 +-
 .../continuous/GridEventConsumeSelfTest.java    |   7 +-
 .../fs/GridGgfsProcessorValidationSelfTest.java |   2 +-
 .../util/future/GridFutureAdapterSelfTest.java  |   6 +-
 .../loadtests/colocation/GridTestMain.java      |   9 +-
 .../GridContinuousOperationsLoadTest.java       |   6 +-
 .../ignite/loadtests/dsi/GridDsiClient.java     |   2 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |   8 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   4 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |  10 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   2 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |   2 +-
 .../ignite/p2p/GridP2PTimeoutSelfTest.java      |   2 +-
 .../p2p/GridP2PEventFilterExternalPath1.java    |   2 +-
 .../p2p/GridP2PEventFilterExternalPath2.java    |   2 +-
 .../hibernate/GridHibernateRegionFactory.java   |   2 +-
 .../scalar/pimps/ScalarProjectionPimp.scala     |  17 +--
 .../processors/schedule/ScheduleFutureImpl.java | 108 +++++++++++--------
 .../schedule/GridScheduleSelfTest.java          |  45 ++++----
 .../visor/commands/ack/VisorAckCommand.scala    |   5 +-
 .../visor/commands/gc/VisorGcCommand.scala      |   5 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   7 +-
 121 files changed, 923 insertions(+), 550 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java
----------------------------------------------------------------------
diff --git 
a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java
 
b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java
index a0be78e..6a0f029 100644
--- 
a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java
+++ 
b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java
@@ -70,12 +70,10 @@ public final class CacheAffinityExample {
     }
 
     /**
-     * Collocates jobs with keys they need to work on using {@link 
org.apache.ignite.IgniteCompute#affinityRun(String, Object, Runnable)}
+     * Collocates jobs with keys they need to work on using {@link 
IgniteCompute#affinityRun(String, Object, Runnable)}
      * method.
-     *
-     * @throws IgniteCheckedException If failed.
      */
-    private static void visitUsingAffinityRun() throws IgniteCheckedException {
+    private static void visitUsingAffinityRun() {
         Ignite g = Ignition.ignite();
 
         final GridCache<Integer, String> cache = g.cache(CACHE_NAME);
@@ -97,13 +95,11 @@ public final class CacheAffinityExample {
     }
 
     /**
-     * Collocates jobs with keys they need to work on using {@link 
org.apache.ignite.IgniteCluster#mapKeysToNodes(String, Collection)}
+     * Collocates jobs with keys they need to work on using {@link 
IgniteCluster#mapKeysToNodes(String, Collection)}
      * method. The difference from {@code affinityRun(...)} method is that 
here we process multiple keys
      * in a single job.
-     *
-     * @throws IgniteCheckedException If failed.
      */
-    private static void visitUsingMapKeysToNodes() throws 
IgniteCheckedException {
+    private static void visitUsingMapKeysToNodes() {
         final Ignite g = Ignition.ignite();
 
         Collection<Integer> keys = new ArrayList<>(KEY_CNT);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingExample.java
----------------------------------------------------------------------
diff --git 
a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingExample.java
 
b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingExample.java
index 8aabf4e..86449fb 100644
--- 
a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingExample.java
+++ 
b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingExample.java
@@ -99,9 +99,8 @@ public final class MessagingExample {
      * Start listening to messages on all grid nodes within passed in 
projection.
      *
      * @param msg Grid messaging.
-     * @throws IgniteCheckedException If failed.
      */
-    private static void startListening(IgniteMessaging msg) throws 
IgniteCheckedException {
+    private static void startListening(IgniteMessaging msg) {
         // Add ordered message listener.
         msg.remoteListen(TOPIC.ORDERED, new IgniteBiPredicate<UUID, String>() {
             @IgniteInstanceResource
@@ -113,7 +112,7 @@ public final class MessagingExample {
                 try {
                     
g.message(g.cluster().forNodeId(nodeId)).send(TOPIC.ORDERED, msg);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
 
@@ -132,7 +131,7 @@ public final class MessagingExample {
                 try {
                     
g.message(g.cluster().forNodeId(nodeId)).send(TOPIC.UNORDERED, msg);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
----------------------------------------------------------------------
diff --git 
a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
 
b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
index 11edc44..9b631df 100644
--- 
a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
+++ 
b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
@@ -80,7 +80,7 @@ public class MessagingPingPongExample {
 
                         return false; // Unsubscribe.
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         throw new GridClosureException(e);
                     }
                 }
@@ -112,7 +112,7 @@ public class MessagingPingPongExample {
 
                         return true; // Continue listening.
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         throw new GridClosureException(e);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
index da5ebb9..3c86458 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
@@ -52,7 +52,7 @@ import java.util.concurrent.*;
  * <li>{@code affinity(...)} methods colocate jobs with nodes on which a 
specified key is cached.</li>
  * </ul>
  * Note that if attempt is made to execute a computation over an empty 
projection (i.e. projection that does
- * not have any alive nodes), then {@link ClusterGroupEmptyException} will be 
thrown out of result future.
+ * not have any alive nodes), then {@link 
org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException} will be 
thrown out of result future.
  * <h1 class="header">Serializable</h1>
  * Also note that {@link Runnable} and {@link Callable} implementations must 
support serialization as required
  * by the configured marshaller. For example, {@link 
IgniteOptimizedMarshaller} requires {@link Serializable}
@@ -120,10 +120,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param job Job which will be co-located on the node with given affinity 
key.
      * @see ComputeJobContext#cacheName()
      * @see ComputeJobContext#affinityKey()
-     * @throws IgniteCheckedException If job failed.
+     * @throws IgniteException If job failed.
      */
     @IgniteAsyncSupported
-    public void affinityRun(@Nullable String cacheName, Object affKey, 
Runnable job) throws IgniteCheckedException;
+    public void affinityRun(@Nullable String cacheName, Object affKey, 
Runnable job) throws IgniteException;
 
     /**
      * Executes given job on the node where data for provided affinity key is 
located
@@ -135,12 +135,12 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param affKey Affinity key.
      * @param job Job which will be co-located on the node with given affinity 
key.
      * @return Job result.
-     * @throws IgniteCheckedException If job failed.
+     * @throws IgniteException If job failed.
      * @see ComputeJobContext#cacheName()
      * @see ComputeJobContext#affinityKey()
      */
     @IgniteAsyncSupported
-    public <R> R affinityCall(@Nullable String cacheName, Object affKey, 
Callable<R> job) throws IgniteCheckedException;
+    public <R> R affinityCall(@Nullable String cacheName, Object affKey, 
Callable<R> job) throws IgniteException;
 
     /**
      * Executes given task on the grid projection. For step-by-step 
explanation of task execution process
@@ -153,10 +153,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      *      class name is used as task name.
      * @param arg Optional argument of task execution, can be {@code null}.
      * @return Task result.
-     * @throws IgniteCheckedException If task failed.
+     * @throws IgniteException If task failed.
      */
     @IgniteAsyncSupported
-    public <T, R> R execute(Class<? extends ComputeTask<T, R>> taskCls, 
@Nullable T arg) throws IgniteCheckedException;
+    public <T, R> R execute(Class<? extends ComputeTask<T, R>> taskCls, 
@Nullable T arg) throws IgniteException;
 
     /**
      * Executes given task on this grid projection. For step-by-step 
explanation of task execution process
@@ -169,10 +169,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      *      class name is used as task name.
      * @param arg Optional argument of task execution, can be {@code null}.
      * @return Task result.
-     * @throws IgniteCheckedException If task failed.
+     * @throws IgniteException If task failed.
      */
     @IgniteAsyncSupported
-    public <T, R> R execute(ComputeTask<T, R> task, @Nullable T arg) throws 
IgniteCheckedException;
+    public <T, R> R execute(ComputeTask<T, R> task, @Nullable T arg) throws 
IgniteException;
 
     /**
      * Executes given task on this grid projection. For step-by-step 
explanation of task execution process
@@ -186,11 +186,11 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param taskName Name of the task to execute.
      * @param arg Optional argument of task execution, can be {@code null}.
      * @return Task result.
-     * @throws IgniteCheckedException If task failed.
+     * @throws IgniteException If task failed.
      * @see ComputeTask for information about task execution.
      */
     @IgniteAsyncSupported
-    public <T, R> R execute(String taskName, @Nullable T arg) throws 
IgniteCheckedException;
+    public <T, R> R execute(String taskName, @Nullable T arg) throws 
IgniteException;
 
     /**
      * Broadcasts given job to all nodes in grid projection.
@@ -198,10 +198,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param job Job to broadcast to all projection nodes.
-     * @throws IgniteCheckedException If job failed.
+     * @throws IgniteException If job failed.
      */
     @IgniteAsyncSupported
-    public void broadcast(Runnable job) throws IgniteCheckedException;
+    public void broadcast(Runnable job) throws IgniteException;
 
     /**
      * Broadcasts given job to all nodes in grid projection. Every 
participating node will return a
@@ -211,10 +211,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      *
      * @param job Job to broadcast to all projection nodes.
      * @return Collection of results for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public <R> Collection<R> broadcast(Callable<R> job) throws 
IgniteCheckedException;
+    public <R> Collection<R> broadcast(Callable<R> job) throws IgniteException;
 
     /**
      * Broadcasts given closure job with passed in argument to all nodes in 
grid projection.
@@ -226,10 +226,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param job Job to broadcast to all projection nodes.
      * @param arg Job closure argument.
      * @return Collection of results for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public <R, T> Collection<R> broadcast(IgniteClosure<T, R> job, @Nullable T 
arg) throws IgniteCheckedException;
+    public <R, T> Collection<R> broadcast(IgniteClosure<T, R> job, @Nullable T 
arg) throws IgniteException;
 
     /**
      * Executes provided job on a node in this grid projection.
@@ -237,10 +237,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param job Job closure to execute.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public void run(Runnable job) throws IgniteCheckedException;
+    public void run(Runnable job) throws IgniteException;
 
     /**
      * Executes collection of jobs on grid nodes within this grid projection.
@@ -248,10 +248,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param jobs Collection of jobs to execute.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public void run(Collection<? extends Runnable> jobs) throws 
IgniteCheckedException;
+    public void run(Collection<? extends Runnable> jobs) throws 
IgniteException;
 
     /**
      * Executes provided job on a node in this grid projection. The result of 
the
@@ -261,10 +261,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      *
      * @param job Job to execute.
      * @return Job result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public <R> R call(Callable<R> job) throws IgniteCheckedException;
+    public <R> R call(Callable<R> job) throws IgniteException;
 
     /**
      * Executes collection of jobs on nodes within this grid projection.
@@ -274,10 +274,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      *
      * @param jobs Collection of jobs to execute.
      * @return Collection of job results for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public <R> Collection<R> call(Collection<? extends Callable<R>> jobs) 
throws IgniteCheckedException;
+    public <R> Collection<R> call(Collection<? extends Callable<R>> jobs) 
throws IgniteException;
 
     /**
      * Executes collection of jobs on nodes within this grid projection. The 
returned
@@ -288,10 +288,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param jobs Collection of jobs to execute.
      * @param rdc Reducer to reduce all job results into one individual return 
value.
      * @return Future with reduced job result for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public <R1, R2> R2 call(Collection<? extends Callable<R1>> jobs, 
IgniteReducer<R1, R2> rdc) throws IgniteCheckedException;
+    public <R1, R2> R2 call(Collection<? extends Callable<R1>> jobs, 
IgniteReducer<R1, R2> rdc) throws IgniteException;
 
     /**
      * Executes provided closure job on a node in this grid projection. This 
method is different
@@ -303,10 +303,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param job Job to run.
      * @param arg Job argument.
      * @return Job result.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public <R, T> R apply(IgniteClosure<T, R> job, @Nullable T arg) throws 
IgniteCheckedException;
+    public <R, T> R apply(IgniteClosure<T, R> job, @Nullable T arg) throws 
IgniteException;
 
     /**
      * Executes provided closure job on nodes within this grid projection. A 
new job is executed for
@@ -318,10 +318,10 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param job Job to run.
      * @param args Job arguments.
      * @return Collection of job results.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
-    public <T, R> Collection<R> apply(IgniteClosure<T, R> job, Collection<? 
extends T> args) throws IgniteCheckedException;
+    public <T, R> Collection<R> apply(IgniteClosure<T, R> job, Collection<? 
extends T> args) throws IgniteException;
 
     /**
      * Executes provided closure job on nodes within this grid projection. A 
new job is executed for
@@ -335,11 +335,11 @@ public interface IgniteCompute extends IgniteAsyncSupport 
{
      * @param args Job arguments.
      * @param rdc Reducer to reduce all job results into one individual return 
value.
      * @return Future with reduced job result for this execution.
-     * @throws IgniteCheckedException If execution failed.
+     * @throws IgniteException If execution failed.
      */
     @IgniteAsyncSupported
     public <R1, R2, T> R2 apply(IgniteClosure<T, R1> job, Collection<? extends 
T> args,
-        IgniteReducer<R1, R2> rdc) throws IgniteCheckedException;
+        IgniteReducer<R1, R2> rdc) throws IgniteException;
 
     /**
      * Gets tasks future for active tasks started on local node.
@@ -418,9 +418,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      *      class name will be used as task's name.
      * @param clsLdr Task class loader. This class loader is in charge
      *      of loading all necessary resources for task execution.
-     * @throws IgniteCheckedException If task is invalid and cannot be 
deployed.
+     * @throws IgniteException If task is invalid and cannot be deployed.
      */
-    public void localDeployTask(Class<? extends ComputeTask> taskCls, 
ClassLoader clsLdr) throws IgniteCheckedException;
+    public void localDeployTask(Class<? extends ComputeTask> taskCls, 
ClassLoader clsLdr) throws IgniteException;
 
     /**
      * Gets map of all locally deployed tasks keyed by their task name .
@@ -435,9 +435,9 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * undeployed on every node.
      *
      * @param taskName Name of the task to undeploy.
-     * @throws IgniteCheckedException Thrown if undeploy failed.
+     * @throws IgniteException Thrown if undeploy failed.
      */
-    public void undeployTask(String taskName) throws IgniteCheckedException;
+    public void undeployTask(String taskName) throws IgniteException;
 
     /** {@inheritDoc} */
     @Override public <R> ComputeTaskFuture<R> future();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java
index e5fd803..d0b010a 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java
@@ -22,7 +22,7 @@ import org.jetbrains.annotations.*;
 /**
  * Deployment or re-deployment failed.
  */
-public class IgniteDeploymentException extends IgniteCheckedException {
+public class IgniteDeploymentException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
index 6a42639..e3b33ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
@@ -31,12 +31,12 @@ import java.util.*;
  * <pre name="code" class="java">
  * GridEvents evts = GridGain.grid().events();
  * </pre> * <p>
- * Local subscription, defined by {@link 
#localListen(org.apache.ignite.lang.IgnitePredicate, int...)} method, will add
+ * Local subscription, defined by {@link #localListen(IgnitePredicate, 
int...)} method, will add
  * a listener for specified events on local node only. This listener will be 
notified whenever any
  * of subscribed events happen on local node regardless of whether local node 
belongs to underlying
  * grid projection or not.
  * <p>
- * Remote subscription, defined by {@link 
#remoteListen(org.apache.ignite.lang.IgniteBiPredicate, 
org.apache.ignite.lang.IgnitePredicate, int...)}, will add an
+ * Remote subscription, defined by {@link #remoteListen(IgniteBiPredicate, 
IgnitePredicate, int...)}, will add an
  * event listener for specified events on all nodes in the projection 
(possibly including local node if
  * it belongs to the projection as well). All projection nodes will then be 
notified of the subscribed events.
  * If the events pass the remote event filter, the events will be sent to 
local node for local listener notification.
@@ -69,11 +69,11 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      * @param timeout Maximum time to wait for result, {@code 0} to wait 
forever.
      * @param types Event types to be queried.
      * @return Collection of grid events returned from specified nodes.
-     * @throws IgniteCheckedException If query failed.
+     * @throws IgniteException If query failed.
      */
     @IgniteAsyncSupported
     public <T extends IgniteEvent> List<T> remoteQuery(IgnitePredicate<T> p, 
long timeout, @Nullable int... types)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Adds event listener for specified events to all nodes in the projection 
(possibly including
@@ -97,13 +97,13 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      *      provided remote filter will be sent to local node.
      * @param <T> Type of the event.
      * @return {@code Operation ID} that can be passed to {@link 
#stopRemoteListen(UUID)} method to stop listening.
-     * @throws IgniteCheckedException If failed to add listener.
+     * @throws IgniteException If failed to add listener.
      */
     @IgniteAsyncSupported
     public <T extends IgniteEvent> UUID remoteListen(@Nullable 
IgniteBiPredicate<UUID, T> locLsnr,
         @Nullable IgnitePredicate<T> rmtFilter,
         @Nullable int... types)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Adds event listener for specified events to all nodes in the projection 
(possibly including
@@ -138,7 +138,7 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      * @param <T> Type of the event.
      * @return {@code Operation ID} that can be passed to {@link 
#stopRemoteListen(UUID)} method to stop listening.
      * @see #stopRemoteListen(UUID)
-     * @throws IgniteCheckedException If failed to add listener.
+     * @throws IgniteException If failed to add listener.
      */
     @IgniteAsyncSupported
     public <T extends IgniteEvent> UUID remoteListen(int bufSize,
@@ -147,7 +147,7 @@ public interface IgniteEvents extends IgniteAsyncSupport {
         @Nullable IgniteBiPredicate<UUID, T> locLsnr,
         @Nullable IgnitePredicate<T> rmtFilter,
         @Nullable int... types)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Stops listening to remote events. This will unregister all listeners 
identified with provided
@@ -156,12 +156,12 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param opId Operation ID that was returned from
-     *      {@link #remoteListen(org.apache.ignite.lang.IgniteBiPredicate, 
org.apache.ignite.lang.IgnitePredicate, int...)} method.
-     * @see #remoteListen(org.apache.ignite.lang.IgniteBiPredicate, 
org.apache.ignite.lang.IgnitePredicate, int...)
-     * @throws IgniteCheckedException If failed to stop listeners.
+     *      {@link #remoteListen(IgniteBiPredicate, IgnitePredicate, int...)} 
method.
+     * @see #remoteListen(IgniteBiPredicate, IgnitePredicate, int...)
+     * @throws IgniteException If failed to stop listeners.
      */
     @IgniteAsyncSupported
-    public void stopRemoteListen(UUID opId) throws IgniteCheckedException;
+    public void stopRemoteListen(UUID opId) throws IgniteException;
 
     /**
      * Waits for the specified events.
@@ -172,11 +172,11 @@ public interface IgniteEvents extends IgniteAsyncSupport {
      *      end the wait.
      * @param types Types of the events to wait for. If not provided, all 
events will be passed to the filter.
      * @return Grid event.
-     * @throws IgniteCheckedException If wait was interrupted.
+     * @throws IgniteException If wait was interrupted.
      */
     @IgniteAsyncSupported
     public <T extends IgniteEvent> T waitForLocal(@Nullable IgnitePredicate<T> 
filter, @Nullable int... types)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Queries local node for events using passed-in predicate filter for 
event selection.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
index 5c7aaa7..a1cd6c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteManaged.java
@@ -145,10 +145,10 @@ public interface IgniteManaged extends IgniteAsyncSupport 
{
      *
      * @param name Service name.
      * @param svc Service instance.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
     @IgniteAsyncSupported
-    public void deployClusterSingleton(String name, ManagedService svc) throws 
IgniteCheckedException;
+    public void deployClusterSingleton(String name, ManagedService svc) throws 
IgniteException;
 
     /**
      * Deploys a per-node singleton service. GridGain will guarantee that 
there is always
@@ -163,10 +163,10 @@ public interface IgniteManaged extends IgniteAsyncSupport 
{
      *
      * @param name Service name.
      * @param svc Service instance.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
     @IgniteAsyncSupported
-    public void deployNodeSingleton(String name, ManagedService svc) throws 
IgniteCheckedException;
+    public void deployNodeSingleton(String name, ManagedService svc) throws 
IgniteException;
 
     /**
      * Deploys one instance of this service on the primary node for a given 
affinity key.
@@ -199,14 +199,14 @@ public interface IgniteManaged extends IgniteAsyncSupport 
{
      * @param cacheName Name of the cache on which affinity for key should be 
calculated, {@code null} for
      *      default cache.
      * @param affKey Affinity cache key.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
     @IgniteAsyncSupported
     public void deployKeyAffinitySingleton(String name,
         ManagedService svc,
         @Nullable String cacheName,
         Object affKey)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Deploys multiple instances of the service on the grid. GridGain will 
deploy a
@@ -238,14 +238,14 @@ public interface IgniteManaged extends IgniteAsyncSupport 
{
      * @param svc Service instance.
      * @param totalCnt Maximum number of deployed services in the grid, {@code 
0} for unlimited.
      * @param maxPerNodeCnt Maximum number of deployed services on each node, 
{@code 0} for unlimited.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
     @IgniteAsyncSupported
     public void deployMultiple(String name,
         ManagedService svc,
         int totalCnt,
         int maxPerNodeCnt)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /**
      * Deploys multiple instances of the service on the grid according to 
provided
@@ -283,10 +283,10 @@ public interface IgniteManaged extends IgniteAsyncSupport 
{
      * </pre>
      *
      * @param cfg Service configuration.
-     * @throws IgniteCheckedException If failed to deploy service.
+     * @throws IgniteException If failed to deploy service.
      */
     @IgniteAsyncSupported
-    public void deploy(ManagedServiceConfiguration cfg) throws 
IgniteCheckedException;
+    public void deploy(ManagedServiceConfiguration cfg) throws IgniteException;
 
     /**
      * Cancels service deployment. If a service with specified name was 
deployed on the grid,
@@ -299,10 +299,10 @@ public interface IgniteManaged extends IgniteAsyncSupport 
{
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
      * @param name Name of service to cancel.
-     * @throws IgniteCheckedException If failed to cancel service.
+     * @throws IgniteException If failed to cancel service.
      */
     @IgniteAsyncSupported
-    public void cancel(String name) throws IgniteCheckedException;
+    public void cancel(String name) throws IgniteException;
 
     /**
      * Cancels all deployed services.
@@ -312,10 +312,10 @@ public interface IgniteManaged extends IgniteAsyncSupport 
{
      * <p>
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
-     * @throws IgniteCheckedException If failed to cancel services.
+     * @throws IgniteException If failed to cancel services.
      */
     @IgniteAsyncSupported
-    public void cancelAll() throws IgniteCheckedException;
+    public void cancelAll() throws IgniteException;
 
     /**
      * Gets metadata about all deployed services.
@@ -352,6 +352,7 @@ public interface IgniteManaged extends IgniteAsyncSupport {
      * @param sticky Whether or not GridGain should always contact the same 
remote
      *      service or try to load-balance between services.
      * @return Either proxy over remote service or local service if it is 
deployed locally.
+     * @throws IgniteException If failed to create service proxy.
      */
     public <T> T serviceProxy(String name, Class<? super T> svcItf, boolean 
sticky) throws IgniteException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
index 68e4f48..a859b05 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteMessaging.java
@@ -18,6 +18,7 @@
 package org.apache.ignite;
 
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -34,12 +35,12 @@ import java.util.*;
  * <p>
  * There are {@code 2} ways to subscribe to message listening, {@code local} 
and {@code remote}.
  * <p>
- * Local subscription, defined by {@link #localListen(Object, 
org.apache.ignite.lang.IgniteBiPredicate)} method, will add
+ * Local subscription, defined by {@link #localListen(Object, 
IgniteBiPredicate)} method, will add
  * a listener for a given topic on local node only. This listener will be 
notified whenever any
  * node within grid projection will send a message for a given topic to this 
node. Local listen
  * subscription will happen regardless of whether local node belongs to this 
grid projection or not.
  * <p>
- * Remote subscription, defined by {@link #remoteListen(Object, 
org.apache.ignite.lang.IgniteBiPredicate)}, will add a
+ * Remote subscription, defined by {@link #remoteListen(Object, 
IgniteBiPredicate)}, will add a
  * message listener for a given topic to all nodes in the projection (possibly 
including this node if
  * it belongs to the projection as well). This means that any node within this 
grid projection can send
  * a message for a given topic and all nodes within projection will receive 
listener notification.
@@ -64,10 +65,10 @@ public interface IgniteMessaging extends IgniteAsyncSupport 
{
      *
      * @param topic Topic to send to, {@code null} for default topic.
      * @param msg Message to send.
-     * @throws IgniteCheckedException If failed to send a message to any of 
the nodes.
-     * @throws org.apache.ignite.cluster.ClusterGroupEmptyException Thrown in 
case when this projection is empty.
+     * @throws IgniteException If failed to send a message to any of the nodes.
+     * @throws ClusterGroupEmptyException Thrown in case when cluster group is 
empty.
      */
-    public void send(@Nullable Object topic, Object msg) throws 
IgniteCheckedException;
+    public void send(@Nullable Object topic, Object msg) throws 
IgniteException;
 
     /**
      * Sends given messages with specified topic to the nodes in this 
projection.
@@ -75,10 +76,10 @@ public interface IgniteMessaging extends IgniteAsyncSupport 
{
      * @param topic Topic to send to, {@code null} for default topic.
      * @param msgs Messages to send. Order of the sending is undefined. If the 
method produces
      *      the exception none or some messages could have been sent already.
-     * @throws IgniteCheckedException If failed to send a message to any of 
the nodes.
-     * @throws org.apache.ignite.cluster.ClusterGroupEmptyException Thrown in 
case when this projection is empty.
+     * @throws IgniteException If failed to send a message to any of the nodes.
+     * @throws ClusterGroupEmptyException Thrown in case when cluster group is 
empty.
      */
-    public void send(@Nullable Object topic, Collection<?> msgs) throws 
IgniteCheckedException;
+    public void send(@Nullable Object topic, Collection<?> msgs) throws 
IgniteException;
 
     /**
      * Sends given message with specified topic to the nodes in this 
projection. Messages sent with
@@ -93,11 +94,11 @@ public interface IgniteMessaging extends IgniteAsyncSupport 
{
      * @param topic Topic to send to, {@code null} for default topic.
      * @param msg Message to send.
      * @param timeout Message timeout in milliseconds, {@code 0} for default
-     *      which is {@link 
org.apache.ignite.configuration.IgniteConfiguration#getNetworkTimeout()}.
-     * @throws IgniteCheckedException If failed to send a message to any of 
the nodes.
-     * @throws org.apache.ignite.cluster.ClusterGroupEmptyException Thrown in 
case when this projection is empty.
+     *      which is {@link IgniteConfiguration#getNetworkTimeout()}.
+     * @throws IgniteException If failed to send a message to any of the nodes.
+     * @throws ClusterGroupEmptyException Thrown in case when cluster group is 
empty.
      */
-    public void sendOrdered(@Nullable Object topic, Object msg, long timeout) 
throws IgniteCheckedException;
+    public void sendOrdered(@Nullable Object topic, Object msg, long timeout) 
throws IgniteException;
 
     /**
      * Adds local listener for given topic on local node only. This listener 
will be notified whenever any
@@ -130,21 +131,21 @@ public interface IgniteMessaging extends 
IgniteAsyncSupport {
      * @param p Predicate that is called on each node for each received 
message. If predicate returns {@code false},
      *      then it will be unsubscribed from any further notifications.
      * @return {@code Operation ID} that can be passed to {@link 
#stopRemoteListen(UUID)} method to stop listening.
-     * @throws IgniteCheckedException If failed to add listener.
+     * @throws IgniteException If failed to add listener.
      */
     @IgniteAsyncSupported
-    public UUID remoteListen(@Nullable Object topic, IgniteBiPredicate<UUID, 
?> p) throws IgniteCheckedException;
+    public UUID remoteListen(@Nullable Object topic, IgniteBiPredicate<UUID, 
?> p) throws IgniteException;
 
     /**
      * Unregisters all listeners identified with provided operation ID on all 
nodes in this projection.
      * <p>
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
-     * @param opId Listen ID that was returned from {@link 
#remoteListen(Object, org.apache.ignite.lang.IgniteBiPredicate)} method.
-     * @throws IgniteCheckedException If failed to unregister listeners.
+     * @param opId Listen ID that was returned from {@link 
#remoteListen(Object, IgniteBiPredicate)} method.
+     * @throws IgniteException If failed to unregister listeners.
      */
     @IgniteAsyncSupported
-    public void stopRemoteListen(UUID opId) throws IgniteCheckedException;
+    public void stopRemoteListen(UUID opId) throws IgniteException;
 
     /** {@inheritDoc} */
     @Override IgniteMessaging withAsync();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
index 26fd0f3..df4ab91 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite;
 
-import org.apache.ignite.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.scheduler.*;
 import org.jetbrains.annotations.*;
 
@@ -49,7 +49,7 @@ public interface IgniteScheduler {
     /**
      * Executes given closure on internal system thread pool asynchronously.
      * <p>
-     * Note that class {@link org.apache.ignite.lang.IgniteRunnable} 
implements {@link Runnable} and class {@link 
org.apache.ignite.lang.IgniteOutClosure}
+     * Note that class {@link IgniteRunnable} implements {@link Runnable} and 
class {@link IgniteOutClosure}
      * implements {@link Callable} interface.
      *
      * @param r Runnable to execute. If {@code null} - this method is no-op.
@@ -57,21 +57,21 @@ public interface IgniteScheduler {
      * @see #callLocal(Callable)
      * @see org.apache.ignite.lang.IgniteClosure
      */
-    public IgniteInternalFuture<?> runLocal(@Nullable Runnable r);
+    public IgniteFuture<?> runLocal(@Nullable Runnable r);
 
     /**
      * Executes given callable on internal system thread pool asynchronously.
      * <p>
-     * Note that class {@link org.apache.ignite.lang.IgniteRunnable} 
implements {@link Runnable} and class {@link 
org.apache.ignite.lang.IgniteOutClosure}
+     * Note that class {@link IgniteRunnable} implements {@link Runnable} and 
class {@link IgniteOutClosure}
      * implements {@link Callable} interface.
      *
      * @param c Callable to execute. If {@code null} - this method is no-op.
      * @return Future for this execution.
      * @param <R> Type of the return value for the closure.
      * @see #runLocal(Runnable)
-     * @see org.apache.ignite.lang.IgniteOutClosure
+     * @see IgniteOutClosure
      */
-    public <R> IgniteInternalFuture<R> callLocal(@Nullable Callable<R> c);
+    public <R> IgniteFuture<R> callLocal(@Nullable Callable<R> c);
 
     /**
      * Schedules job for execution using local <b>cron-based</b> scheduling.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java 
b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index a8c3619..56b1261 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -19,6 +19,7 @@ package org.apache.ignite;
 
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lifecycle.*;
 import org.jetbrains.annotations.*;
 
@@ -31,7 +32,7 @@ import java.util.*;
  * <h1 class="header">Grid Loaders</h1>
  * Although user can apply grid factory directly to start and stop grid, grid 
is
  * often started and stopped by grid loaders. Grid loaders can be found in
- * {@link org.gridgain.grid.startup} package, for example:
+ * {@link org.apache.ignite.startup} package, for example:
  * <ul>
  * <li>{@link org.apache.ignite.startup.cmdline.CommandLineStartup}</li>
  * <li>{@gglink org.gridgain.grid.startup.servlet.GridServletStartup}</li>
@@ -268,11 +269,16 @@ public class Ignition {
      * configuration file. If such file is not found, then all system defaults 
will be used.
      *
      * @return Started grid.
-     * @throws IgniteCheckedException If default grid could not be started. 
This exception will be thrown
+     * @throws IgniteException If default grid could not be started. This 
exception will be thrown
      *      also if default grid has already been started.
      */
-    public static Ignite start() throws IgniteCheckedException {
-        return IgnitionEx.start();
+    public static Ignite start() throws IgniteException {
+        try {
+            return IgnitionEx.start();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /**
@@ -281,11 +287,16 @@ public class Ignition {
      *
      * @param cfg Grid configuration. This cannot be {@code null}.
      * @return Started grid.
-     * @throws IgniteCheckedException If grid could not be started. This 
exception will be thrown
+     * @throws IgniteException If grid could not be started. This exception 
will be thrown
      *      also if named grid has already been started.
      */
-    public static Ignite start(IgniteConfiguration cfg) throws 
IgniteCheckedException {
-        return IgnitionEx.start(cfg);
+    public static Ignite start(IgniteConfiguration cfg) throws IgniteException 
{
+        try {
+            return IgnitionEx.start(cfg);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /**
@@ -300,12 +311,17 @@ public class Ignition {
      * @param springCfgPath Spring XML configuration file path or URL.
      * @return Started grid. If Spring configuration contains multiple grid 
instances,
      *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or 
configuration
+     * @throws IgniteException If grid could not be started or configuration
      *      read. This exception will be thrown also if grid with given name 
has already
      *      been started or Spring XML configuration file is invalid.
      */
-    public static Ignite start(@Nullable String springCfgPath) throws 
IgniteCheckedException {
-        return IgnitionEx.start(springCfgPath);
+    public static Ignite start(@Nullable String springCfgPath) throws 
IgniteException {
+        try {
+            return IgnitionEx.start(springCfgPath);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /**
@@ -320,12 +336,17 @@ public class Ignition {
      * @param springCfgUrl Spring XML configuration file URL. This cannot be 
{@code null}.
      * @return Started grid. If Spring configuration contains multiple grid 
instances,
      *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or 
configuration
+     * @throws IgniteException If grid could not be started or configuration
      *      read. This exception will be thrown also if grid with given name 
has already
      *      been started or Spring XML configuration file is invalid.
      */
-    public static Ignite start(URL springCfgUrl) throws IgniteCheckedException 
{
-        return IgnitionEx.start(springCfgUrl);
+    public static Ignite start(URL springCfgUrl) throws IgniteException {
+        try {
+            return IgnitionEx.start(springCfgUrl);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
index bc9e3f4..7ac45be 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
@@ -137,7 +137,7 @@ public class CacheManager implements 
javax.cache.CacheManager {
                 try {
                     ignite = Ignition.start(cfg);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     throw new CacheException(e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroupEmptyException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroupEmptyException.java
 
b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroupEmptyException.java
index 8567cca..f03e08c 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroupEmptyException.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroupEmptyException.java
@@ -20,8 +20,8 @@ package org.apache.ignite.cluster;
 import org.jetbrains.annotations.*;
 
 /**
- * This exception defines illegal call on empty projection. Thrown by 
projection when operation
- * that requires at least one node is called on empty projection.
+ * This exception defines illegal call on empty cluster group. Thrown by 
cluster group when operation
+ * that requires at least one node is called on empty cluster group.
  */
 public class ClusterGroupEmptyException extends ClusterTopologyException {
     /** */

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java 
b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
index fe64804..e37e0eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResult.java
@@ -67,7 +67,7 @@ public interface ComputeJobResult {
      *      If job on remote node was rejected (cancelled while it was on 
waiting queue), then
      *      {@link ComputeExecutionRejectedException} will be returned.
      *      <p>
-     *      If node on which job was computing failed, then {@link 
org.apache.ignite.cluster.ClusterTopologyException} is
+     *      If node on which job was computing failed, then {@link 
org.apache.ignite.internal.cluster.ClusterTopologyCheckedException} is
      *      returned.
      */
     public IgniteCheckedException getException();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java 
b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
index 5c6e7fb..8d7756b 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTask.java
@@ -87,7 +87,7 @@ import java.util.*;
  *          <ul>
  *          <li>
  *              Job has failed due to node crash. In this case {@link 
ComputeJobResult#getException()}
- *              method will return an instance of {@link 
org.apache.ignite.cluster.ClusterTopologyException} exception.
+ *              method will return an instance of {@link 
org.apache.ignite.internal.cluster.ClusterTopologyCheckedException} exception.
  *          </li>
  *          <li>
  *              Job execution was rejected, i.e. remote node has cancelled job 
before it got
@@ -146,7 +146,7 @@ import java.util.*;
  * <li>
  * {@link ComputeTaskAdapter} provides default implementation for {@link 
ComputeTask#result(ComputeJobResult, List)}
  * method which provides automatic fail-over to another node if remote job has 
failed
- * due to node crash (detected by {@link 
org.apache.ignite.cluster.ClusterTopologyException} exception) or due to job
+ * due to node crash (detected by {@link 
org.apache.ignite.internal.cluster.ClusterTopologyCheckedException} exception) 
or due to job
  * execution rejection (detected by {@link ComputeExecutionRejectedException} 
exception).
  * Here is an example of how a you would implement your task using {@link 
ComputeTaskAdapter}:
  * <pre name="code" class="java">

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java 
b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
index b105249..b2a340a 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.compute;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.cluster.*;
 
 import java.util.*;
 
@@ -76,7 +76,7 @@ public abstract class ComputeTaskAdapter<T, R> implements 
ComputeTask<T, R> {
      * <p>
      * If remote job resulted in exception ({@link 
ComputeJobResult#getException()} is not {@code null}),
      * then {@link ComputeJobResultPolicy#FAILOVER} policy will be returned if 
the exception is instance
-     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link 
ComputeExecutionRejectedException}, which means that
+     * of {@link 
org.apache.ignite.internal.cluster.ClusterTopologyCheckedException} or {@link 
ComputeExecutionRejectedException}, which means that
      * remote node either failed or job execution was rejected before it got a 
chance to start. In all
      * other cases the exception will be rethrown which will ultimately cause 
task to fail.
      *
@@ -94,7 +94,7 @@ public abstract class ComputeTaskAdapter<T, R> implements 
ComputeTask<T, R> {
         if (e != null) {
             // Don't failover user's code errors.
             if (e instanceof ComputeExecutionRejectedException ||
-                e instanceof ClusterTopologyException ||
+                e instanceof ClusterTopologyCheckedException ||
                 // Failover exception is always wrapped.
                 e.hasCause(ComputeJobFailoverException.class))
                 return ComputeJobResultPolicy.FAILOVER;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java 
b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
index 8476fc6..d5401fa 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
@@ -17,8 +17,7 @@
 
 package org.apache.ignite.compute;
 
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
+import org.apache.ignite.lang.*;
 
 import java.util.*;
 import java.util.concurrent.*;
@@ -29,27 +28,20 @@ import java.util.concurrent.*;
  * hierarchy.
  * @param <R> Type of the task result returning from {@link 
ComputeTask#reduce(List)} method.
  */
-public interface ComputeTaskFuture<R> extends IgniteInternalFuture<R> {
+public interface ComputeTaskFuture<R> extends IgniteFuture<R> {
     /**
      * {@inheritDoc}
      *
      * @throws ComputeTaskTimeoutException If task execution timed out.
      */
-    @Override public R get() throws IgniteCheckedException;
+    @Override public R get();
 
     /**
      * {@inheritDoc}
      *
      * @throws ComputeTaskTimeoutException If task execution timed out.
      */
-    @Override public R get(long timeout) throws IgniteCheckedException;
-
-    /**
-     * {@inheritDoc}
-     *
-     * @throws ComputeTaskTimeoutException If task execution timed out.
-     */
-    @Override public R get(long timeout, TimeUnit unit) throws 
IgniteCheckedException;
+    @Override public R get(long timeout, TimeUnit unit);
 
     /**
      * Gets task session of execution grid task.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index a88c9ef..9962b71 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -251,7 +251,7 @@ class GridEventConsumeHandler implements 
GridContinuousHandler {
             GridDeployment dep = ctx.deploy().deploy(cls, 
U.detectClassLoader(cls));
 
             if (dep == null)
-                throw new IgniteDeploymentException("Failed to deploy event 
filter: " + filter);
+                throw new IgniteDeploymentCheckedException("Failed to deploy 
event filter: " + filter);
 
             depInfo = new GridDeploymentInfoBean(dep);
 
@@ -270,7 +270,7 @@ class GridEventConsumeHandler implements 
GridContinuousHandler {
                 depInfo.userVersion(), nodeId, depInfo.classLoaderId(), 
depInfo.participants(), null);
 
             if (dep == null)
-                throw new IgniteDeploymentException("Failed to obtain 
deployment for class: " + clsName);
+                throw new IgniteDeploymentCheckedException("Failed to obtain 
deployment for class: " + clsName);
 
             filter = ctx.config().getMarshaller().unmarshal(filterBytes, 
dep.classLoader());
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
index 9aca42c..f70cf6a 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
@@ -129,7 +129,7 @@ public class GridMessageListenHandler implements 
GridContinuousHandler {
         GridDeployment dep = ctx.deploy().deploy(pda.deployClass(), 
pda.classLoader());
 
         if (dep == null)
-            throw new IgniteDeploymentException("Failed to deploy message 
listener.");
+            throw new IgniteDeploymentCheckedException("Failed to deploy 
message listener.");
 
         depInfo = new GridDeploymentInfoBean(dep);
 
@@ -146,7 +146,7 @@ public class GridMessageListenHandler implements 
GridContinuousHandler {
             depInfo.userVersion(), nodeId, depInfo.classLoaderId(), 
depInfo.participants(), null);
 
         if (dep == null)
-            throw new IgniteDeploymentException("Failed to obtain deployment 
for class: " + clsName);
+            throw new IgniteDeploymentCheckedException("Failed to obtain 
deployment for class: " + clsName);
 
         ClassLoader ldr = dep.classLoader();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/internal/GridTaskFutureImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskFutureImpl.java 
b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskFutureImpl.java
index 6f162d6..1958f9a 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskFutureImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskFutureImpl.java
@@ -31,7 +31,7 @@ import java.util.*;
  * This class provide implementation for task future.
  * @param <R> Type of the task result returning from {@link 
org.apache.ignite.compute.ComputeTask#reduce(List)} method.
  */
-public class GridTaskFutureImpl<R> extends GridFutureAdapter<R> implements 
ComputeTaskFuture<R> {
+public class GridTaskFutureImpl<R> extends GridFutureAdapter<R> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -41,6 +41,9 @@ public class GridTaskFutureImpl<R> extends 
GridFutureAdapter<R> implements Compu
     /** */
     private GridKernalContext ctx;
 
+    /** */
+    private ComputeFuture<R> userFut;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -60,6 +63,15 @@ public class GridTaskFutureImpl<R> extends 
GridFutureAdapter<R> implements Compu
 
         this.ses = ses;
         this.ctx = ctx;
+
+        userFut = new ComputeFuture<>(this);
+    }
+
+    /**
+     * @return Future returned by public API.
+     */
+    public ComputeTaskFuture<R> publicFuture() {
+        return userFut;
     }
 
     /**
@@ -67,7 +79,7 @@ public class GridTaskFutureImpl<R> extends 
GridFutureAdapter<R> implements Compu
      *
      * @return Task timeout.
      */
-    @Override public ComputeTaskSession getTaskSession() {
+    public ComputeTaskSession getTaskSession() {
         if (ses == null)
             throw new IllegalStateException("Cannot access task session after 
future has been deserialized.");
 
@@ -119,4 +131,21 @@ public class GridTaskFutureImpl<R> extends 
GridFutureAdapter<R> implements Compu
     @Override public String toString() {
         return S.toString(GridTaskFutureImpl.class, this, "super", 
super.toString());
     }
+
+    /**
+     *
+     */
+    private static class ComputeFuture<R> extends IgniteFutureImpl<R> 
implements ComputeTaskFuture<R> {
+        /**
+         * @param fut Future.
+         */
+        private ComputeFuture(GridTaskFutureImpl<R> fut) {
+            super(fut);
+        }
+
+        /** {@inheritDoc} */
+        @Override public ComputeTaskSession getTaskSession() {
+            return ((GridTaskFutureImpl<R>)fut).getTaskSession();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/internal/IgniteComputeImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComputeImpl.java 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComputeImpl.java
index 16d6e9e..c8635ef 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComputeImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComputeImpl.java
@@ -81,8 +81,7 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
     }
 
     /** {@inheritDoc} */
-    @Override public void affinityRun(@Nullable String cacheName, Object 
affKey, Runnable job)
-        throws IgniteCheckedException {
+    @Override public void affinityRun(@Nullable String cacheName, Object 
affKey, Runnable job) {
         A.notNull(affKey, "affKey");
         A.notNull(job, "job");
 
@@ -91,14 +90,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             saveOrGet(ctx.closure().affinityRun(cacheName, affKey, job, 
prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <R> R affinityCall(@Nullable String cacheName, Object 
affKey, Callable<R> job)
-        throws IgniteCheckedException {
+    @Override public <R> R affinityCall(@Nullable String cacheName, Object 
affKey, Callable<R> job) {
         A.notNull(affKey, "affKey");
         A.notNull(job, "job");
 
@@ -107,13 +108,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().affinityCall(cacheName, affKey, 
job, prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <T, R> R execute(String taskName, @Nullable T arg) throws 
IgniteCheckedException {
+    @Override public <T, R> R execute(String taskName, @Nullable T arg) {
         A.notNull(taskName, "taskName");
 
         guard();
@@ -124,14 +128,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
 
             return (R)saveOrGet(ctx.task().execute(taskName, arg));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <T, R> R execute(Class<? extends ComputeTask<T, R>> 
taskCls,
-        @Nullable T arg) throws IgniteCheckedException {
+    @Override public <T, R> R execute(Class<? extends ComputeTask<T, R>> 
taskCls, @Nullable T arg) {
         A.notNull(taskCls, "taskCls");
 
         guard();
@@ -142,13 +148,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
 
             return saveOrGet(ctx.task().execute(taskCls, arg));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <T, R> R execute(ComputeTask<T, R> task, @Nullable T arg) 
throws IgniteCheckedException {
+    @Override public <T, R> R execute(ComputeTask<T, R> task, @Nullable T arg) 
{
         A.notNull(task, "task");
 
         guard();
@@ -159,13 +168,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
 
             return saveOrGet(ctx.task().execute(task, arg));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void broadcast(Runnable job) throws 
IgniteCheckedException {
+    @Override public void broadcast(Runnable job) {
         A.notNull(job, "job");
 
         guard();
@@ -173,13 +185,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             saveOrGet(ctx.closure().runAsync(BROADCAST, job, prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <R> Collection<R> broadcast(Callable<R> job) throws 
IgniteCheckedException {
+    @Override public <R> Collection<R> broadcast(Callable<R> job) {
         A.notNull(job, "job");
 
         guard();
@@ -187,14 +202,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().callAsync(BROADCAST, 
Arrays.asList(job), prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <R, T> Collection<R> broadcast(IgniteClosure<T, R> job, 
@Nullable T arg)
-        throws IgniteCheckedException {
+    @Override public <R, T> Collection<R> broadcast(IgniteClosure<T, R> job, 
@Nullable T arg) {
         A.notNull(job, "job");
 
         guard();
@@ -202,13 +219,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().broadcast(job, arg, prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void run(Runnable job) throws IgniteCheckedException {
+    @Override public void run(Runnable job) {
         A.notNull(job, "job");
 
         guard();
@@ -216,13 +236,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             saveOrGet(ctx.closure().runAsync(BALANCE, job, prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void run(Collection<? extends Runnable> jobs) throws 
IgniteCheckedException {
+    @Override public void run(Collection<? extends Runnable> jobs) {
         A.notEmpty(jobs, "jobs");
 
         guard();
@@ -230,13 +253,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             saveOrGet(ctx.closure().runAsync(BALANCE, jobs, prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <R, T> R apply(IgniteClosure<T, R> job, @Nullable T arg) 
throws IgniteCheckedException {
+    @Override public <R, T> R apply(IgniteClosure<T, R> job, @Nullable T arg) {
         A.notNull(job, "job");
 
         guard();
@@ -244,13 +270,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().callAsync(job, arg, prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <R> R call(Callable<R> job) throws IgniteCheckedException 
{
+    @Override public <R> R call(Callable<R> job) {
         A.notNull(job, "job");
 
         guard();
@@ -258,13 +287,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().callAsync(BALANCE, job, 
prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <R> Collection<R> call(Collection<? extends Callable<R>> 
jobs) throws IgniteCheckedException {
+    @Override public <R> Collection<R> call(Collection<? extends Callable<R>> 
jobs) {
         A.notEmpty(jobs, "jobs");
 
         guard();
@@ -272,14 +304,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().callAsync(BALANCE, jobs, 
prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <T, R> Collection<R> apply(final IgniteClosure<T, R> job,
-        @Nullable Collection<? extends T> args) throws IgniteCheckedException {
+    @Override public <T, R> Collection<R> apply(final IgniteClosure<T, R> job, 
@Nullable Collection<? extends T> args) {
         A.notNull(job, "job");
         A.notNull(args, "args");
 
@@ -288,14 +322,16 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().callAsync(job, args, prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public <R1, R2> R2 call(Collection<? extends Callable<R1>> jobs, 
IgniteReducer<R1, R2> rdc)
-        throws IgniteCheckedException {
+    @Override public <R1, R2> R2 call(Collection<? extends Callable<R1>> jobs, 
IgniteReducer<R1, R2> rdc) {
         A.notEmpty(jobs, "jobs");
         A.notNull(rdc, "rdc");
 
@@ -304,6 +340,9 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().forkjoinAsync(BALANCE, jobs, rdc, 
prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
@@ -311,7 +350,7 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
 
     /** {@inheritDoc} */
     @Override public <R1, R2, T> R2 apply(IgniteClosure<T, R1> job, 
Collection<? extends T> args,
-        IgniteReducer<R1, R2> rdc) throws IgniteCheckedException {
+        IgniteReducer<R1, R2> rdc) {
         A.notNull(job, "job");
         A.notNull(rdc, "rdc");
         A.notNull(args, "args");
@@ -321,6 +360,9 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
         try {
             return saveOrGet(ctx.closure().callAsync(job, args, rdc, 
prj.nodes()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
@@ -385,7 +427,7 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
     }
 
     /** {@inheritDoc} */
-    @Override public void localDeployTask(Class<? extends ComputeTask> 
taskCls, ClassLoader clsLdr) throws IgniteCheckedException {
+    @Override public void localDeployTask(Class<? extends ComputeTask> 
taskCls, ClassLoader clsLdr) {
         A.notNull(taskCls, "taskCls", clsLdr, "clsLdr");
 
         guard();
@@ -396,6 +438,9 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
             if (dep == null)
                 throw new IgniteDeploymentException("Failed to deploy task 
(was task (re|un)deployed?): " + taskCls);
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
@@ -414,7 +459,7 @@ public class IgniteComputeImpl extends 
IgniteAsyncSupportAdapter<IgniteCompute>
     }
 
     /** {@inheritDoc} */
-    @Override public void undeployTask(String taskName) throws 
IgniteCheckedException {
+    @Override public void undeployTask(String taskName) {
         A.notNull(taskName, "taskName");
 
         guard();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/internal/IgniteDeploymentCheckedException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDeploymentCheckedException.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDeploymentCheckedException.java
new file mode 100644
index 0000000..03450f1
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDeploymentCheckedException.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Deployment or re-deployment failed.
+ */
+public class IgniteDeploymentCheckedException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates new exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public IgniteDeploymentCheckedException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates new exception with given throwable as a nested cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public IgniteDeploymentCheckedException(Throwable cause) {
+        this(cause.getMessage(), cause);
+    }
+
+    /**
+     * Creates a new exception with given error message and optional nested 
cause exception.
+     *
+     * @param msg Error message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public IgniteDeploymentCheckedException(String msg, @Nullable Throwable 
cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/modules/core/src/main/java/org/apache/ignite/internal/IgniteEventsImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEventsImpl.java 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEventsImpl.java
index a63dfef..53b54a2 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEventsImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEventsImpl.java
@@ -67,7 +67,7 @@ public class IgniteEventsImpl extends 
IgniteAsyncSupportAdapter<IgniteEvents> im
 
     /** {@inheritDoc} */
     @Override public <T extends IgniteEvent> List<T> 
remoteQuery(IgnitePredicate<T> p, long timeout,
-        @Nullable int... types) throws IgniteCheckedException {
+        @Nullable int... types) {
         A.notNull(p, "p");
 
         guard();
@@ -75,6 +75,9 @@ public class IgniteEventsImpl extends 
IgniteAsyncSupportAdapter<IgniteEvents> im
         try {
             return 
saveOrGet(ctx.event().remoteEventsAsync(compoundPredicate(p, types), 
prj.nodes(), timeout));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
@@ -82,14 +85,14 @@ public class IgniteEventsImpl extends 
IgniteAsyncSupportAdapter<IgniteEvents> im
 
     /** {@inheritDoc} */
     @Override public <T extends IgniteEvent> UUID remoteListen(@Nullable 
IgniteBiPredicate<UUID, T> locLsnr,
-        @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) throws 
IgniteCheckedException {
+        @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) {
         return remoteListen(1, 0, true, locLsnr, rmtFilter, types);
     }
 
     /** {@inheritDoc} */
     @Override public <T extends IgniteEvent> UUID remoteListen(int bufSize, 
long interval,
         boolean autoUnsubscribe, @Nullable IgniteBiPredicate<UUID, T> locLsnr, 
@Nullable IgnitePredicate<T> rmtFilter,
-        @Nullable int... types) throws IgniteCheckedException {
+        @Nullable int... types) {
         A.ensure(bufSize > 0, "bufSize > 0");
         A.ensure(interval >= 0, "interval >= 0");
 
@@ -100,13 +103,16 @@ public class IgniteEventsImpl extends 
IgniteAsyncSupportAdapter<IgniteEvents> im
                 new GridEventConsumeHandler((IgniteBiPredicate<UUID, 
IgniteEvent>)locLsnr,
                     (IgnitePredicate<IgniteEvent>)rmtFilter, types), bufSize, 
interval, autoUnsubscribe, prj.predicate()));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void stopRemoteListen(UUID opId) throws 
IgniteCheckedException {
+    @Override public void stopRemoteListen(UUID opId) {
         A.notNull(opId, "consumeId");
 
         guard();
@@ -114,6 +120,9 @@ public class IgniteEventsImpl extends 
IgniteAsyncSupportAdapter<IgniteEvents> im
         try {
             saveOrGet(ctx.continuous().stopRoutine(opId));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
@@ -121,12 +130,15 @@ public class IgniteEventsImpl extends 
IgniteAsyncSupportAdapter<IgniteEvents> im
 
     /** {@inheritDoc} */
     @Override public <T extends IgniteEvent> T waitForLocal(@Nullable 
IgnitePredicate<T> filter,
-        @Nullable int... types) throws IgniteCheckedException {
+        @Nullable int... types) {
         guard();
 
         try {
             return saveOrGet(ctx.event().waitForEvent(filter, types));
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7cdd0c17/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 96631ad..db4ea55 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
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.portable.*;
@@ -2727,7 +2728,7 @@ public class IgniteKernal extends ClusterGroupAdapter 
implements IgniteEx, Ignit
                         try {
                             
compute(forNodes(neighbors)).execute(GridKillTask.class, false);
                         }
-                        catch (ClusterGroupEmptyException ignored) {
+                        catch (ClusterGroupEmptyCheckedException ignored) {
                             // No-op, nothing to restart.
                         }
                     }

Reply via email to