Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7324895EE for ; Fri, 12 Dec 2014 07:15:28 +0000 (UTC) Received: (qmail 1142 invoked by uid 500); 12 Dec 2014 07:15:28 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 1104 invoked by uid 500); 12 Dec 2014 07:15:28 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 1090 invoked by uid 99); 12 Dec 2014 07:15:28 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 12 Dec 2014 07:15:28 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Fri, 12 Dec 2014 07:15:01 +0000 Received: (qmail 95526 invoked by uid 99); 12 Dec 2014 07:14:41 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 12 Dec 2014 07:14:41 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 09BFDA285FE; Fri, 12 Dec 2014 07:14:41 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Date: Fri, 12 Dec 2014 07:15:20 -0000 Message-Id: In-Reply-To: <858f2843d730461b9380ed10f06a037b@git.apache.org> References: <858f2843d730461b9380ed10f06a037b@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [42/64] [abbrv] [partial] incubator-ignite git commit: Rename GridException to IgniteCheckedException, GridRuntimeException to IgniteException. X-Virus-Checked: Checked by ClamAV on apache.org http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheDataStructures.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheDataStructures.java b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheDataStructures.java index eaa72ef..2fccd01 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheDataStructures.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheDataStructures.java @@ -9,7 +9,7 @@ package org.gridgain.grid.cache.datastructures; -import org.gridgain.grid.*; +import org.apache.ignite.*; import org.gridgain.grid.cache.*; import org.jetbrains.annotations.*; @@ -31,19 +31,19 @@ public interface GridCacheDataStructures { * @param initVal Initial value for sequence. If sequence already cached, {@code initVal} will be ignored. * @param create Boolean flag indicating whether data structure should be created if does not exist. * @return Sequence for the given name. - * @throws GridException If sequence could not be fetched or created. + * @throws IgniteCheckedException If sequence could not be fetched or created. */ @Nullable public GridCacheAtomicSequence atomicSequence(String name, long initVal, boolean create) - throws GridException; + throws IgniteCheckedException; /** * Remove sequence from cache. * * @param name Sequence name. * @return {@code True} if sequence has been removed, {@code false} otherwise. - * @throws GridException If remove failed. + * @throws IgniteCheckedException If remove failed. */ - public boolean removeAtomicSequence(String name) throws GridException; + public boolean removeAtomicSequence(String name) throws IgniteCheckedException; /** * Will get a atomic long from cache and create one if it has not been created yet and {@code create} flag @@ -54,18 +54,18 @@ public interface GridCacheDataStructures { * will be ignored. * @param create Boolean flag indicating whether data structure should be created if does not exist. * @return Atomic long. - * @throws GridException If atomic long could not be fetched or created. + * @throws IgniteCheckedException If atomic long could not be fetched or created. */ - @Nullable public GridCacheAtomicLong atomicLong(String name, long initVal, boolean create) throws GridException; + @Nullable public GridCacheAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteCheckedException; /** * Remove atomic long from cache. * * @param name Name of atomic long. * @return {@code True} if atomic long has been removed, {@code false} otherwise. - * @throws GridException If removing failed. + * @throws IgniteCheckedException If removing failed. */ - public boolean removeAtomicLong(String name) throws GridException; + public boolean removeAtomicLong(String name) throws IgniteCheckedException; /** * Will get a named queue from cache and create one if it has not been created yet and {@code create} flag @@ -85,10 +85,10 @@ public interface GridCacheDataStructures { * collocation. This parameter works only for {@link GridCacheMode#PARTITIONED} cache. * @param create Boolean flag indicating whether data structure should be created if does not exist. * @return Queue with given properties. - * @throws GridException If remove failed. + * @throws IgniteCheckedException If remove failed. */ @Nullable public GridCacheQueue queue(String name, int cap, boolean collocated, - boolean create) throws GridException; + boolean create) throws IgniteCheckedException; /** * Remove queue from cache. Internally one transaction will be created for all elements @@ -99,9 +99,9 @@ public interface GridCacheDataStructures { * * @param name Name queue. * @return {@code True} if queue has been removed and false if it's not cached. - * @throws GridException If remove failed. + * @throws IgniteCheckedException If remove failed. */ - public boolean removeQueue(String name) throws GridException; + public boolean removeQueue(String name) throws IgniteCheckedException; /** * Remove queue from cache. Internally multiple transactions will be created @@ -112,9 +112,9 @@ public interface GridCacheDataStructures { * @param name Name queue. * @param batchSize Batch size. * @return {@code True} if queue has been removed and false if it's not cached. - * @throws GridException If remove failed. + * @throws IgniteCheckedException If remove failed. */ - public boolean removeQueue(String name, int batchSize) throws GridException; + public boolean removeQueue(String name, int batchSize) throws IgniteCheckedException; /** * Will get a named set from cache and create one if it has not been created yet and {@code create} flag @@ -126,18 +126,18 @@ public interface GridCacheDataStructures { * for {@link GridCacheMode#PARTITIONED} cache. * @param create Flag indicating whether set should be created if does not exist. * @return Set with given properties. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - @Nullable public GridCacheSet set(String name, boolean collocated, boolean create) throws GridException; + @Nullable public GridCacheSet set(String name, boolean collocated, boolean create) throws IgniteCheckedException; /** * Removes set from cache. * * @param name Set name. * @return {@code True} if set has been removed and false if it's not cached. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public boolean removeSet(String name) throws GridException; + public boolean removeSet(String name) throws IgniteCheckedException; /** * Will get a atomic reference from cache and create one if it has not been created yet and {@code create} flag @@ -148,19 +148,19 @@ public interface GridCacheDataStructures { * {@code initVal} will be ignored. * @param create Boolean flag indicating whether data structure should be created if does not exist. * @return Atomic reference for the given name. - * @throws GridException If atomic reference could not be fetched or created. + * @throws IgniteCheckedException If atomic reference could not be fetched or created. */ @Nullable public GridCacheAtomicReference atomicReference(String name, @Nullable T initVal, boolean create) - throws GridException; + throws IgniteCheckedException; /** * Remove atomic reference from cache. * * @param name Atomic reference name. * @return {@code True} if atomic reference has been removed, {@code false} otherwise. - * @throws GridException If remove failed. + * @throws IgniteCheckedException If remove failed. */ - public boolean removeAtomicReference(String name) throws GridException; + public boolean removeAtomicReference(String name) throws IgniteCheckedException; /** * Will get a atomic stamped from cache and create one if it has not been created yet and {@code create} flag @@ -173,19 +173,19 @@ public interface GridCacheDataStructures { * {@code initStamp} will be ignored. * @param create Boolean flag indicating whether data structure should be created if does not exist. * @return Atomic stamped for the given name. - * @throws GridException If atomic stamped could not be fetched or created. + * @throws IgniteCheckedException If atomic stamped could not be fetched or created. */ @Nullable public GridCacheAtomicStamped atomicStamped(String name, @Nullable T initVal, - @Nullable S initStamp, boolean create) throws GridException; + @Nullable S initStamp, boolean create) throws IgniteCheckedException; /** * Remove atomic stamped from cache. * * @param name Atomic stamped name. * @return {@code True} if atomic stamped has been removed, {@code false} otherwise. - * @throws GridException If remove failed. + * @throws IgniteCheckedException If remove failed. */ - public boolean removeAtomicStamped(String name) throws GridException; + public boolean removeAtomicStamped(String name) throws IgniteCheckedException; /** * Gets or creates count down latch. If count down latch is not found in cache and {@code create} flag @@ -197,17 +197,17 @@ public interface GridCacheDataStructures { * when its count reaches zero. * @param create Boolean flag indicating whether data structure should be created if does not exist. * @return Count down latch for the given name. - * @throws GridException If operation failed. + * @throws IgniteCheckedException If operation failed. */ @Nullable public GridCacheCountDownLatch countDownLatch(String name, int cnt, boolean autoDel, boolean create) - throws GridException; + throws IgniteCheckedException; /** * Removes count down latch from cache. * * @param name Name of the latch. * @return Count down latch for the given name. - * @throws GridException If operation failed. + * @throws IgniteCheckedException If operation failed. */ - public boolean removeCountDownLatch(String name) throws GridException; + public boolean removeCountDownLatch(String name) throws IgniteCheckedException; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheQueue.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheQueue.java b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheQueue.java index 8cdc4df..e613f47 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheQueue.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheQueue.java @@ -9,7 +9,7 @@ package org.gridgain.grid.cache.datastructures; -import org.gridgain.grid.*; +import org.apache.ignite.*; import org.jetbrains.annotations.*; import java.util.*; @@ -23,7 +23,7 @@ import java.util.concurrent.*; * {@link Collection} interface and provides all methods from collections including * {@link Collection#addAll(Collection)}, {@link Collection#removeAll(Collection)}, and * {@link Collection#retainAll(Collection)} methods for bulk operations. Note that all - * {@link Collection} methods in the queue may throw {@link GridRuntimeException} in case + * {@link Collection} methods in the queue may throw {@link IgniteException} in case * of failure. *

* All queue operations have synchronous and asynchronous counterparts. @@ -73,97 +73,97 @@ public interface GridCacheQueue extends BlockingQueue { public String name(); /** {@inheritDoc} */ - @Override public boolean add(T item) throws GridRuntimeException; + @Override public boolean add(T item) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean offer(T item) throws GridRuntimeException; + @Override public boolean offer(T item) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean offer(T item, long timeout, TimeUnit unit) throws GridRuntimeException; + @Override public boolean offer(T item, long timeout, TimeUnit unit) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean addAll(Collection items) throws GridRuntimeException; + @Override public boolean addAll(Collection items) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean contains(Object item) throws GridRuntimeException; + @Override public boolean contains(Object item) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean containsAll(Collection items) throws GridRuntimeException; + @Override public boolean containsAll(Collection items) throws IgniteException; /** {@inheritDoc} */ - @Override public void clear() throws GridRuntimeException; + @Override public void clear() throws IgniteException; /** {@inheritDoc} */ - @Override public boolean remove(Object item) throws GridRuntimeException; + @Override public boolean remove(Object item) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean removeAll(Collection items) throws GridRuntimeException; + @Override public boolean removeAll(Collection items) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean isEmpty() throws GridRuntimeException; + @Override public boolean isEmpty() throws IgniteException; /** {@inheritDoc} */ - @Override public Iterator iterator() throws GridRuntimeException; + @Override public Iterator iterator() throws IgniteException; /** {@inheritDoc} */ - @Override public Object[] toArray() throws GridRuntimeException; + @Override public Object[] toArray() throws IgniteException; /** {@inheritDoc} */ - @Override public T[] toArray(T[] a) throws GridRuntimeException; + @Override public T[] toArray(T[] a) throws IgniteException; /** {@inheritDoc} */ - @Override public boolean retainAll(Collection items) throws GridRuntimeException; + @Override public boolean retainAll(Collection items) throws IgniteException; /** {@inheritDoc} */ - @Override public int size() throws GridRuntimeException; + @Override public int size() throws IgniteException; /** {@inheritDoc} */ - @Override @Nullable public T poll() throws GridRuntimeException; + @Override @Nullable public T poll() throws IgniteException; /** {@inheritDoc} */ - @Override @Nullable public T peek() throws GridRuntimeException; + @Override @Nullable public T peek() throws IgniteException; /** {@inheritDoc} */ - @Override public void put(T item) throws GridRuntimeException; + @Override public void put(T item) throws IgniteException; /** {@inheritDoc} */ - @Override @Nullable public T take() throws GridRuntimeException; + @Override @Nullable public T take() throws IgniteException; /** {@inheritDoc} */ - @Override @Nullable public T poll(long timeout, TimeUnit unit) throws GridRuntimeException; + @Override @Nullable public T poll(long timeout, TimeUnit unit) throws IgniteException; /** * Removes all of the elements from this queue. Method is used in massive queues with huge numbers of elements. * * @param batchSize Batch size. - * @throws GridRuntimeException if operation failed. + * @throws IgniteException if operation failed. */ - public void clear(int batchSize) throws GridRuntimeException; + public void clear(int batchSize) throws IgniteException; /** * Gets maximum number of elements of the queue. * * @return Maximum number of elements. If queue is unbounded {@code Integer.MAX_SIZE} will return. - * @throws GridException If operation failed. + * @throws IgniteCheckedException If operation failed. */ - public int capacity() throws GridException; + public int capacity() throws IgniteCheckedException; /** * Returns {@code true} if this queue is bounded. * * @return {@code true} if this queue is bounded. - * @throws GridException If operation failed. + * @throws IgniteCheckedException If operation failed. */ - public boolean bounded() throws GridException; + public boolean bounded() throws IgniteCheckedException; /** * Returns {@code true} if this queue can be kept on the one node only. * Returns {@code false} if this queue can be kept on the many nodes. * * @return {@code true} if this queue is in {@code collocated} mode {@code false} otherwise. - * @throws GridException If operation failed. + * @throws IgniteCheckedException If operation failed. */ - public boolean collocated() throws GridException; + public boolean collocated() throws IgniteCheckedException; /** * Gets status of queue. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheSet.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheSet.java b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheSet.java index bf651bb..1a44106 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheSet.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/datastructures/GridCacheSet.java @@ -9,7 +9,7 @@ package org.gridgain.grid.cache.datastructures; -import org.gridgain.grid.*; +import org.apache.ignite.*; import java.util.*; @@ -17,7 +17,7 @@ import java.util.*; * Set implementation based on on In-Memory Data Grid. *

Overview

* Cache set implements {@link Set} interface and provides all methods from collections. - * Note that all {@link Collection} methods in the set may throw {@link GridRuntimeException} in case of failure + * Note that all {@link Collection} methods in the set may throw {@link IgniteException} in case of failure * or if set was removed. *

Collocated vs Non-collocated

* Set items can be placed on one node or distributed throughout grid nodes @@ -40,9 +40,9 @@ public interface GridCacheSet extends Set { * Returns {@code false} if this set can be kept on the many nodes. * * @return {@code True} if this set is in {@code collocated} mode {@code false} otherwise. - * @throws GridException If operation failed. + * @throws IgniteCheckedException If operation failed. */ - public boolean collocated() throws GridException; + public boolean collocated() throws IgniteCheckedException; /** * Gets status of set. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/fifo/GridCacheFifoEvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/eviction/fifo/GridCacheFifoEvictionPolicy.java b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/fifo/GridCacheFifoEvictionPolicy.java index e9d0350..527604d 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/eviction/fifo/GridCacheFifoEvictionPolicy.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/fifo/GridCacheFifoEvictionPolicy.java @@ -9,6 +9,7 @@ package org.gridgain.grid.cache.eviction.fifo; +import org.apache.ignite.*; import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.eviction.*; @@ -185,7 +186,7 @@ public class GridCacheFifoEvictionPolicy implements GridCacheEvictionPolic try { return entry.peek(F.asList(GLOBAL)) == null; } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(null, e.getMessage(), e); assert false : "Should never happen: " + e; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/ggfs/GridCacheGgfsPerBlockLruEvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/eviction/ggfs/GridCacheGgfsPerBlockLruEvictionPolicy.java b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/ggfs/GridCacheGgfsPerBlockLruEvictionPolicy.java index 3d4d710..a367b6c 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/eviction/ggfs/GridCacheGgfsPerBlockLruEvictionPolicy.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/ggfs/GridCacheGgfsPerBlockLruEvictionPolicy.java @@ -9,8 +9,8 @@ package org.gridgain.grid.cache.eviction.ggfs; +import org.apache.ignite.*; import org.apache.ignite.fs.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.eviction.*; import org.gridgain.grid.kernal.processors.ggfs.*; @@ -261,9 +261,9 @@ public class GridCacheGgfsPerBlockLruEvictionPolicy implements GridCacheEviction * * @param path Path. * @return {@code True} in case non block of related file must be excluded. - * @throws GridException In case of faulty patterns. + * @throws IgniteCheckedException In case of faulty patterns. */ - public boolean exclude(IgniteFsPath path) throws GridException { + public boolean exclude(IgniteFsPath path) throws IgniteCheckedException { assert path != null; Collection excludePatterns0; @@ -280,7 +280,7 @@ public class GridCacheGgfsPerBlockLruEvictionPolicy implements GridCacheEviction excludePatterns0.add(Pattern.compile(excludePath)); } catch (PatternSyntaxException ignore) { - throw new GridException("Invalid regex pattern: " + excludePath); + throw new IgniteCheckedException("Invalid regex pattern: " + excludePath); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/random/GridCacheRandomEvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/eviction/random/GridCacheRandomEvictionPolicy.java b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/random/GridCacheRandomEvictionPolicy.java index 6149279..537a980 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/eviction/random/GridCacheRandomEvictionPolicy.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/eviction/random/GridCacheRandomEvictionPolicy.java @@ -9,6 +9,7 @@ package org.gridgain.grid.cache.eviction.random; +import org.apache.ignite.*; import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.eviction.*; @@ -96,7 +97,7 @@ public class GridCacheRandomEvictionPolicy implements GridCacheEvictionPol try { return entry.peek(F.asList(GLOBAL)) == null; } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(null, e.getMessage(), e); assert false : "Should never happen: " + e; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheContinuousQuery.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheContinuousQuery.java b/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheContinuousQuery.java index d2e44e7..a2675aa 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheContinuousQuery.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheContinuousQuery.java @@ -9,9 +9,9 @@ package org.gridgain.grid.cache.query; +import org.apache.ignite.*; import org.apache.ignite.cluster.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.jetbrains.annotations.*; @@ -301,9 +301,9 @@ public interface GridCacheContinuousQuery extends AutoCloseable { * and {@link GridCacheMode#REPLICATED REPLICATED} caches * query will be always executed locally. * - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ - public void execute() throws GridException; + public void execute() throws IgniteCheckedException; /** * Starts continuous query execution on provided set of nodes. @@ -316,9 +316,9 @@ public interface GridCacheContinuousQuery extends AutoCloseable { * query will be always executed locally. * * @param prj Grid projection. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ - public void execute(@Nullable ClusterGroup prj) throws GridException; + public void execute(@Nullable ClusterGroup prj) throws IgniteCheckedException; /** * Stops continuous query execution. @@ -328,7 +328,7 @@ public interface GridCacheContinuousQuery extends AutoCloseable { * If you need to repeat execution, use {@link GridCacheQueries#createContinuousQuery()} * method to create new query. * - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ - @Override public void close() throws GridException; + @Override public void close() throws IgniteCheckedException; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheQueryFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheQueryFuture.java index 0a8de52..9d3b045 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheQueryFuture.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/query/GridCacheQueryFuture.java @@ -9,8 +9,8 @@ package org.gridgain.grid.cache.query; +import org.apache.ignite.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.jetbrains.annotations.*; import java.util.*; @@ -25,9 +25,9 @@ public interface GridCacheQueryFuture extends IgniteFuture> { * be returned from {@link #next()} method without blocking. * * @return Number of fetched elements which are available immediately. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ - public int available() throws GridException; + public int available() throws IgniteCheckedException; /** * Returns next element from result set. @@ -36,9 +36,9 @@ public interface GridCacheQueryFuture extends IgniteFuture> { * elements available immediately. * * @return Next fetched element or {@code null} if all the elements have been fetched. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - @Nullable public T next() throws GridException; + @Nullable public T next() throws IgniteCheckedException; /** * Checks if all data is fetched by the query. @@ -52,7 +52,7 @@ public interface GridCacheQueryFuture extends IgniteFuture> { * associated with this future. * * @return {@inheritDoc} - * @throws GridException {@inheritDoc} + * @throws IgniteCheckedException {@inheritDoc} */ - @Override public boolean cancel() throws GridException; + @Override public boolean cancel() throws IgniteCheckedException; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapter.java b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapter.java index e367327..5b45feb 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapter.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapter.java @@ -94,9 +94,9 @@ public abstract class GridCacheLoadOnlyStoreAdapter implements GridCach * * @param args Arguments passes into {@link GridCache#loadCache(org.apache.ignite.lang.IgniteBiPredicate, long, Object...)} method. * @return Iterator over input records. - * @throws GridException If iterator can't be created with the given arguments. + * @throws IgniteCheckedException If iterator can't be created with the given arguments. */ - protected abstract Iterator inputIterator(@Nullable Object... args) throws GridException; + protected abstract Iterator inputIterator(@Nullable Object... args) throws IgniteCheckedException; /** * This method should transform raw data records into valid key-value pairs @@ -112,7 +112,7 @@ public abstract class GridCacheLoadOnlyStoreAdapter implements GridCach /** {@inheritDoc} */ @Override public void loadCache(IgniteBiInClosure c, @Nullable Object... args) - throws GridException { + throws IgniteCheckedException { ExecutorService exec = new ThreadPoolExecutor( threadsCnt, threadsCnt, @@ -219,40 +219,40 @@ public abstract class GridCacheLoadOnlyStoreAdapter implements GridCach /** {@inheritDoc} */ @Override public V load(@Nullable GridCacheTx tx, K key) - throws GridException { + throws IgniteCheckedException { return null; } /** {@inheritDoc} */ @Override public void loadAll(@Nullable GridCacheTx tx, - @Nullable Collection keys, IgniteBiInClosure c) throws GridException { + @Nullable Collection keys, IgniteBiInClosure c) throws IgniteCheckedException { // No-op. } /** {@inheritDoc} */ - @Override public void put(@Nullable GridCacheTx tx, K key, @Nullable V val) throws GridException { + @Override public void put(@Nullable GridCacheTx tx, K key, @Nullable V val) throws IgniteCheckedException { // No-op. } /** {@inheritDoc} */ @Override public void putAll(@Nullable GridCacheTx tx, @Nullable Map map) - throws GridException { + throws IgniteCheckedException { // No-op. } /** {@inheritDoc} */ - @Override public void remove(@Nullable GridCacheTx tx, K key) throws GridException { + @Override public void remove(@Nullable GridCacheTx tx, K key) throws IgniteCheckedException { // No-op. } /** {@inheritDoc} */ @Override public void removeAll(@Nullable GridCacheTx tx, @Nullable Collection keys) - throws GridException { + throws IgniteCheckedException { // No-op. } /** {@inheritDoc} */ - @Override public void txEnd(GridCacheTx tx, boolean commit) throws GridException { + @Override public void txEnd(GridCacheTx tx, boolean commit) throws IgniteCheckedException { // No-op. } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java index ba1be4f..73dde4c 100644 --- a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java +++ b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStore.java @@ -9,6 +9,7 @@ package org.gridgain.grid.cache.store; +import org.apache.ignite.*; import org.apache.ignite.lang.*; import org.gridgain.grid.*; import org.gridgain.grid.cache.*; @@ -64,7 +65,7 @@ import java.util.Date; * signature should be the following: *
  * public class PortableCacheStore implements GridCacheStore<Integer, GridPortableObject> {
- *     public void put(@Nullable GridCacheTx tx, Integer key, GridPortableObject val) throws GridException {
+ *     public void put(@Nullable GridCacheTx tx, Integer key, GridPortableObject val) throws IgniteCheckedException {
  *         ...
  *     }
  *
@@ -96,9 +97,9 @@ public interface GridCacheStore {
      * @param tx Cache transaction.
      * @param key Key to load.
      * @return Loaded value or {@code null} if value was not found.
-     * @throws GridException If load failed.
+     * @throws IgniteCheckedException If load failed.
      */
-    @Nullable public V load(@Nullable GridCacheTx tx, K key) throws GridException;
+    @Nullable public V load(@Nullable GridCacheTx tx, K key) throws IgniteCheckedException;
 
     /**
      * Loads all values from underlying persistent storage. Note that keys are not
@@ -117,9 +118,9 @@ public interface GridCacheStore {
      * @param clo Closure for loaded values.
      * @param args Arguments passes into
      *      {@link GridCache#loadCache(org.apache.ignite.lang.IgniteBiPredicate, long, Object...)} method.
-     * @throws GridException If loading failed.
+     * @throws IgniteCheckedException If loading failed.
      */
-    public void loadCache(IgniteBiInClosure clo, @Nullable Object... args) throws GridException;
+    public void loadCache(IgniteBiInClosure clo, @Nullable Object... args) throws IgniteCheckedException;
 
     /**
      * Loads all values for given keys and passes every value to the provided closure.
@@ -131,10 +132,10 @@ public interface GridCacheStore {
      * @param tx Cache transaction.
      * @param keys Collection of keys to load.
      * @param c Closure to call for every loaded element.
-     * @throws GridException If load failed.
+     * @throws IgniteCheckedException If load failed.
      */
     public void loadAll(@Nullable GridCacheTx tx, Collection keys, IgniteBiInClosure c)
-        throws GridException;
+        throws IgniteCheckedException;
 
     /**
      * Stores a given value in persistent storage. Note that if write-behind is configured for a
@@ -143,9 +144,9 @@ public interface GridCacheStore {
      * @param tx Cache transaction, if write-behind is not enabled, {@code null} otherwise.
      * @param key Key to put.
      * @param val Value to put.
-     * @throws GridException If put failed.
+     * @throws IgniteCheckedException If put failed.
      */
-    public void put(@Nullable GridCacheTx tx, K key, V val) throws GridException;
+    public void put(@Nullable GridCacheTx tx, K key, V val) throws IgniteCheckedException;
 
     /**
      * Stores given key value pairs in persistent storage. Note that if write-behind is configured
@@ -153,9 +154,9 @@ public interface GridCacheStore {
      *
      * @param tx Cache transaction, if write-behind is not enabled, {@code null} otherwise.
      * @param map Values to store.
-     * @throws GridException If store failed.
+     * @throws IgniteCheckedException If store failed.
      */
-    public void putAll(@Nullable GridCacheTx tx, Map map) throws GridException;
+    public void putAll(@Nullable GridCacheTx tx, Map map) throws IgniteCheckedException;
 
     /**
      * Removes the value identified by given key from persistent storage. Note that  if write-behind is
@@ -164,9 +165,9 @@ public interface GridCacheStore {
      *
      * @param tx Cache transaction, if write-behind is not enabled, {@code null} otherwise.
      * @param key Key to remove.
-     * @throws GridException If remove failed.
+     * @throws IgniteCheckedException If remove failed.
      */
-    public void remove(@Nullable GridCacheTx tx, K key) throws GridException;
+    public void remove(@Nullable GridCacheTx tx, K key) throws IgniteCheckedException;
 
     /**
      * Removes all vales identified by given keys from persistent storage. Note that if write-behind
@@ -175,9 +176,9 @@ public interface GridCacheStore {
      *
      * @param tx Cache transaction, if write-behind is not enabled, {@code null} otherwise.
      * @param keys Keys to remove.
-     * @throws GridException If remove failed.
+     * @throws IgniteCheckedException If remove failed.
      */
-    public void removeAll(@Nullable GridCacheTx tx, Collection keys) throws GridException;
+    public void removeAll(@Nullable GridCacheTx tx, Collection keys) throws IgniteCheckedException;
 
     /**
      * Tells store to commit or rollback a transaction depending on the value of the {@code 'commit'}
@@ -185,9 +186,9 @@ public interface GridCacheStore {
      *
      * @param tx Cache transaction being ended.
      * @param commit {@code True} if transaction should commit, {@code false} for rollback.
-     * @throws GridException If commit or rollback failed. Note that commit failure in some cases
+     * @throws IgniteCheckedException If commit or rollback failed. Note that commit failure in some cases
      *      may bring cache transaction into {@link GridCacheTxState#UNKNOWN} which will
      *      consequently cause all transacted entries to be invalidated.
      */
-    public void txEnd(GridCacheTx tx, boolean commit) throws GridException;
+    public void txEnd(GridCacheTx tx, boolean commit) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
index 09c968c..7aa19ab 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.cache.store;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
 
@@ -38,16 +38,16 @@ public abstract class GridCacheStoreAdapter implements GridCacheStore clo, Object... args)
-        throws GridException {
+        throws IgniteCheckedException {
         /* No-op. */
     }
 
     /** {@inheritDoc} */
     @Override public void loadAll(@Nullable GridCacheTx tx, Collection keys,
-        IgniteBiInClosure c) throws GridException {
+        IgniteBiInClosure c) throws IgniteCheckedException {
         assert keys != null;
 
         for (K key : keys) {
@@ -60,7 +60,7 @@ public abstract class GridCacheStoreAdapter implements GridCacheStore map)
-        throws GridException {
+        throws IgniteCheckedException {
         assert map != null;
 
         for (Map.Entry e : map.entrySet())
@@ -69,7 +69,7 @@ public abstract class GridCacheStoreAdapter implements GridCacheStore keys)
-        throws GridException {
+        throws IgniteCheckedException {
         assert keys != null;
 
         for (K key : keys)
@@ -83,9 +83,9 @@ public abstract class GridCacheStoreAdapter implements GridCacheStore implements GridCacheStore implements GridCacheStore clo, @Nullable Object... args) throws GridException {
+    @Override public void loadCache(IgniteBiInClosure clo, @Nullable Object... args) throws IgniteCheckedException {
         delegate.loadCache(clo, args);
     }
 
     /** {@inheritDoc} */
     @Override public void loadAll(@Nullable GridCacheTx tx, Collection keys, final IgniteBiInClosure c)
-        throws GridException {
+        throws IgniteCheckedException {
         if (keys.size() > loadAllThreshold) {
             delegate.loadAll(tx, keys, c);
 
@@ -165,27 +165,27 @@ public class GridCacheStoreBalancingWrapper implements GridCacheStore map) throws GridException {
+    @Override public void putAll(@Nullable GridCacheTx tx, Map map) throws IgniteCheckedException {
         delegate.putAll(tx, map);
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(@Nullable GridCacheTx tx, K key) throws GridException {
+    @Override public void remove(@Nullable GridCacheTx tx, K key) throws IgniteCheckedException {
         delegate.remove(tx, key);
     }
 
     /** {@inheritDoc} */
-    @Override public void removeAll(@Nullable GridCacheTx tx, Collection keys) throws GridException {
+    @Override public void removeAll(@Nullable GridCacheTx tx, Collection keys) throws IgniteCheckedException {
         delegate.removeAll(tx, keys);
     }
 
     /** {@inheritDoc} */
-    @Override public void txEnd(GridCacheTx tx, boolean commit) throws GridException {
+    @Override public void txEnd(GridCacheTx tx, boolean commit) throws IgniteCheckedException {
         delegate.txEnd(tx, commit);
     }
 
@@ -261,9 +261,9 @@ public class GridCacheStoreBalancingWrapper implements GridCacheStore extends GridCacheStoreAdapter {
     private boolean initOk;
 
     /** {@inheritDoc} */
-    @Override public void txEnd(GridCacheTx tx, boolean commit) throws GridException {
+    @Override public void txEnd(GridCacheTx tx, boolean commit) throws IgniteCheckedException {
         init();
 
         Connection conn = tx.removeMeta(ATTR_CONN);
@@ -177,7 +177,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
                     conn.rollback();
             }
             catch (SQLException e) {
-                throw new GridException("Failed to end transaction [xid=" + tx.xid() + ", commit=" + commit + ']', e);
+                throw new IgniteCheckedException("Failed to end transaction [xid=" + tx.xid() + ", commit=" + commit + ']', e);
             }
             finally {
                 closeConnection(conn);
@@ -190,7 +190,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"RedundantTypeArguments"})
-    @Override public V load(@Nullable GridCacheTx tx, K key) throws GridException {
+    @Override public V load(@Nullable GridCacheTx tx, K key) throws IgniteCheckedException {
         init();
 
         if (log.isDebugEnabled())
@@ -213,7 +213,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
                 return fromBytes(rs.getBytes(2));
         }
         catch (SQLException e) {
-            throw new GridException("Failed to load object: " + key, e);
+            throw new IgniteCheckedException("Failed to load object: " + key, e);
         }
         finally {
             end(tx, conn, stmt);
@@ -223,7 +223,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void put(@Nullable GridCacheTx tx, K key, V val) throws GridException {
+    @Override public void put(@Nullable GridCacheTx tx, K key, V val) throws IgniteCheckedException {
         init();
 
         if (log.isDebugEnabled())
@@ -253,7 +253,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
             }
         }
         catch (SQLException e) {
-            throw new GridException("Failed to put object [key=" + key + ", val=" + val + ']', e);
+            throw new IgniteCheckedException("Failed to put object [key=" + key + ", val=" + val + ']', e);
         }
         finally {
             end(tx, conn, stmt);
@@ -261,7 +261,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(@Nullable GridCacheTx tx, K key) throws GridException {
+    @Override public void remove(@Nullable GridCacheTx tx, K key) throws IgniteCheckedException {
         init();
 
         if (log.isDebugEnabled())
@@ -281,7 +281,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
             stmt.executeUpdate();
         }
         catch (SQLException e) {
-            throw new GridException("Failed to remove object: " + key, e);
+            throw new IgniteCheckedException("Failed to remove object: " + key, e);
         }
         finally {
             end(tx, conn, stmt);
@@ -361,16 +361,16 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
     /**
      * Initializes store.
      *
-     * @throws GridException If failed to initialize.
+     * @throws IgniteCheckedException If failed to initialize.
      */
-    private void init() throws GridException {
+    private void init() throws IgniteCheckedException {
         if (initLatch.getCount() > 0) {
             if (initGuard.compareAndSet(false, true)) {
                 if (log.isDebugEnabled())
                     log.debug("Initializing cache store.");
 
                 if (F.isEmpty(connUrl))
-                    throw new GridException("Failed to initialize cache store (connection URL is not provided).");
+                    throw new IgniteCheckedException("Failed to initialize cache store (connection URL is not provided).");
 
                 if (!initSchema) {
                     initLatch.countDown();
@@ -379,7 +379,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
                 }
 
                 if (F.isEmpty(createTblQry))
-                    throw new GridException("Failed to initialize cache store (create table query is not provided).");
+                    throw new IgniteCheckedException("Failed to initialize cache store (create table query is not provided).");
 
                 Connection conn = null;
 
@@ -397,7 +397,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
                     initOk = true;
                 }
                 catch (SQLException e) {
-                    throw new GridException("Failed to create database table.", e);
+                    throw new IgniteCheckedException("Failed to create database table.", e);
                 }
                 finally {
                     U.closeQuiet(stmt);
@@ -412,7 +412,7 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
         }
 
         if (!initOk)
-            throw new GridException("Cache store was not properly initialized.");
+            throw new IgniteCheckedException("Cache store was not properly initialized.");
     }
 
     /**
@@ -522,9 +522,9 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
      *
      * @param obj Object to convert to byte array.
      * @return Byte array.
-     * @throws GridException If failed to convert.
+     * @throws IgniteCheckedException If failed to convert.
      */
-    protected byte[] toBytes(Object obj) throws GridException {
+    protected byte[] toBytes(Object obj) throws IgniteCheckedException {
         return marsh.marshal(obj);
     }
 
@@ -534,9 +534,9 @@ public class GridCacheJdbcBlobStore extends GridCacheStoreAdapter {
      * @param bytes Bytes to deserialize.
      * @param  Result object type.
      * @return Deserialized object.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    protected  X fromBytes(byte[] bytes) throws GridException {
+    protected  X fromBytes(byte[] bytes) throws IgniteCheckedException {
         if (bytes == null || bytes.length == 0)
             return null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoop.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoop.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoop.java
index bbad190..75e637b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoop.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoop.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.hadoop;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -45,34 +45,34 @@ public interface GridHadoop {
      *
      * @param jobId Job ID to get status for.
      * @return Job execution status or {@code null} in case job with the given ID is not found.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Nullable public GridHadoopJobStatus status(GridHadoopJobId jobId) throws GridException;
+    @Nullable public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException;
 
     /**
      * Returns job counters.
      *
      * @param jobId Job ID to get counters for.
      * @return Job counters object.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public GridHadoopCounters counters(GridHadoopJobId jobId) throws GridException;
+    public GridHadoopCounters counters(GridHadoopJobId jobId) throws IgniteCheckedException;
 
     /**
      * Gets Hadoop finish future for particular job.
      *
      * @param jobId Job ID.
      * @return Job finish future or {@code null} in case job with the given ID is not found.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteFuture finishFuture(GridHadoopJobId jobId) throws GridException;
+    @Nullable public IgniteFuture finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException;
 
     /**
      * Kills job.
      *
      * @param jobId Job ID.
      * @return {@code True} if job was killed.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public boolean kill(GridHadoopJobId jobId) throws GridException;
+    public boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopCounterWriter.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopCounterWriter.java
index 9a00bd8..6606a6a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopCounterWriter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopCounterWriter.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.hadoop;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 /**
  * The object that writes some system counters to some storage for each running job. This operation is a part of
@@ -22,7 +22,7 @@ public interface GridHadoopCounterWriter {
      * @param jobInfo Job info.
      * @param jobId Job id.
      * @param cntrs Counters.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void write(GridHadoopJobInfo jobInfo, GridHadoopJobId jobId, GridHadoopCounters cntrs) throws GridException;
+    public void write(GridHadoopJobInfo jobInfo, GridHadoopJobId jobId, GridHadoopCounters cntrs) throws IgniteCheckedException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJob.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJob.java
index 8cda8e8..34326d3 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJob.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJob.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.hadoop;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 import java.util.*;
 
@@ -36,16 +36,16 @@ public interface GridHadoopJob {
      *
      * @return Input splits.
      */
-    public Collection input() throws GridException;
+    public Collection input() throws IgniteCheckedException;
 
     /**
      * Returns context for task execution.
      *
      * @param info Task info.
      * @return Task Context.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public GridHadoopTaskContext getTaskContext(GridHadoopTaskInfo info) throws GridException;
+    public GridHadoopTaskContext getTaskContext(GridHadoopTaskInfo info) throws IgniteCheckedException;
 
     /**
      * Does all the needed initialization for the job. Will be called on each node where tasks for this job must
@@ -56,9 +56,9 @@ public interface GridHadoopJob {
      *
      * @param external If {@code true} then this job instance resides in external process.
      * @param locNodeId Local node ID.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void initialize(boolean external, UUID locNodeId) throws GridException;
+    public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException;
 
     /**
      * Release all the resources.
@@ -67,25 +67,25 @@ public interface GridHadoopJob {
      * {@code false} and on instance in external process with parameter {@code true}.
      *
      * @param external If {@code true} then this job instance resides in external process.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void dispose(boolean external) throws GridException;
+    public void dispose(boolean external) throws IgniteCheckedException;
 
     /**
      * Prepare local environment for the task.
      *
      * @param info Task info.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void prepareTaskEnvironment(GridHadoopTaskInfo info) throws GridException;
+    public void prepareTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException;
 
     /**
      * Cleans up local environment of the task.
      *
      * @param info Task info.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void cleanupTaskEnvironment(GridHadoopTaskInfo info) throws GridException;
+    public void cleanupTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException;
 
     /**
      * Cleans up the job staging directory.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJobInfo.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJobInfo.java
index 948093a..5552267 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJobInfo.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopJobInfo.java
@@ -51,9 +51,9 @@ public interface GridHadoopJobInfo extends Serializable {
      * @param jobId Job ID.
      * @param log Logger.
      * @return Job.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    GridHadoopJob createJob(GridHadoopJobId jobId, IgniteLogger log) throws GridException;
+    GridHadoopJob createJob(GridHadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException;
 
     /**
      * @return Number of reducers configured for job.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopMapReducePlanner.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopMapReducePlanner.java
index 247e5b2..bdb7bcb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopMapReducePlanner.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopMapReducePlanner.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.hadoop;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
-import org.gridgain.grid.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -28,5 +28,5 @@ public interface GridHadoopMapReducePlanner {
      * @return Map reduce plan.
      */
     public GridHadoopMapReducePlan preparePlan(GridHadoopJob job, Collection top,
-        @Nullable GridHadoopMapReducePlan oldPlan) throws GridException;
+        @Nullable GridHadoopMapReducePlan oldPlan) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopSerialization.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopSerialization.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopSerialization.java
index 3d65b66..3aa2127 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopSerialization.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopSerialization.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.hadoop;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -23,9 +23,9 @@ public interface GridHadoopSerialization extends AutoCloseable {
      *
      * @param out Output.
      * @param obj Object to serialize.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void write(DataOutput out, Object obj) throws GridException;
+    public void write(DataOutput out, Object obj) throws IgniteCheckedException;
 
     /**
      * Reads object from the given input optionally reusing given instance.
@@ -33,14 +33,14 @@ public interface GridHadoopSerialization extends AutoCloseable {
      * @param in Input.
      * @param obj Object.
      * @return New object or reused instance.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public Object read(DataInput in, @Nullable Object obj) throws GridException;
+    public Object read(DataInput in, @Nullable Object obj) throws IgniteCheckedException;
 
     /**
      * Finalise the internal objects.
      * 
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Override public void close() throws GridException;
+    @Override public void close() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTask.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTask.java
index 86890d7..f4d4e1c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTask.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.hadoop;
 
+import org.apache.ignite.*;
 import org.gridgain.grid.*;
 
 import java.io.*;
@@ -53,9 +54,9 @@ public abstract class GridHadoopTask {
      *
      * @param taskCtx Context.
      * @throws GridInterruptedException If interrupted.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract void run(GridHadoopTaskContext taskCtx) throws GridException;
+    public abstract void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException;
 
     /**
      * Interrupts task execution.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskContext.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskContext.java
index 59b3547..a92a332 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskContext.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskContext.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.hadoop;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 import java.util.*;
 
@@ -119,25 +119,25 @@ public abstract class GridHadoopTaskContext {
      * Gets partitioner.
      *
      * @return Partitioner.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract GridHadoopPartitioner partitioner() throws GridException;
+    public abstract GridHadoopPartitioner partitioner() throws IgniteCheckedException;
 
     /**
      * Gets serializer for values.
      *
      * @return Serializer for keys.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract GridHadoopSerialization keySerialization() throws GridException;
+    public abstract GridHadoopSerialization keySerialization() throws IgniteCheckedException;
 
     /**
      * Gets serializer for values.
      *
      * @return Serializer for values.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract GridHadoopSerialization valueSerialization() throws GridException;
+    public abstract GridHadoopSerialization valueSerialization() throws IgniteCheckedException;
 
     /**
      * Gets sorting comparator.
@@ -156,9 +156,9 @@ public abstract class GridHadoopTaskContext {
     /**
      * Execute current task.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract void run() throws GridException;
+    public abstract void run() throws IgniteCheckedException;
 
     /**
      * Cancel current task execution.
@@ -168,14 +168,14 @@ public abstract class GridHadoopTaskContext {
     /**
      * Prepare local environment for the task.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract void prepareTaskEnvironment() throws GridException;
+    public abstract void prepareTaskEnvironment() throws IgniteCheckedException;
 
     /**
      *  Cleans up local environment of the task.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract void cleanupTaskEnvironment() throws GridException;
+    public abstract void cleanupTaskEnvironment() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskInput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskInput.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskInput.java
index 7fdea1b..27f5226 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskInput.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskInput.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.hadoop;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 import java.util.*;
 
@@ -41,7 +41,7 @@ public interface GridHadoopTaskInput extends AutoCloseable {
     /**
      * Closes input.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Override public void close() throws GridException;
+    @Override public void close() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskOutput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskOutput.java b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskOutput.java
index 19d5252..b26f8ad 100644
--- a/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskOutput.java
+++ b/modules/core/src/main/java/org/gridgain/grid/hadoop/GridHadoopTaskOutput.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.hadoop;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 /**
  * Task output.
@@ -21,12 +21,12 @@ public interface GridHadoopTaskOutput extends AutoCloseable {
      * @param key Key.
      * @param val Value.
      */
-    public void write(Object key, Object val) throws GridException;
+    public void write(Object key, Object val) throws IgniteCheckedException;
 
     /**
      * Closes output.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Override public void close() throws GridException;
+    @Override public void close() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/ClusterGroupAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/ClusterGroupAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/ClusterGroupAdapter.java
index 42b1588..47d8d96 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/ClusterGroupAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/ClusterGroupAdapter.java
@@ -247,7 +247,7 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public final ClusterMetrics metrics() throws GridException {
+    @Override public final ClusterMetrics metrics() throws IgniteCheckedException {
         guard();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponent.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponent.java
index 80cb6db..3e0822a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponent.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponent.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -23,26 +23,26 @@ public interface GridComponent {
     /**
      * Starts grid component.
      *
-     * @throws GridException Throws in case of any errors.
+     * @throws IgniteCheckedException Throws in case of any errors.
      */
-    public void start() throws GridException;
+    public void start() throws IgniteCheckedException;
 
     /**
      * Stops grid component.
      *
      * @param cancel If {@code true}, then all ongoing tasks or jobs for relevant
      *      components need to be cancelled.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void stop(boolean cancel) throws GridException;
+    public void stop(boolean cancel) throws IgniteCheckedException;
 
     /**
      * Callback that notifies that kernal has successfully started,
      * including all managers and processors.
      *
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void onKernalStart() throws GridException;
+    public void onKernalStart() throws IgniteCheckedException;
 
     /**
      * Callback to notify that kernal is about to stop.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponentType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponentType.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponentType.java
index 032c489..a617c99 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponentType.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridComponentType.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.jetbrains.annotations.*;
 
 import java.lang.reflect.*;
@@ -132,9 +132,9 @@ public enum GridComponentType {
      * @param ctx Kernal context.
      * @param noOp No-op flag.
      * @return Created component.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public  T create(GridKernalContext ctx, boolean noOp) throws GridException {
+    public  T create(GridKernalContext ctx, boolean noOp) throws IgniteCheckedException {
         return create0(ctx, noOp ? noOpClsName : clsName);
     }
 
@@ -144,10 +144,10 @@ public enum GridComponentType {
      * @param ctx Kernal context.
      * @param mandatory If the component is mandatory.
      * @return Created component.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public  T createIfInClassPath(GridKernalContext ctx, boolean mandatory)
-        throws GridException {
+        throws IgniteCheckedException {
         String cls = clsName;
 
         try {
@@ -168,9 +168,9 @@ public enum GridComponentType {
      *
      * @param noOp No-op flag.
      * @return Created component.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public  T create(boolean noOp) throws GridException {
+    public  T create(boolean noOp) throws IgniteCheckedException {
         return create0(null, noOp ? noOpClsName : clsName);
     }
 
@@ -179,9 +179,9 @@ public enum GridComponentType {
      *
      * @param ctx Kernal context.
      * @return Created component or no-op implementation.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public  T createOptional(GridKernalContext ctx) throws GridException {
+    public  T createOptional(GridKernalContext ctx) throws IgniteCheckedException {
         return createOptional0(ctx);
     }
 
@@ -189,9 +189,9 @@ public enum GridComponentType {
      * First tries to find main component class, if it is not found creates no-op implementation.
      *
      * @return Created component or no-op implementation.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public  T createOptional() throws GridException {
+    public  T createOptional() throws IgniteCheckedException {
         return createOptional0(null);
     }
 
@@ -200,10 +200,10 @@ public enum GridComponentType {
      *
      * @param ctx Kernal context.
      * @return Created component or no-op implementation.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    private  T createOptional0(@Nullable GridKernalContext ctx) throws GridException {
+    private  T createOptional0(@Nullable GridKernalContext ctx) throws IgniteCheckedException {
         Class cls;
 
         try {
@@ -214,7 +214,7 @@ public enum GridComponentType {
                 cls = Class.forName(noOpClsName);
             }
             catch (ClassNotFoundException e) {
-                throw new GridException("Failed to find both real component class and no-op class.", e);
+                throw new IgniteCheckedException("Failed to find both real component class and no-op class.", e);
             }
         }
 
@@ -241,10 +241,10 @@ public enum GridComponentType {
      * @param ctx Kernal context.
      * @param clsName Component class name.
      * @return Component instance.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    private  T create0(@Nullable GridKernalContext ctx, String clsName) throws GridException {
+    private  T create0(@Nullable GridKernalContext ctx, String clsName) throws IgniteCheckedException {
         try {
             Class cls = Class.forName(clsName);
 
@@ -268,8 +268,8 @@ public enum GridComponentType {
      * @param err Creation error.
      * @return Component creation exception.
      */
-    private GridException componentException(Exception err) {
-        return new GridException("Failed to create GridGain component (consider adding " + module +
+    private IgniteCheckedException componentException(Exception err) {
+        return new IgniteCheckedException("Failed to create GridGain component (consider adding " + module +
             " module to classpath) [component=" + this + ", cls=" + clsName + ']', err);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridEventConsumeHandler.java
index 9f65ead..a48f6e1 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridEventConsumeHandler.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
@@ -99,7 +100,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
 
     /** {@inheritDoc} */
     @Override public boolean register(final UUID nodeId, final UUID routineId, final GridKernalContext ctx)
-        throws GridException {
+        throws IgniteCheckedException {
         assert nodeId != null;
         assert routineId != null;
         assert ctx != null;
@@ -143,7 +144,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
 
                                 ctx.continuous().addNotification(nodeId, routineId, wrapper, null);
                             }
-                            catch (GridException e) {
+                            catch (IgniteCheckedException e) {
                                 U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, e);
                             }
                         }
@@ -217,7 +218,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
                 try {
                     wrapper.p2pUnmarshal(ctx.config().getMarshaller(), ldr);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(ctx.log(getClass()), "Failed to unmarshal event.", e);
                 }
             }
@@ -231,7 +232,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public void p2pMarshal(GridKernalContext ctx) throws GridException {
+    @Override public void p2pMarshal(GridKernalContext ctx) throws IgniteCheckedException {
         assert ctx != null;
         assert ctx.config().isPeerClassLoadingEnabled();
 
@@ -252,7 +253,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public void p2pUnmarshal(UUID nodeId, GridKernalContext ctx) throws GridException {
+    @Override public void p2pUnmarshal(UUID nodeId, GridKernalContext ctx) throws IgniteCheckedException {
         assert nodeId != null;
         assert ctx != null;
         assert ctx.config().isPeerClassLoadingEnabled();
@@ -342,9 +343,9 @@ class GridEventConsumeHandler implements GridContinuousHandler {
 
         /**
          * @param marsh Marshaller.
-         * @throws GridException In case of error.
+         * @throws IgniteCheckedException In case of error.
          */
-        void p2pMarshal(IgniteMarshaller marsh) throws GridException {
+        void p2pMarshal(IgniteMarshaller marsh) throws IgniteCheckedException {
             assert marsh != null;
 
             bytes = marsh.marshal(evt);
@@ -353,9 +354,9 @@ class GridEventConsumeHandler implements GridContinuousHandler {
         /**
          * @param marsh Marshaller.
          * @param ldr Class loader.
-         * @throws GridException In case of error.
+         * @throws IgniteCheckedException In case of error.
          */
-        void p2pUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws GridException {
+        void p2pUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws IgniteCheckedException {
             assert marsh != null;
 
             assert evt == null;