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

Branch: refs/heads/ignite-26
Commit: 52440dfb7b66e02c79b1662ae32134917e0a2a1a
Parents: d5bef13
Author: sboikov <sboi...@gridgain.com>
Authored: Thu Jan 29 13:55:25 2015 +0300
Committer: sboikov <sboi...@gridgain.com>
Committed: Thu Jan 29 14:21:20 2015 +0300

----------------------------------------------------------------------
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   3 +-
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java |   5 +-
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |   5 +-
 .../java/org/apache/ignite/IgniteCluster.java   |  13 +-
 .../org/apache/ignite/IgniteDataLoader.java     |  28 +-
 .../ignite/IgniteInterruptedException.java      |   5 +-
 .../CacheConsistentHashAffinityFunction.java    |   3 +-
 .../datastructures/CacheCountDownLatch.java     |  12 +-
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |   3 +-
 .../impl/GridClientAbstractProjection.java      |   6 +-
 .../apache/ignite/hadoop/GridHadoopTask.java    |   2 +-
 .../org/apache/ignite/internal/GridKernal.java  |  34 +-
 .../ignite/internal/IgniteClusterAsyncImpl.java |   4 +-
 .../IgniteFutureCancelledCheckedException.java  |  58 ++
 .../IgniteFutureTimeoutCheckedException.java    |  58 ++
 .../ignite/internal/IgniteInternalFuture.java   |  16 +-
 .../IgniteInterruptedCheckedException.java      |  58 ++
 .../internal/executor/GridExecutorService.java  |   4 +-
 .../internal/fs/common/GridGgfsLogger.java      |   6 +-
 .../discovery/GridDiscoveryManager.java         |   2 +-
 .../processors/cache/CacheLockImpl.java         |   2 +-
 .../processors/cache/GridCacheAdapter.java      |   6 +-
 .../cache/GridCacheEvictionManager.java         |   4 +-
 .../GridCachePartitionExchangeManager.java      |  10 +-
 .../processors/cache/GridCacheTtlManager.java   |   3 +-
 .../cache/GridCacheWriteBehindStore.java        |  14 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   2 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   2 +-
 .../preloader/GridDhtPartitionDemandPool.java   |  10 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   3 +-
 .../GridDhtPartitionsExchangeFuture.java        |   6 +-
 .../query/GridCacheDistributedQueryFuture.java  |   3 +-
 .../query/GridCacheDistributedQueryManager.java |   2 +-
 .../query/GridCacheQueryFutureAdapter.java      |   7 +-
 .../clock/GridClockDeltaSnapshot.java           |   8 +-
 .../processors/clock/GridClockServer.java       |   2 +-
 .../clock/GridClockSyncProcessor.java           |   2 +-
 .../continuous/GridContinuousProcessor.java     |   4 +-
 .../dataload/GridDataLoaderProcessor.java       |   4 +-
 .../dataload/IgniteDataLoaderImpl.java          |  20 +-
 .../processors/fs/GridGgfsDataManager.java      |  10 +-
 .../processors/fs/GridGgfsDeleteWorker.java     |   5 +-
 .../processors/fs/GridGgfsFileWorkerBatch.java  |   3 +-
 .../fs/GridGgfsFragmentizerManager.java         |   7 +-
 .../processors/fs/GridGgfsMetaManager.java      |   2 +-
 .../internal/processors/fs/GridGgfsServer.java  |   4 +-
 .../internal/processors/job/GridJobWorker.java  |   4 +-
 .../processors/query/GridQueryProcessor.java    |   4 +-
 .../service/GridServiceProcessor.java           |   4 +-
 .../processors/streamer/IgniteStreamerImpl.java |  12 +-
 .../internal/util/GridAtomicInitializer.java    |   5 +-
 .../apache/ignite/internal/util/GridUtils.java  |  73 ++-
 .../util/future/GridCompoundFuture.java         |   2 +-
 .../internal/util/future/GridFutureAdapter.java |  14 +-
 .../util/future/GridFutureAdapterEx.java        |  12 +-
 .../internal/util/future/IgniteFutureImpl.java  | 202 +++++++
 .../GridIpcSharedMemoryServerEndpoint.java      |   7 +-
 .../ignite/internal/util/lang/GridFunc.java     |  12 +-
 .../internal/util/lang/GridPlainFuture.java     |   2 +-
 .../util/lang/GridPlainFutureAdapter.java       |   4 +-
 .../ignite/internal/util/nio/GridNioFuture.java |  16 +-
 .../internal/util/nio/GridNioFutureImpl.java    |  13 +-
 .../ignite/internal/util/nio/GridNioServer.java |  11 +-
 .../ignite/internal/util/worker/GridWorker.java |   7 +-
 .../org/apache/ignite/lang/IgniteFuture.java    | 141 +++++
 .../lang/IgniteFutureCancelledException.java    |   2 +-
 .../lang/IgniteFutureTimeoutException.java      |   2 +-
 .../IgniteOptimizedObjectStreamRegistry.java    |  15 +-
 .../ignite/scheduler/SchedulerFuture.java       |  10 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   3 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    |  11 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   8 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   3 +-
 .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java |   3 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   3 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |   3 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   7 +-
 .../window/StreamerBoundedSizeBatchWindow.java  |  11 +-
 .../window/StreamerBoundedTimeBatchWindow.java  |  11 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |   8 +-
 .../GridTaskCancelSingleNodeSelfTest.java       |   2 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |   4 +-
 .../events/GridEventStorageManagerSelfTest.java |   4 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   2 +-
 .../cache/GridCacheGenericTestStore.java        |   3 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   3 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |   4 +-
 .../processors/cache/GridCacheTestStore.java    |   3 +-
 ...BehindStorePartitionedMultiNodeSelfTest.java |   6 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |  10 +-
 .../distributed/GridCacheEventAbstractTest.java |   2 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   3 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   3 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   2 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   3 +-
 .../GridCachePartitionedEntryLockSelfTest.java  |   3 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   2 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   5 +-
 .../local/GridCacheDaemonNodeLocalSelfTest.java |   2 +-
 .../GridDataLoaderProcessorSelfTest.java        |   2 +-
 ...heGgfsPerBlockLruEvictionPolicySelfTest.java |   3 +-
 .../processors/fs/GridGgfsAbstractSelfTest.java |   4 +-
 .../streamer/GridStreamerSelfTest.java          |   3 +-
 .../util/future/GridEmbeddedFutureSelfTest.java |   2 +-
 .../util/future/GridFutureAdapterSelfTest.java  |  17 +-
 .../util/future/IgniteFutureImplTest.java       | 525 +++++++++++++++++++
 .../util/future/nio/GridNioFutureSelfTest.java  |  12 +-
 .../communication/GridIoManagerBenchmark.java   |   4 +-
 .../GridContinuousOperationsLoadTest.java       |  10 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   2 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |   4 +-
 .../streamer/GridStreamerIndexLoadTest.java     |   2 +-
 .../ignite/loadtests/streamer/QueryClosure.java |   5 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   2 +-
 .../index/GridStreamerIndexSelfTest.java        |   2 +-
 .../GridTestSafeThreadFactory.java              |   3 +-
 .../ignite/testframework/GridTestUtils.java     |  12 +-
 .../testsuites/IgniteLangSelfTestSuite.java     |   1 +
 .../internal/fs/hadoop/GridGgfsHadoopIpcIo.java |   2 +-
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |   4 +-
 .../taskexecutor/GridHadoopExecutorService.java |   3 +-
 .../hadoop/v2/GridHadoopV2CleanupTask.java      |   3 +-
 .../hadoop/v2/GridHadoopV2MapTask.java          |   3 +-
 .../hadoop/v2/GridHadoopV2ReduceTask.java       |   3 +-
 .../hadoop/v2/GridHadoopV2SetupTask.java        |   3 +-
 .../hadoop/v2/GridHadoopV2Splitter.java         |   3 +-
 .../hibernate/GridCacheHibernateBlobStore.java  |   3 +-
 .../query/h2/GridH2IndexRebuildTest.java        |   6 +-
 .../scalar/pimps/ScalarProjectionPimp.scala     |  12 +-
 .../processors/schedule/ScheduleFutureImpl.java |  16 +-
 .../util/nodestart/GridNodeCallableImpl.java    |   7 +-
 .../deployment/uri/GridUriDeploymentSpi.java    |   3 +-
 133 files changed, 1490 insertions(+), 379 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
----------------------------------------------------------------------
diff --git 
a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
 
b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
index 8bc0fb7..143d7cb 100644
--- 
a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
+++ 
b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
@@ -22,6 +22,7 @@ import com.amazonaws.auth.*;
 import com.amazonaws.services.s3.*;
 import com.amazonaws.services.s3.model.*;
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
@@ -276,7 +277,7 @@ public class S3CheckpointSpi extends IgniteSpiAdapter 
implements CheckpointSpi,
                     try {
                         U.sleep(200);
                     }
-                    catch (IgniteInterruptedException e) {
+                    catch (IgniteInterruptedCheckedException e) {
                         throw new IgniteSpiException("Thread has been 
interrupted.", e);
                     }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
----------------------------------------------------------------------
diff --git 
a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
 
b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
index f22ed3a..9e36f38 100644
--- 
a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
+++ 
b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
@@ -22,6 +22,7 @@ import com.amazonaws.auth.*;
 import com.amazonaws.services.s3.*;
 import com.amazonaws.services.s3.model.*;
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -249,7 +250,7 @@ public class TcpDiscoveryS3IpFinder extends 
TcpDiscoveryIpFinderAdapter {
                             try {
                                 U.sleep(200);
                             }
-                            catch (IgniteInterruptedException e) {
+                            catch (IgniteInterruptedCheckedException e) {
                                 throw new IgniteSpiException("Thread has been 
interrupted.", e);
                             }
                     }
@@ -269,7 +270,7 @@ public class TcpDiscoveryS3IpFinder extends 
TcpDiscoveryIpFinderAdapter {
             try {
                 U.await(initLatch);
             }
-            catch (IgniteInterruptedException e) {
+            catch (IgniteInterruptedCheckedException e) {
                 throw new IgniteSpiException("Thread has been interrupted.", 
e);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
 
b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
index d247a1a..2cabf8d 100644
--- 
a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
+++ 
b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
@@ -22,6 +22,7 @@ import com.amazonaws.auth.*;
 import com.amazonaws.services.s3.*;
 import com.amazonaws.services.s3.model.*;
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.checkpoint.*;
 import org.apache.ignite.testsuites.*;
@@ -206,9 +207,9 @@ public class S3CheckpointSpiSelfTest extends 
GridSpiAbstractTest<S3CheckpointSpi
      * Wrapper around {@link 
GridTestUtils#retryAssert(org.apache.ignite.IgniteLogger, int, long, 
GridAbsClosure)}.
      * Provides s3-specific timeouts.
      * @param assertion Closure with assertion inside.
-     * @throws org.apache.ignite.IgniteInterruptedException If was interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
was interrupted.
      */
-    private void assertWithRetries(GridAbsClosureX assertion) throws 
IgniteInterruptedException {
+    private void assertWithRetries(GridAbsClosureX assertion) throws 
IgniteInterruptedCheckedException {
         GridTestUtils.retryAssert(log, 6, 5000, assertion);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index b5bdbd9..29d4cf1 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite;
 
+import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -108,7 +109,7 @@ public interface IgniteCluster extends ClusterGroup, 
IgniteAsyncSupport {
      * <ul>
      * <li>For local caches it returns only local node mapped to all keys.</li>
      * <li>
-     *      For fully replicated caches, {@link 
org.apache.ignite.cache.affinity.CacheAffinityFunction} is
+     *      For fully replicated caches, {@link CacheAffinityFunction} is
      *      used to determine which keys are mapped to which groups of nodes.
      * </li>
      * <li>For partitioned caches, the returned map represents node-to-key 
affinity.</li>
@@ -117,10 +118,10 @@ public interface IgniteCluster extends ClusterGroup, 
IgniteAsyncSupport {
      * @param cacheName Cache name, if {@code null}, then default cache 
instance is used.
      * @param keys Cache keys to map to nodes.
      * @return Map of nodes to cache keys or empty map if there are no alive 
nodes for this cache.
-     * @throws IgniteCheckedException If failed to map cache keys.
+     * @throws IgniteException If failed to map cache keys.
      */
     public <K> Map<ClusterNode, Collection<K>> mapKeysToNodes(@Nullable String 
cacheName,
-        @Nullable Collection<? extends K> keys) throws IgniteCheckedException;
+        @Nullable Collection<? extends K> keys) throws IgniteException;
 
     /**
      * This method provides ability to detect which cache keys are mapped to 
which nodes
@@ -131,7 +132,7 @@ public interface IgniteCluster extends ClusterGroup, 
IgniteAsyncSupport {
      * <ul>
      * <li>For local caches it returns only local node ID.</li>
      * <li>
-     *      For fully replicated caches first node ID returned by {@link 
org.apache.ignite.cache.affinity.CacheAffinityFunction}
+     *      For fully replicated caches first node ID returned by {@link 
CacheAffinityFunction}
      *      is returned.
      * </li>
      * <li>For partitioned caches, the returned node ID is the primary node 
for the key.</li>
@@ -141,9 +142,9 @@ public interface IgniteCluster extends ClusterGroup, 
IgniteAsyncSupport {
      * @param key Cache key to map to a node.
      * @return Primary node for the key or {@code null} if cache with given 
name
      *      is not present in the grid.
-     * @throws IgniteCheckedException If failed to map key.
+     * @throws IgniteException If failed to map key.
      */
-    @Nullable public <K> ClusterNode mapKeyToNode(@Nullable String cacheName, 
K key) throws IgniteCheckedException;
+    @Nullable public <K> ClusterNode mapKeyToNode(@Nullable String cacheName, 
K key) throws IgniteException;
 
     /**
      * Starts one or more nodes on remote host(s).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
index aa77747..796e0c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
@@ -236,11 +236,11 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * @param key Key.
      * @return Future fo this operation.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws IgniteInterruptedException If thread has been interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteInternalFuture<?> removeData(K key)  throws 
IgniteCheckedException, IgniteInterruptedException, IllegalStateException;
+    public IgniteInternalFuture<?> removeData(K key)  throws 
IgniteCheckedException, IgniteInterruptedCheckedException, 
IllegalStateException;
 
     /**
      * Adds data for loading on remote node. This method can be called from 
multiple
@@ -254,11 +254,11 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * @param val Value or {@code null} if respective entry must be removed 
from cache.
      * @return Future fo this operation.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws IgniteInterruptedException If thread has been interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteInternalFuture<?> addData(K key, @Nullable V val) throws 
IgniteCheckedException, IgniteInterruptedException,
+    public IgniteInternalFuture<?> addData(K key, @Nullable V val) throws 
IgniteCheckedException, IgniteInterruptedCheckedException,
         IllegalStateException;
 
     /**
@@ -272,11 +272,11 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * @param entry Entry.
      * @return Future fo this operation.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws IgniteInterruptedException If thread has been interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteInternalFuture<?> addData(Map.Entry<K, V> entry) throws 
IgniteCheckedException, IgniteInterruptedException,
+    public IgniteInternalFuture<?> addData(Map.Entry<K, V> entry) throws 
IgniteCheckedException, IgniteInterruptedCheckedException,
         IllegalStateException;
 
     /**
@@ -319,33 +319,33 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * use {@link #tryFlush()} method.
      *
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws IgniteInterruptedException If thread has been interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      * @see #tryFlush()
      */
-    public void flush() throws IgniteCheckedException, 
IgniteInterruptedException, IllegalStateException;
+    public void flush() throws IgniteCheckedException, 
IgniteInterruptedCheckedException, IllegalStateException;
 
     /**
      * Makes an attempt to load remaining data. This method is mostly similar 
to {@link #flush},
      * with the difference that it won't wait and will exit immediately.
      *
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws IgniteInterruptedException If thread has been interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      * @see #flush()
      */
-    public void tryFlush() throws IgniteCheckedException, 
IgniteInterruptedException, IllegalStateException;
+    public void tryFlush() throws IgniteCheckedException, 
IgniteInterruptedCheckedException, IllegalStateException;
 
     /**
      * Loads any remaining data and closes this loader.
      *
      * @param cancel {@code True} to cancel ongoing loading operations.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws IgniteInterruptedException If thread has been interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread has been interrupted.
      */
-    public void close(boolean cancel) throws IgniteCheckedException, 
IgniteInterruptedException;
+    public void close(boolean cancel) throws IgniteCheckedException, 
IgniteInterruptedCheckedException;
 
     /**
      * Closes data loader. This method is identical to calling {@link 
#close(boolean) close(false)} method.
@@ -354,7 +354,7 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * {@code try-with-resources} statement.
      *
      * @throws IgniteCheckedException If failed to close data loader.
-     * @throws IgniteInterruptedException If thread has been interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread has been interrupted.
      */
-    @Override public void close() throws IgniteCheckedException, 
IgniteInterruptedException;
+    @Override public void close() throws IgniteCheckedException, 
IgniteInterruptedCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java
index 1e7f267..9d6f1ff 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java
@@ -18,10 +18,9 @@
 package org.apache.ignite;
 
 /**
- * This exception is used to wrap standard {@link InterruptedException} into 
{@link IgniteCheckedException}.
+ * This exception is used to wrap standard {@link InterruptedException} into 
{@link IgniteException}.
  */
-@SuppressWarnings({"TypeMayBeWeakened"})
-public class IgniteInterruptedException extends IgniteCheckedException {
+public class IgniteInterruptedException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
index 3305534..cd12dc4 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
@@ -589,7 +590,7 @@ public class CacheConsistentHashAffinityFunction implements 
CacheAffinityFunctio
                 try {
                     U.await(initLatch);
                 }
-                catch (IgniteInterruptedException ignored) {
+                catch (IgniteInterruptedCheckedException ignored) {
                     // Recover interrupted state flag.
                     Thread.currentThread().interrupt();
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheCountDownLatch.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheCountDownLatch.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheCountDownLatch.java
index 0f4aae0..ca83ba1 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheCountDownLatch.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheCountDownLatch.java
@@ -84,11 +84,11 @@ public interface CacheCountDownLatch {
      *      <li>has its interrupted status set on entry to this method; or
      *      <li>is interrupted while waiting,
      * </ul>
-     * then {@link org.apache.ignite.IgniteInterruptedException} is thrown and 
the current thread's
+     * then {@link 
org.apache.ignite.internal.IgniteInterruptedCheckedException} is thrown and the 
current thread's
      * interrupted status is cleared.
      *
      * @throws IgniteCheckedException If operation failed.
-     * @throws org.apache.ignite.IgniteInterruptedException if the current 
thread is interrupted
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException if 
the current thread is interrupted
      *      while waiting
      */
     public void await() throws IgniteCheckedException;
@@ -118,7 +118,7 @@ public interface CacheCountDownLatch {
      *      <li>has its interrupted status set on entry to this method; or
      *      <li>is interrupted while waiting,
      * </ul>
-     * then {@link org.apache.ignite.IgniteInterruptedException} is thrown and 
the current thread's
+     * then {@link 
org.apache.ignite.internal.IgniteInterruptedCheckedException} is thrown and the 
current thread's
      * interrupted status is cleared.
      * <p>
      * If the specified waiting time elapses then the value {@code false}
@@ -128,7 +128,7 @@ public interface CacheCountDownLatch {
      * @param timeout The maximum time to wait in milliseconds.
      * @return {@code True} if the count reached zero and {@code false}
      *      if the waiting time elapsed before the count reached zero.
-     * @throws org.apache.ignite.IgniteInterruptedException If the current 
thread is interrupted
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
the current thread is interrupted
      *      while waiting.
      * @throws IgniteCheckedException If operation failed.
      */
@@ -159,7 +159,7 @@ public interface CacheCountDownLatch {
      *      <li>has its interrupted status set on entry to this method; or
      *      <li>is interrupted while waiting,
      * </ul>
-     * then {@link org.apache.ignite.IgniteInterruptedException} is thrown and 
the current thread's
+     * then {@link 
org.apache.ignite.internal.IgniteInterruptedCheckedException} is thrown and the 
current thread's
      * interrupted status is cleared.
      * <p>
      * If the specified waiting time elapses then the value {@code false}
@@ -171,7 +171,7 @@ public interface CacheCountDownLatch {
      * @param unit The time unit of the {@code timeout} argument.
      * @return {@code True} if the count reached zero and {@code false}
      *      if the waiting time elapsed before the count reached zero.
-     * @throws org.apache.ignite.IgniteInterruptedException If the current 
thread is interrupted
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
the current thread is interrupted
      *      while waiting.
      * @throws IgniteCheckedException If operation failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
index b453741..ff581b3 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache.store.jdbc;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.tostring.*;
@@ -435,7 +436,7 @@ public class CacheJdbcBlobStore<K, V> extends 
CacheStoreAdapter<K, V> {
                 try {
                     U.await(initLatch);
                 }
-                catch (IgniteInterruptedException e) {
+                catch (IgniteInterruptedCheckedException e) {
                     throw new IgniteException(e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/client/impl/GridClientAbstractProjection.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/client/impl/GridClientAbstractProjection.java
 
b/modules/core/src/main/java/org/apache/ignite/client/impl/GridClientAbstractProjection.java
index 7f209d7..511ba20 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/client/impl/GridClientAbstractProjection.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/client/impl/GridClientAbstractProjection.java
@@ -17,11 +17,11 @@
 
 package org.apache.ignite.client.impl;
 
-import org.apache.ignite.*;
 import org.apache.ignite.client.*;
 import org.apache.ignite.client.balancer.*;
 import org.apache.ignite.client.impl.connection.*;
 import org.apache.ignite.client.util.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -143,7 +143,7 @@ abstract class GridClientAbstractProjection<T extends 
GridClientAbstractProjecti
         catch (GridClientException e) {
             return new GridClientFutureAdapter<>(e);
         }
-        catch (IgniteInterruptedException | InterruptedException e) {
+        catch (IgniteInterruptedCheckedException | InterruptedException e) {
             Thread.currentThread().interrupt();
 
             return new GridClientFutureAdapter<>(
@@ -215,7 +215,7 @@ abstract class GridClientAbstractProjection<T extends 
GridClientAbstractProjecti
         catch (GridClientException e) {
             return new GridClientFutureAdapter<>(e);
         }
-        catch (IgniteInterruptedException | InterruptedException e) {
+        catch (IgniteInterruptedCheckedException | InterruptedException e) {
             Thread.currentThread().interrupt();
 
             return new GridClientFutureAdapter<>(new 
GridClientException("Interrupted when (re)trying to perform " +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoopTask.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoopTask.java 
b/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoopTask.java
index 9a99226..515dc52 100644
--- a/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoopTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoopTask.java
@@ -60,7 +60,7 @@ public abstract class GridHadoopTask {
      * Runs task.
      *
      * @param taskCtx Context.
-     * @throws org.apache.ignite.IgniteInterruptedException If interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
interrupted.
      * @throws IgniteCheckedException If failed.
      */
     public abstract void run(GridHadoopTaskContext taskCtx) throws 
IgniteCheckedException;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java 
b/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
index 8ffafac..9d51ac0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
@@ -823,7 +823,7 @@ public class GridKernal extends ClusterGroupAdapter 
implements GridEx, IgniteMXB
 
             if (verCheckErr != null)
                 U.error(log, verCheckErr.getMessage());
-            else if (X.hasCause(e, InterruptedException.class, 
IgniteInterruptedException.class))
+            else if (X.hasCause(e, InterruptedException.class, 
IgniteInterruptedCheckedException.class))
                 U.warn(log, "Grid startup routine has been interrupted (will 
rollback).");
             else
                 U.error(log, "Got exception while starting (will rollback 
startup routine).", e);
@@ -2624,7 +2624,7 @@ public class GridKernal extends ClusterGroupAdapter 
implements GridEx, IgniteMXB
      * @param maxConn Number of parallel SSH connections to one host.
      * @return Future with results.
      * @throws IgniteCheckedException In case of error.
-     * @see {@link org.apache.ignite.IgniteCluster#startNodes(java.io.File, 
boolean, int, int)}.
+     * @see {@link IgniteCluster#startNodes(java.io.File, boolean, int, int)}.
      */
     IgniteInternalFuture<Collection<GridTuple3<String, Boolean, String>>> 
startNodesAsync(File file, boolean restart,                                     
                                                       int timeout, int 
maxConn) throws IgniteCheckedException {
         A.notNull(file, "file");
@@ -2653,8 +2653,13 @@ public class GridKernal extends ClusterGroupAdapter 
implements GridEx, IgniteMXB
 
     /** {@inheritDoc} */
     @Override public Collection<GridTuple3<String, Boolean, String>> 
startNodes(
-        Collection<Map<String, Object>> hosts, @Nullable Map<String, Object> 
dflts, boolean restart, int timeout,
-        int maxConn) throws IgniteCheckedException {
+        Collection<Map<String, Object>> hosts,
+        @Nullable Map<String, Object> dflts,
+        boolean restart,
+        int timeout,
+        int maxConn)
+        throws IgniteCheckedException
+    {
         return startNodesAsync(hosts, dflts, restart, timeout, maxConn).get();
     }
 
@@ -2666,11 +2671,16 @@ public class GridKernal extends ClusterGroupAdapter 
implements GridEx, IgniteMXB
      * @param maxConn Number of parallel SSH connections to one host.
      * @return Future with results.
      * @throws IgniteCheckedException In case of error.
-     * @see {@link 
org.apache.ignite.IgniteCluster#startNodes(java.util.Collection, java.util.Map, 
boolean, int, int)}.
+     * @see {@link IgniteCluster#startNodes(java.util.Collection, 
java.util.Map, boolean, int, int)}.
      */
     IgniteInternalFuture<Collection<GridTuple3<String, Boolean, String>>> 
startNodesAsync(
-        Collection<Map<String, Object>> hosts, @Nullable Map<String, Object> 
dflts, boolean restart, int timeout,
-        int maxConn) throws IgniteCheckedException {
+        Collection<Map<String, Object>> hosts,
+        @Nullable Map<String, Object> dflts,
+        boolean restart,
+        int timeout,
+        int maxConn)
+        throws IgniteCheckedException
+    {
         A.notNull(hosts, "hosts");
 
         guard();
@@ -3110,7 +3120,7 @@ public class GridKernal extends ClusterGroupAdapter 
implements GridEx, IgniteMXB
 
     /** {@inheritDoc} */
     @Override public <K> Map<ClusterNode, Collection<K>> mapKeysToNodes(String 
cacheName,
-        @Nullable Collection<? extends K> keys) throws IgniteCheckedException {
+        @Nullable Collection<? extends K> keys) {
         if (F.isEmpty(keys))
             return Collections.emptyMap();
 
@@ -3119,13 +3129,16 @@ public class GridKernal extends ClusterGroupAdapter 
implements GridEx, IgniteMXB
         try {
             return ctx.affinity().mapKeysToNodes(cacheName, keys);
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <K> ClusterNode mapKeyToNode(String cacheName, 
K key) throws IgniteCheckedException {
+    @Nullable @Override public <K> ClusterNode mapKeyToNode(String cacheName, 
K key) {
         A.notNull(key, "key");
 
         guard();
@@ -3133,6 +3146,9 @@ public class GridKernal extends ClusterGroupAdapter 
implements GridEx, IgniteMXB
         try {
             return ctx.affinity().mapKeyToNode(cacheName, key);
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
index a02c931..f426ad5 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
@@ -74,12 +74,12 @@ public class IgniteClusterAsyncImpl extends 
IgniteAsyncSupportAdapter<IgniteClus
 
     /** {@inheritDoc} */
     @Override public <K> Map<ClusterNode, Collection<K>> 
mapKeysToNodes(@Nullable String cacheName,
-        @Nullable Collection<? extends K> keys) throws IgniteCheckedException {
+        @Nullable Collection<? extends K> keys) {
         return grid.mapKeysToNodes(cacheName, keys);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <K> ClusterNode mapKeyToNode(@Nullable String 
cacheName, K key) throws IgniteCheckedException {
+    @Nullable @Override public <K> ClusterNode mapKeyToNode(@Nullable String 
cacheName, K key) {
         return grid.mapKeyToNode(cacheName, key);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureCancelledCheckedException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureCancelledCheckedException.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureCancelledCheckedException.java
new file mode 100644
index 0000000..da2f738
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureCancelledCheckedException.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.*;
+
+/**
+ * Future computation cannot be retrieved because it was cancelled.
+ */
+public class IgniteFutureCancelledCheckedException extends 
IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates new exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public IgniteFutureCancelledCheckedException(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 IgniteFutureCancelledCheckedException(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 IgniteFutureCancelledCheckedException(String msg, @Nullable 
Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureTimeoutCheckedException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureTimeoutCheckedException.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureTimeoutCheckedException.java
new file mode 100644
index 0000000..1697577
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFutureTimeoutCheckedException.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.*;
+
+/**
+ * Future computation completion is timed out.
+ */
+public class IgniteFutureTimeoutCheckedException extends 
IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates new exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public IgniteFutureTimeoutCheckedException(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 IgniteFutureTimeoutCheckedException(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 IgniteFutureTimeoutCheckedException(String msg, @Nullable Throwable 
cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
index cc376fb..2860fb9 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
@@ -35,8 +35,8 @@ public interface IgniteInternalFuture<R> {
      * returns computation result.
      *
      * @return Computation result.
-     * @throws IgniteInterruptedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was interrupted.
-     * @throws IgniteFutureCancelledException Subclass of {@link 
IgniteCheckedException} throws if computation was cancelled.
+     * @throws IgniteInterruptedCheckedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was interrupted.
+     * @throws IgniteFutureCancelledCheckedException Subclass of {@link 
IgniteCheckedException} throws if computation was cancelled.
      * @throws IgniteCheckedException If computation failed.
      */
     public R get() throws IgniteCheckedException;
@@ -48,9 +48,9 @@ public interface IgniteInternalFuture<R> {
      *
      * @param timeout The maximum time to wait in milliseconds.
      * @return Computation result.
-     * @throws IgniteInterruptedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was interrupted.
-     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of 
{@link IgniteCheckedException} thrown if the wait was timed out.
-     * @throws IgniteFutureCancelledException Subclass of {@link 
IgniteCheckedException} throws if computation was cancelled.
+     * @throws IgniteInterruptedCheckedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was interrupted.
+     * @throws IgniteFutureTimeoutCheckedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was timed out.
+     * @throws IgniteFutureCancelledCheckedException Subclass of {@link 
IgniteCheckedException} throws if computation was cancelled.
      * @throws IgniteCheckedException If computation failed.
      */
     public R get(long timeout) throws IgniteCheckedException;
@@ -62,9 +62,9 @@ public interface IgniteInternalFuture<R> {
      * @param timeout The maximum time to wait.
      * @param unit The time unit of the {@code timeout} argument.
      * @return Computation result.
-     * @throws IgniteInterruptedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was interrupted.
-     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of 
{@link IgniteCheckedException} thrown if the wait was timed out.
-     * @throws IgniteFutureCancelledException Subclass of {@link 
IgniteCheckedException} throws if computation was cancelled.
+     * @throws IgniteInterruptedCheckedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was interrupted.
+     * @throws IgniteFutureTimeoutCheckedException Subclass of {@link 
IgniteCheckedException} thrown if the wait was timed out.
+     * @throws IgniteFutureCancelledCheckedException Subclass of {@link 
IgniteCheckedException} throws if computation was cancelled.
      * @throws IgniteCheckedException If computation failed.
      */
     public R get(long timeout, TimeUnit unit) throws IgniteCheckedException;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/IgniteInterruptedCheckedException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInterruptedCheckedException.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInterruptedCheckedException.java
new file mode 100644
index 0000000..282217d
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInterruptedCheckedException.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.*;
+
+/**
+ * This exception is used to wrap standard {@link InterruptedException} into 
{@link IgniteCheckedException}.
+ */
+@SuppressWarnings({"TypeMayBeWeakened"})
+public class IgniteInterruptedCheckedException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates new exception with given throwable as a nested cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public IgniteInterruptedCheckedException(InterruptedException cause) {
+        this(cause.getMessage(), cause);
+    }
+
+    /**
+     * Creates a new exception with given error message and optional nested 
cause exception.
+     *
+     * @param msg Error message.
+     */
+    public IgniteInterruptedCheckedException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * 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 IgniteInterruptedCheckedException(String msg, InterruptedException 
cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
index 019fd59..0074585 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
@@ -528,7 +528,7 @@ public class GridExecutorService implements 
ExecutorService, Externalizable {
                     // Cancel next tasks (avoid current task cancellation 
below in loop).
                     continue;
                 }
-                catch (IgniteFutureTimeoutException ignored) {
+                catch (IgniteFutureTimeoutCheckedException ignored) {
                     if (log.isDebugEnabled())
                         log.debug("Timeout occurred during getting task 
result: " + fut);
 
@@ -708,7 +708,7 @@ public class GridExecutorService implements 
ExecutorService, Externalizable {
 
                 return res;
             }
-            catch (IgniteFutureTimeoutException e) {
+            catch (IgniteFutureTimeoutCheckedException e) {
                 TimeoutException e2 = new TimeoutException();
 
                 e2.initCause(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/fs/common/GridGgfsLogger.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/fs/common/GridGgfsLogger.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/fs/common/GridGgfsLogger.java
index 3c01835..075d6f1 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/fs/common/GridGgfsLogger.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/fs/common/GridGgfsLogger.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.fs.common;
 
-import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jdk8.backport.*;
 
@@ -444,7 +444,7 @@ public final class GridGgfsLogger {
             try {
                 U.join(flushWorker);
             }
-            catch (IgniteInterruptedException ignore) {
+            catch (IgniteInterruptedCheckedException ignore) {
                 // No-op.
             }
 
@@ -694,7 +694,7 @@ public final class GridGgfsLogger {
                         try {
                             U.await(flushCond, 1000L, TimeUnit.MILLISECONDS);
                         }
-                        catch (IgniteInterruptedException ignore) {
+                        catch (IgniteInterruptedCheckedException ignore) {
                             t.interrupt();
 
                             break;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index a4d0fa6..9d53f23 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -1591,7 +1591,7 @@ public class GridDiscoveryManager extends 
GridManagerAdapter<DiscoverySpi> {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws IgniteInterruptedException {
+        @Override protected void body() throws 
IgniteInterruptedCheckedException {
             while (!isCancelled()) {
                 U.sleep(METRICS_UPDATE_FREQ);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
index 8788eac..a35f8a1 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
@@ -137,7 +137,7 @@ class CacheLockImpl<K, V> implements Lock {
 
                 return res;
             }
-            catch (IgniteInterruptedException e) {
+            catch (IgniteInterruptedCheckedException e) {
                 if (!fut.cancel()) {
                     if (fut.isDone()) {
                         Boolean res = fut.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 92d2039..3c4b075 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -3469,7 +3469,7 @@ public abstract class GridCacheAdapter<K, V> implements 
GridCache<K, V>,
                 try {
                     return fut.get();
                 }
-                catch (IgniteInterruptedException ignored) {
+                catch (IgniteInterruptedCheckedException ignored) {
                     // Interrupted status of current thread was cleared, retry 
to get lock.
                     isInterrupted = true;
                 }
@@ -4220,7 +4220,7 @@ public abstract class GridCacheAdapter<K, V> implements 
GridCache<K, V>,
 
                 return t;
             }
-            catch (IgniteInterruptedException | IgniteTxHeuristicException | 
IgniteTxRollbackException e) {
+            catch (IgniteInterruptedCheckedException | 
IgniteTxHeuristicException | IgniteTxRollbackException e) {
                 throw e;
             }
             catch (IgniteCheckedException e) {
@@ -4392,7 +4392,7 @@ public abstract class GridCacheAdapter<K, V> implements 
GridCache<K, V>,
         catch (InterruptedException e) {
             Thread.currentThread().interrupt();
 
-            return new GridFinishedFutureEx<>(new 
IgniteInterruptedException("Failed to wait for asynchronous " +
+            return new GridFinishedFutureEx<>(new 
IgniteInterruptedCheckedException("Failed to wait for asynchronous " +
                 "operation permit (thread got interrupted).", e));
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index 837a6e4..fc24a02 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -1120,7 +1120,7 @@ public class GridCacheEvictionManager<K, V> extends 
GridCacheManagerAdapter<K, V
             try {
                 t = fut.get();
             }
-            catch (IgniteFutureCancelledException ignored) {
+            catch (IgniteFutureCancelledCheckedException ignored) {
                 assert false : "Future has been cancelled, but manager is not 
stopping: " + fut;
 
                 return;
@@ -1387,7 +1387,7 @@ public class GridCacheEvictionManager<K, V> extends 
GridCacheManagerAdapter<K, V
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             try {
                 assert !cctx.isNear() && evictSync;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 4cb2d10..190e351 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -228,7 +228,7 @@ public class GridCachePartitionExchangeManager<K, V> 
extends GridCacheSharedMana
 
                     break;
                 }
-                catch (IgniteFutureTimeoutException ignored) {
+                catch (IgniteFutureTimeoutCheckedException ignored) {
                     if (first) {
                         U.warn(log, "Failed to wait for initial partition map 
exchange. " +
                             "Possible reasons are: " + U.nl() +
@@ -246,7 +246,7 @@ public class GridCachePartitionExchangeManager<K, V> 
extends GridCacheSharedMana
             for (GridCacheContext<K, V> cacheCtx : cctx.cacheContexts())
                 cacheCtx.preloader().onInitialExchangeComplete(null);
         }
-        catch (IgniteFutureTimeoutException e) {
+        catch (IgniteFutureTimeoutCheckedException e) {
             IgniteCheckedException err = new IgniteCheckedException("Timed out 
waiting for exchange future: " + fut, e);
 
             for (GridCacheContext<K, V> cacheCtx : cctx.cacheContexts())
@@ -263,7 +263,7 @@ public class GridCachePartitionExchangeManager<K, V> 
extends GridCacheSharedMana
     @Override protected void onKernalStop0(boolean cancel) {
         // Finish all exchange futures.
         for (GridDhtPartitionsExchangeFuture<K, V> f : exchFuts.values())
-            f.onDone(new IgniteInterruptedException("Grid is stopping: " + 
cctx.gridName()));
+            f.onDone(new IgniteInterruptedCheckedException("Grid is stopping: 
" + cctx.gridName()));
 
         U.cancel(exchWorker);
 
@@ -734,7 +734,7 @@ public class GridCachePartitionExchangeManager<K, V> 
extends GridCacheSharedMana
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             long timeout = cctx.gridConfig().getNetworkTimeout();
 
             boolean startEvtFired = false;
@@ -861,7 +861,7 @@ public class GridCachePartitionExchangeManager<K, V> 
extends GridCacheSharedMana
                         }
                     }
                 }
-                catch (IgniteInterruptedException e) {
+                catch (IgniteInterruptedCheckedException e) {
                     throw e;
                 }
                 catch (IgniteCheckedException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index 9b37c86..885b64c 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.thread.*;
@@ -108,7 +109,7 @@ public class GridCacheTtlManager<K, V> extends 
GridCacheManagerAdapter<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             while (!isCancelled()) {
                 long now = U.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
index a31d3b5..8b44d3a 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
@@ -467,7 +467,7 @@ public class GridCacheWriteBehindStore<K, V> extends 
CacheStore<K, V> implements
 
             updateCache(entry.getKey(), entry, StoreOperation.PUT);
         }
-        catch (IgniteInterruptedException e) {
+        catch (IgniteInterruptedCheckedException e) {
             throw new CacheWriterException(e);
         }
     }
@@ -487,7 +487,7 @@ public class GridCacheWriteBehindStore<K, V> extends 
CacheStore<K, V> implements
 
             updateCache((K)key, null, StoreOperation.RMV);
         }
-        catch (IgniteInterruptedException e) {
+        catch (IgniteInterruptedCheckedException e) {
             throw new CacheWriterException(e);
         }
     }
@@ -508,12 +508,12 @@ public class GridCacheWriteBehindStore<K, V> extends 
CacheStore<K, V> implements
      * @param key Key for which update is performed.
      * @param val New value, may be null for remove operation.
      * @param operation Updated value status
-     * @throws org.apache.ignite.IgniteInterruptedException If interrupted 
while waiting for value to be flushed.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
interrupted while waiting for value to be flushed.
      */
     private void updateCache(K key,
         @Nullable Entry<? extends K, ? extends V> val,
         StoreOperation operation)
-        throws IgniteInterruptedException {
+        throws IgniteInterruptedCheckedException {
         StatefulValue<K, V> newVal = new StatefulValue<>(val, operation);
 
         StatefulValue<K, V> prev;
@@ -743,7 +743,7 @@ public class GridCacheWriteBehindStore<K, V> extends 
CacheStore<K, V> implements
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             while (!stopping.get() || writeCache.sizex() > 0) {
                 awaitOperationsAvailable();
 
@@ -978,9 +978,9 @@ public class GridCacheWriteBehindStore<K, V> extends 
CacheStore<K, V> implements
         /**
          * Awaits a signal on flush condition
          *
-         * @throws org.apache.ignite.IgniteInterruptedException If thread was 
interrupted.
+         * @throws 
org.apache.ignite.internal.IgniteInterruptedCheckedException If thread was 
interrupted.
          */
-        private void waitForFlush() throws IgniteInterruptedException {
+        private void waitForFlush() throws IgniteInterruptedCheckedException {
             U.await(flushCond);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index a3a0ab7..d8a8651 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -244,7 +244,7 @@ public class GridDhtLocalPartition<K, V> implements 
Comparable<GridDhtLocalParti
         catch (InterruptedException e) {
             Thread.currentThread().interrupt();
 
-            throw new IgniteInterruptedException(e);
+            throw new IgniteInterruptedCheckedException(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 947bec4..513e461 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -420,7 +420,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends 
GridCompoundIdentityFutu
             try {
                 get();
             }
-            catch (IgniteInterruptedException e) {
+            catch (IgniteInterruptedCheckedException e) {
                 onError(new IgniteCheckedException("Got interrupted while 
waiting for replies to be sent.", e));
             }
             catch (IgniteCheckedException ignored) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 16a56e2..f4d8956 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -550,7 +550,7 @@ public final class GridDhtForceKeysFuture<K, V> extends 
GridCompoundFuture<Objec
 
                 return true;
             }
-            catch (IgniteInterruptedException e) {
+            catch (IgniteInterruptedCheckedException e) {
                 // Fail.
                 onDone(e);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index 8a6fef8..f183d5f 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -478,7 +478,7 @@ public class GridDhtPartitionDemandPool<K, V> {
          * @param entry Preloaded entry.
          * @param topVer Topology version.
          * @return {@code False} if partition has become invalid during 
preloading.
-         * @throws org.apache.ignite.IgniteInterruptedException If interrupted.
+         * @throws 
org.apache.ignite.internal.IgniteInterruptedCheckedException If interrupted.
          */
         private boolean preloadEntry(ClusterNode pick, int p, 
GridCacheEntryInfo<K, V> entry, long topVer)
             throws IgniteCheckedException {
@@ -539,7 +539,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                     return false;
                 }
             }
-            catch (IgniteInterruptedException e) {
+            catch (IgniteInterruptedCheckedException e) {
                 throw e;
             }
             catch (IgniteCheckedException e) {
@@ -806,7 +806,7 @@ public class GridDhtPartitionDemandPool<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             try {
                 int preloadOrder = cctx.config().getPreloadOrder();
 
@@ -822,7 +822,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                             fut.get();
                         }
                     }
-                    catch (IgniteInterruptedException ignored) {
+                    catch (IgniteInterruptedCheckedException ignored) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to wait for ordered preload 
future (grid is stopping): " +
                                 "[cacheName=" + cctx.name() + ", 
preloadOrder=" + preloadOrder + ']');
@@ -902,7 +902,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                                         missed.addAll(set);
                                     }
                                 }
-                                catch (IgniteInterruptedException e) {
+                                catch (IgniteInterruptedCheckedException e) {
                                     throw e;
                                 }
                                 catch (ClusterTopologyException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
index 865a16e..24a0bc5 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
@@ -19,6 +19,7 @@ package 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.thread.*;
@@ -208,7 +209,7 @@ class GridDhtPartitionSupplyPool<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             while (!isCancelled()) {
                 DemandMessage<K, V> msg = poll(queue, this);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 219737f..9f0bc31 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -404,9 +404,9 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends 
GridFutureAdapter<Lon
     /**
      * Starts activity.
      *
-     * @throws org.apache.ignite.IgniteInterruptedException If interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
interrupted.
      */
-    public void init() throws IgniteInterruptedException {
+    public void init() throws IgniteInterruptedCheckedException {
         assert oldestNode.get() != null;
 
         if (init.compareAndSet(false, true)) {
@@ -494,7 +494,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends 
GridFutureAdapter<Lon
                     top.beforeExchange(exchId);
                 }
             }
-            catch (IgniteInterruptedException e) {
+            catch (IgniteInterruptedCheckedException e) {
                 onDone(e);
 
                 throw e;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
index b80b2b6..0ab0645 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -192,7 +193,7 @@ public class GridCacheDistributedQueryFuture<K, V, R> 
extends GridCacheQueryFutu
 
     /** {@inheritDoc} */
     @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-    @Override protected void loadAllPages() throws IgniteInterruptedException {
+    @Override protected void loadAllPages() throws 
IgniteInterruptedCheckedException {
         assert !Thread.holdsLock(mux);
 
         U.await(firstPageLatch);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index cb7e9eb..b3aeff0 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -300,7 +300,7 @@ public class GridCacheDistributedQueryManager<K, V> extends 
GridCacheQueryManage
                         try {
                             U.sleep(RESEND_FREQ);
                         }
-                        catch (IgniteInterruptedException e1) {
+                        catch (IgniteInterruptedCheckedException e1) {
                             U.error(log,
                                 "Waiting for queries response resending was 
interrupted (response will not be sent) " +
                                 "[nodeId=" + nodeId + ", response=" + res + 
"]", e1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index 4202c99..ef45148 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.timeout.*;
@@ -496,9 +497,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> 
extends GridFutureAda
     /**
      * Loads all left pages.
      *
-     * @throws org.apache.ignite.IgniteInterruptedException If thread is 
interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
thread is interrupted.
      */
-    protected abstract void loadAllPages() throws IgniteInterruptedException;
+    protected abstract void loadAllPages() throws 
IgniteInterruptedCheckedException;
 
     /**
      * Clears future.
@@ -538,7 +539,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> 
extends GridFutureAda
         try {
             cancelQuery();
 
-            onDone(new IgniteFutureTimeoutException("Query timed out."));
+            onDone(new IgniteFutureTimeoutCheckedException("Query timed 
out."));
         }
         catch (IgniteCheckedException e) {
             onDone(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
index 7e135f5..e8c9586 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.clock;
 
-import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.tostring.*;
@@ -94,9 +94,9 @@ public class GridClockDeltaSnapshot {
      * Awaits either until snapshot is ready or timeout elapses.
      *
      * @param timeout Timeout to wait.
-     * @throws IgniteInterruptedException If wait was interrupted.
+     * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException If 
wait was interrupted.
      */
-    public synchronized void awaitReady(long timeout) throws 
IgniteInterruptedException {
+    public synchronized void awaitReady(long timeout) throws 
IgniteInterruptedCheckedException {
         long start = System.currentTimeMillis();
 
         try {
@@ -112,7 +112,7 @@ public class GridClockDeltaSnapshot {
         catch (InterruptedException e) {
             Thread.currentThread().interrupt();
 
-            throw new IgniteInterruptedException(e);
+            throw new IgniteInterruptedCheckedException(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
index f188181..f794177 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
@@ -174,7 +174,7 @@ public class GridClockServer {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             DatagramPacket packet = new DatagramPacket(new 
byte[GridClockMessage.PACKET_SIZE],
                 GridClockMessage.PACKET_SIZE);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
index d1c9931..f23bdfd 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
@@ -339,7 +339,7 @@ public class GridClockSyncProcessor extends 
GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, 
IgniteInterruptedException {
+        @Override protected void body() throws InterruptedException, 
IgniteInterruptedCheckedException {
             while (!isCancelled()) {
                 GridDiscoveryTopologySnapshot top = lastSnapshot;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52440dfb/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 4e84eac..e391c6e 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -1001,7 +1001,7 @@ public class GridContinuousProcessor extends 
GridProcessorAdapter {
                             try {
                                 U.sleep(interval0);
                             }
-                            catch (IgniteInterruptedException ignored) {
+                            catch (IgniteInterruptedCheckedException ignored) {
                                 break;
                             }
 
@@ -1153,7 +1153,7 @@ public class GridContinuousProcessor extends 
GridProcessorAdapter {
 
                     break;
                 }
-                catch (IgniteInterruptedException e) {
+                catch (IgniteInterruptedCheckedException e) {
                     throw e;
                 }
                 catch (IgniteCheckedException e) {

Reply via email to