http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java index 176fdd0..76b42cf 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -12,14 +12,13 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near; 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.gridgain.grid.kernal.processors.cache.*; import org.gridgain.grid.kernal.processors.cache.distributed.*; -import org.gridgain.grid.util.typedef.*; -import org.gridgain.grid.util.typedef.internal.*; import org.gridgain.grid.util.future.*; import org.gridgain.grid.util.tostring.*; +import org.gridgain.grid.util.typedef.*; +import org.gridgain.grid.util.typedef.internal.*; import org.jetbrains.annotations.*; import java.io.*; @@ -173,7 +172,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu try { tx.close(); } - catch (IgniteCheckedException ex) { + catch (IgniteException ex) { U.error(log, "Failed to invalidate transaction: " + tx, ex); } }
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java index 5ff258f..1c6ca83 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java @@ -792,7 +792,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> { if (prepFut != null) prepFut.get(); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { if (log.isDebugEnabled()) log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); } @@ -814,7 +814,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> { // Check for errors in prepare future. f.get(); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { if (log.isDebugEnabled()) log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); } @@ -955,7 +955,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> { fut.onError(e); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to prepare transaction: " + this, e); fut.onError(e); @@ -975,7 +975,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> { fut.onError(e); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to prepare transaction: " + this, e); fut.onError(e); @@ -1006,7 +1006,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> { if (prep != null) prep.get(); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { if (log.isDebugEnabled()) log.debug("Failed to prepare transaction during rollback (will ignore) [tx=" + this + ", msg=" + e.getMessage() + ']'); @@ -1020,7 +1020,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> { try { f.get(); // Check for errors of a parent future. } - catch (IgniteCheckedException e) { + catch (IgniteException e) { log.debug("Failed to prepare transaction during rollback (will ignore) [tx=" + this + ", msg=" + e.getMessage() + ']'); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java index b3d658b..3356665 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java @@ -72,7 +72,7 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu } /** {@inheritDoc} */ - @Override protected void cancelQuery() throws IgniteCheckedException { + @Override protected void cancelQuery() throws IgniteException { final GridCacheQueryManager<K, V> qryMgr = cctx.queries(); assert qryMgr != null; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java index df55b83..0a6cb46 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java @@ -58,7 +58,7 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap } /** {@inheritDoc} */ - @Override protected void cancelQuery() throws IgniteCheckedException { + @Override protected void cancelQuery() throws IgniteException { if (fut != null) fut.cancel(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java index 96e05f7..2476302 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java @@ -33,12 +33,12 @@ public class GridCacheQueryErrorFuture<T> extends GridFinishedFuture<Collection< } /** {@inheritDoc} */ - @Override public int available() throws IgniteCheckedException { + @Override public int available() throws IgniteException { return 0; } /** {@inheritDoc} */ - @Nullable @Override public T next() throws IgniteCheckedException { + @Nullable @Override public T next() throws IgniteException { return null; } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java index 797e31a..cb70fb6 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java @@ -455,7 +455,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda } /** {@inheritDoc} */ - @Override public Collection<R> get() throws IgniteCheckedException { + @Override public Collection<R> get() throws IgniteException { if (!isDone()) loadAllPages(); @@ -463,7 +463,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda } /** {@inheritDoc} */ - @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException { + @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteException { if (!isDone()) loadAllPages(); @@ -498,7 +498,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda } /** {@inheritDoc} */ - @Override public boolean cancel() throws IgniteCheckedException { + @Override public boolean cancel() throws IgniteException { if (onCancelled()) { cancelQuery(); @@ -509,9 +509,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda } /** - * @throws IgniteCheckedException In case of error. + * @throws IgniteException In case of error. */ - protected abstract void cancelQuery() throws IgniteCheckedException; + protected abstract void cancelQuery() throws IgniteException; /** {@inheritDoc} */ @Override public IgniteUuid timeoutId() { @@ -530,7 +530,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda onDone(new IgniteFutureTimeoutException("Query timed out.")); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { onDone(e); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java index 25c0668..b6745e4 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java @@ -1397,7 +1397,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte try { e.getValue().get().closeIfNotShared(recipient(sndId, e.getKey())); } - catch (IgniteCheckedException ex) { + catch (IgniteException ex) { U.error(log, "Failed to close query iterator.", ex); } } @@ -1482,7 +1482,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte try { fut.get().closeIfNotShared(recipient(sndId, reqId)); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to close iterator.", e); } } @@ -1523,7 +1523,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte try { e.getValue().get().closeIfNotShared(recipient(sndId, e.getKey())); } - catch (IgniteCheckedException ex) { + catch (IgniteException ex) { U.error(log, "Failed to close fields query iterator.", ex); } } @@ -1610,7 +1610,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte try { fut.get().closeIfNotShared(recipient(sndId, reqId)); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to close iterator.", e); } } @@ -2255,42 +2255,32 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte /** * @return Value. - * @throws IgniteCheckedException If failed. + * @throws IgniteException If failed. */ - protected abstract V unmarshalValue() throws IgniteCheckedException; + protected abstract V unmarshalValue() throws IgniteException; /** * @return Key. */ K key() { - try { - if (key != null) - return key; + if (key != null) + return key; - key = cctx.marshaller().unmarshal(keyBytes(), cctx.deploy().globalLoader()); + key = cctx.marshaller().unmarshal(keyBytes(), cctx.deploy().globalLoader()); - return key; - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } + return key; } /** * @return Value. */ V value() { - try { - if (val != null) - return val; + if (val != null) + return val; - val = unmarshalValue(); + val = unmarshalValue(); - return val; - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } + return val; } /** @@ -2330,7 +2320,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte /** {@inheritDoc} */ @SuppressWarnings("IfMayBeConditional") - @Override protected V unmarshalValue() throws IgniteCheckedException { + @Override protected V unmarshalValue() throws IgniteException { byte[] bytes = e.getValue(); byte[] val = GridCacheSwapEntryImpl.getValueIfByteArray(bytes); @@ -2393,7 +2383,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte } /** {@inheritDoc} */ - @Override protected V unmarshalValue() throws IgniteCheckedException { + @Override protected V unmarshalValue() throws IgniteException { long ptr = GridCacheOffheapSwapEntry.valueAddress(valPtr.get1(), valPtr.get2()); V val = (V)cctx.portable().unmarshal(ptr, false); @@ -2902,9 +2892,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte * Close if this result does not have any other recipients. * * @param recipient ID of the recipient. - * @throws IgniteCheckedException If failed. + * @throws IgniteException If failed. */ - public void closeIfNotShared(Object recipient) throws IgniteCheckedException { + public void closeIfNotShared(Object recipient) throws IgniteException { assert isDone(); synchronized (recipients) { @@ -3019,7 +3009,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte } /** {@inheritDoc} */ - @Override public void close() throws IgniteCheckedException { + @Override public void close() throws IgniteException { closeIfNotShared(recipient); } @@ -3040,14 +3030,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte private boolean fillNext() { assert next.isEmpty(); - IgniteSpiCloseableIterator<R> it; - - try { - it = get(); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } + IgniteSpiCloseableIterator<R> it = get(); synchronized (recipients) { for (int i = 0; i < NEXT_SIZE; i++) { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java index 9b88858..4a8408b 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java @@ -285,7 +285,7 @@ public class GridCacheContinuousQueryAdapter<K, V> implements GridCacheContinuou } /** {@inheritDoc} */ - @Override public void close() throws IgniteCheckedException { + @Override public void close() throws IgniteException { closeLock.lock(); try { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java index 0f64ab8..49a8f19 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java @@ -39,7 +39,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, - @Nullable String cacheName) throws IgniteCheckedException { + @Nullable String cacheName) throws IgniteException { Map<JdbcDriverMetadataJob, ClusterNode> map = new HashMap<>(); for (ClusterNode n : subgrid) @@ -53,7 +53,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b } /** {@inheritDoc} */ - @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteCheckedException { + @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteException { return F.first(results).getData(); } @@ -92,7 +92,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b } /** {@inheritDoc} */ - @Override public Object execute() throws IgniteCheckedException { + @Override public Object execute() throws IgniteException { byte status; byte[] data; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java index f70716f..70866e6 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java @@ -53,7 +53,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> { private static final ScheduledExecutorService SCHEDULER = Executors.newScheduledThreadPool(1); /** {@inheritDoc} */ - @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, byte[] arg) throws IgniteCheckedException { + @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, byte[] arg) throws IgniteException { assert arg != null; Map<String, Object> args = MARSHALLER.unmarshal(arg, null); @@ -73,7 +73,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> { if (n.id().equals(nodeId)) return F.asMap(new JdbcDriverJob(args, first), n); - throw new IgniteCheckedException("Node doesn't exist or left the grid: " + nodeId); + throw new IgniteException("Node doesn't exist or left the grid: " + nodeId); } else { String cache = (String)args.get("cache"); @@ -82,12 +82,12 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> { if (U.hasCache(n, cache)) return F.asMap(new JdbcDriverJob(args, first), n); - throw new IgniteCheckedException("Can't find node with cache: " + cache); + throw new IgniteException("Can't find node with cache: " + cache); } } /** {@inheritDoc} */ - @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteCheckedException { + @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteException { byte status; byte[] bytes; @@ -114,7 +114,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> { } /** {@inheritDoc} */ - @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteCheckedException { + @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException { return WAIT; } @@ -161,7 +161,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> { } /** {@inheritDoc} */ - @Override public Object execute() throws IgniteCheckedException { + @Override public Object execute() throws IgniteException { String cacheName = argument("cache"); String sql = argument("sql"); Long timeout = argument("timeout"); @@ -206,7 +206,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> { try { fut.get(); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { if (e.hasCause(SQLException.class)) throw new GridInternalException(e.getCause(SQLException.class).getMessage(), e); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java index 0f3b4d9..6d436ab 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java @@ -32,7 +32,7 @@ public class GridCacheQueryJdbcValidationTask extends ComputeTaskSplitAdapter<St /** {@inheritDoc} */ @Override protected Collection<? extends ComputeJob> split(int gridSize, - @Nullable final String cacheName) throws IgniteCheckedException { + @Nullable final String cacheName) throws IgniteException { // Register big data usage. GridLicenseUseRegistry.onUsage(DATA_GRID, getClass()); @@ -51,7 +51,7 @@ public class GridCacheQueryJdbcValidationTask extends ComputeTaskSplitAdapter<St } /** {@inheritDoc} */ - @Override public Boolean reduce(List<ComputeJobResult> results) throws IgniteCheckedException { + @Override public Boolean reduce(List<ComputeJobResult> results) throws IgniteException { return F.first(results).getData(); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java index f19327e..c4c2345 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java @@ -195,11 +195,11 @@ public class GridClosureProcessor extends GridProcessorAdapter { * @param jobs Closures to map. * @param nodes Grid nodes. * @param lb Load balancer. - * @throws IgniteCheckedException Thrown in case of any errors. + * @throws IgniteException Thrown in case of any errors. * @return Mapping. */ private Map<ComputeJob, ClusterNode> absMap(GridClosureCallMode mode, Collection<? extends Runnable> jobs, - Collection<ClusterNode> nodes, ComputeLoadBalancer lb) throws IgniteCheckedException { + Collection<ClusterNode> nodes, ComputeLoadBalancer lb) throws IgniteException { assert mode != null; assert jobs != null; assert nodes != null; @@ -243,12 +243,12 @@ public class GridClosureProcessor extends GridProcessorAdapter { * @param jobs Closures to map. * @param nodes Grid nodes. * @param lb Load balancer. - * @throws IgniteCheckedException Thrown in case of any errors. + * @throws IgniteException Thrown in case of any errors. * @return Mapping. */ private <R> Map<ComputeJob, ClusterNode> outMap(GridClosureCallMode mode, Collection<? extends Callable<R>> jobs, Collection<ClusterNode> nodes, ComputeLoadBalancer lb) - throws IgniteCheckedException { + throws IgniteException { assert mode != null; assert jobs != null; assert nodes != null; @@ -969,7 +969,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { return job.apply(arg); } - @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException { + @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException { ((ComputeJobMasterLeaveAware)job).onMasterNodeLeft(ses); } }; @@ -1004,7 +1004,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { } } - @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException { + @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException { ((ComputeJobMasterLeaveAware)c).onMasterNodeLeft(ses); } }; @@ -1054,7 +1054,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { } } - @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException { + @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException { ((ComputeJobMasterLeaveAware)c).onMasterNodeLeft(ses); } }; @@ -1099,7 +1099,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { return null; } - @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException { + @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException { ((ComputeJobMasterLeaveAware)r).onMasterNodeLeft(ses); } }; @@ -1143,7 +1143,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { return null; } - @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException { + @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException { ((ComputeJobMasterLeaveAware)r).onMasterNodeLeft(ses); } }; @@ -1188,9 +1188,9 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** * @param job Job. * @param node Node. - * @throws IgniteCheckedException In case of error. + * @throws IgniteException In case of error. */ - public void map(ComputeJob job, ClusterNode node) throws IgniteCheckedException { + public void map(ComputeJob job, ClusterNode node) throws IgniteException { assert job != null; assert node != null; @@ -1223,7 +1223,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Nullable @Override public Void reduce(List<ComputeJobResult> results) throws IgniteCheckedException { + @Nullable @Override public Void reduce(List<ComputeJobResult> results) throws IgniteException { return null; } } @@ -1258,7 +1258,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { return absMap(t.get1(), t.get2(), subgrid, lb); } } @@ -1290,7 +1290,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { return absMap(t.get1(), F.asList(t.get2()), subgrid, lb); } } @@ -1331,13 +1331,13 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { return outMap(t.get1(), t.get2(), subgrid, lb); } /** {@inheritDoc} */ @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) - throws IgniteCheckedException { + throws IgniteException { ComputeJobResultPolicy resPlc = super.result(res, rcvd); if (res.getException() == null && resPlc != FAILOVER && !t.get3().collect((R1)res.getData())) @@ -1386,7 +1386,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { ComputeJob job = job(this.job, cacheName, affKey); return Collections.singletonMap(job, lb.getBalancedNode(job, null)); @@ -1427,20 +1427,20 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { ComputeJob job = job(this.job, cacheName, affKey); return Collections.singletonMap(job, lb.getBalancedNode(job, null)); } /** {@inheritDoc} */ - @Override public R reduce(List<ComputeJobResult> res) throws IgniteCheckedException { + @Override public R reduce(List<ComputeJobResult> res) throws IgniteException { for (ComputeJobResult r : res) { if (r.getException() == null) return r.getData(); } - throw new IgniteCheckedException("Failed to find successful job result: " + res); + throw new IgniteException("Failed to find successful job result: " + res); } } @@ -1478,7 +1478,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { return outMap(mode, jobs, subgrid, lb); } @@ -1515,17 +1515,17 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { return outMap(t.get1(), F.asList(t.get2()), subgrid, lb); } /** {@inheritDoc} */ - @Override public R reduce(List<ComputeJobResult> res) throws IgniteCheckedException { + @Override public R reduce(List<ComputeJobResult> res) throws IgniteException { for (ComputeJobResult r : res) if (r.getException() == null) return r.getData(); - throw new IgniteCheckedException("Failed to find successful job result: " + res); + throw new IgniteException("Failed to find successful job result: " + res); } } @@ -1558,19 +1558,19 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { ComputeJob job = job(this.job, this.arg); return Collections.singletonMap(job, lb.getBalancedNode(job, null)); } /** {@inheritDoc} */ - @Override public R reduce(List<ComputeJobResult> res) throws IgniteCheckedException { + @Override public R reduce(List<ComputeJobResult> res) throws IgniteException { for (ComputeJobResult r : res) if (r.getException() == null) return r.getData(); - throw new IgniteCheckedException("Failed to find successful job result: " + res); + throw new IgniteException("Failed to find successful job result: " + res); } } @@ -1603,7 +1603,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { Map<ComputeJob, ClusterNode> map = new HashMap<>(args.size(), 1); JobMapper mapper = new JobMapper(map); @@ -1618,7 +1618,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public Collection<R> reduce(List<ComputeJobResult> res) throws IgniteCheckedException { + @Override public Collection<R> reduce(List<ComputeJobResult> res) throws IgniteException { return F.jobResults(res); } } @@ -1657,7 +1657,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { Map<ComputeJob, ClusterNode> map = new HashMap<>(args.size(), 1); JobMapper mapper = new JobMapper(map); @@ -1673,7 +1673,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) - throws IgniteCheckedException { + throws IgniteException { ComputeJobResultPolicy resPlc = super.result(res, rcvd); if (res.getException() == null && resPlc != FAILOVER && !rdc.collect((R1) res.getData())) @@ -1683,7 +1683,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public R2 reduce(List<ComputeJobResult> res) throws IgniteCheckedException { + @Override public R2 reduce(List<ComputeJobResult> res) throws IgniteException { return rdc.reduce(); } } @@ -1714,7 +1714,7 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) - throws IgniteCheckedException { + throws IgniteException { if (F.isEmpty(subgrid)) return Collections.emptyMap(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java index 14d89f3..f4e2a72 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java @@ -222,7 +222,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter { try { msg.data(marsh.unmarshal(msg.dataBytes(), null)); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to process message (ignoring): " + msg, e); return; @@ -397,7 +397,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter { try { msg.data(marsh.unmarshal(msg.dataBytes(), null)); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to process message (ignoring): " + msg, e); return; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java index 792895d..3a8449c 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java @@ -49,7 +49,7 @@ class GridDataLoaderFuture extends GridFutureAdapter<Object> { } /** {@inheritDoc} */ - @Override public boolean cancel() throws IgniteCheckedException { + @Override public boolean cancel() throws IgniteException { checkValid(); if (onCancelled()) { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java index 10f0487..7d10334 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java @@ -123,7 +123,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter { catch (GridInterruptedException e) { U.warn(log, "Interrupted while waiting for completion of the data loader: " + ldr, e); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to close data loader: " + ldr, e); } } @@ -193,7 +193,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter { try { topic = marsh.unmarshal(req.responseTopicBytes(), null); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to unmarshal topic from request: " + req, e); return; @@ -235,7 +235,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter { col = marsh.unmarshal(req.collectionBytes(), clsLdr); updater = marsh.unmarshal(req.updaterBytes(), clsLdr); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to unmarshal message [nodeId=" + nodeId + ", req=" + req + ']', e); sendResponse(nodeId, topic, req.requestId(), e, false); @@ -278,7 +278,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter { try { errBytes = err != null ? marsh.marshal(err) : null; } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to marshal message.", e); return; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java index e6ce07e..3a3833b 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java @@ -270,14 +270,14 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay } /** {@inheritDoc} */ - @Override public void isolated(boolean isolated) throws IgniteCheckedException { + @Override public void isolated(boolean isolated) throws IgniteException { if (isolated()) return; ClusterNode node = F.first(ctx.grid().forCache(cacheName).nodes()); if (node == null) - throw new IgniteCheckedException("Failed to get node for cache: " + cacheName); + throw new IgniteException("Failed to get node for cache: " + cacheName); GridCacheAttributes a = U.cacheAttributes(node, cacheName); @@ -374,21 +374,21 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay } /** {@inheritDoc} */ - @Override public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, IllegalStateException { + @Override public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteException, IllegalStateException { A.notNull(entry, "entry"); return addData(F.asList(entry)); } /** {@inheritDoc} */ - @Override public IgniteFuture<?> addData(K key, V val) throws IgniteCheckedException, IllegalStateException { + @Override public IgniteFuture<?> addData(K key, V val) throws IgniteException, IllegalStateException { A.notNull(key, "key"); return addData(new Entry0<>(key, val)); } /** {@inheritDoc} */ - @Override public IgniteFuture<?> removeData(K key) throws IgniteCheckedException, IllegalStateException { + @Override public IgniteFuture<?> removeData(K key) throws IgniteException, IllegalStateException { return addData(key, null); } @@ -407,7 +407,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay assert entries != null; if (remaps >= MAX_REMAP_CNT) { - resFut.onDone(new IgniteCheckedException("Failed to finish operation (too many remaps): " + remaps)); + resFut.onDone(new IgniteException("Failed to finish operation (too many remaps): " + remaps)); return; } @@ -479,12 +479,12 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay if (activeKeys.isEmpty()) resFut.onDone(); } - catch (IgniteCheckedException e1) { + catch (IgniteException e1) { if (log.isDebugEnabled()) log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']'); if (cancelled) { - resFut.onDone(new IgniteCheckedException("Data loader has been cancelled: " + + resFut.onDone(new IgniteException("Data loader has been cancelled: " + IgniteDataLoaderImpl.this, e1)); } else @@ -518,9 +518,9 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay /** * Performs flush. * - * @throws IgniteCheckedException If failed. + * @throws IgniteException If failed. */ - private void doFlush() throws IgniteCheckedException { + private void doFlush() throws IgniteException { lastFlushTime = U.currentTimeMillis(); List<IgniteFuture> activeFuts0 = null; @@ -567,7 +567,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay try { fut.get(); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { if (log.isDebugEnabled()) log.debug("Failed to flush buffer: " + e); @@ -605,7 +605,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay /** {@inheritDoc} */ @SuppressWarnings("ForLoopReplaceableByForEach") - @Override public void flush() throws IgniteCheckedException { + @Override public void flush() throws IgniteException { enterBusy(); try { @@ -640,9 +640,9 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay /** * @param cancel {@code True} to close with cancellation. - * @throws IgniteCheckedException If failed. + * @throws IgniteException If failed. */ - @Override public void close(boolean cancel) throws IgniteCheckedException { + @Override public void close(boolean cancel) throws IgniteException { if (!closed.compareAndSet(false, true)) return; @@ -651,7 +651,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay if (log.isDebugEnabled()) log.debug("Closing data loader [ldr=" + this + ", cancel=" + cancel + ']'); - IgniteCheckedException e = null; + IgniteException e = null; try { // Assuming that no methods are called on this loader after this method is called. @@ -668,7 +668,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay ctx.io().removeMessageListener(topic); } - catch (IgniteCheckedException e0) { + catch (IgniteException e0) { e = e0; } @@ -686,7 +686,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay } /** {@inheritDoc} */ - @Override public void close() throws IgniteCheckedException { + @Override public void close() throws IgniteException { close(false); } @@ -910,7 +910,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay curFut.onDone(t.get()); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { curFut.onDone(e); } } @@ -949,7 +949,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay if (topicBytes == null) topicBytes = ctx.config().getMarshaller().marshal(topic); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to marshal (request will not be sent).", e); return; @@ -1071,7 +1071,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay errBytes, jobPda0 != null ? jobPda0.classLoader() : U.gridClassLoader()); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { f.onDone(null, new IgniteCheckedException("Failed to unmarshal response.", e)); return; @@ -1094,7 +1094,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay try { f.cancel(); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { U.error(log, "Failed to cancel mini-future.", e); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java index d47c661..b5edd99 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java @@ -34,7 +34,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage { /** Write exception. */ @GridDirectTransient - private IgniteCheckedException err; + private IgniteException err; /** */ private byte[] errBytes; @@ -51,7 +51,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage { * @param id Request ID. * @param err Error. */ - public GridGgfsAckMessage(IgniteUuid fileId, long id, @Nullable IgniteCheckedException err) { + public GridGgfsAckMessage(IgniteUuid fileId, long id, @Nullable IgniteException err) { this.fileId = fileId; this.id = id; this.err = err; @@ -74,7 +74,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage { /** * @return Error occurred when writing this batch, if any. */ - public IgniteCheckedException error() { + public IgniteException error() { return err; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java index e22c0b7..ee1b4a5 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java @@ -41,33 +41,33 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid } /** {@inheritDoc} */ - @Override public void format() throws IgniteCheckedException { + @Override public void format() throws IgniteException { saveOrGet(ggfs.formatAsync()); } /** {@inheritDoc} */ @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr, - Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException { + Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException { return saveOrGet(ggfs.executeAsync(task, rslvr, paths, arg)); } /** {@inheritDoc} */ @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg) - throws IgniteCheckedException { + throws IgniteException { return saveOrGet(ggfs.executeAsync(task, rslvr, paths, skipNonExistentFiles, maxRangeLen, arg)); } /** {@inheritDoc} */ @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls, - @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException { + @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException { return saveOrGet(ggfs.executeAsync(taskCls, rslvr, paths, arg)); } /** {@inheritDoc} */ @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls, @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles, - long maxRangeLen, @Nullable T arg) throws IgniteCheckedException { + long maxRangeLen, @Nullable T arg) throws IgniteException { return saveOrGet(ggfs.executeAsync(taskCls, rslvr, paths, skipNonExistentFiles, maxRangeLen, arg)); } @@ -88,17 +88,17 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid /** {@inheritDoc} */ @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize, - int seqReadsBeforePrefetch) throws IgniteCheckedException { + int seqReadsBeforePrefetch) throws IgniteException { return ggfs.open(path, bufSize, seqReadsBeforePrefetch); } /** {@inheritDoc} */ - @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException { + @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteException { return ggfs.open(path); } /** {@inheritDoc} */ - @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException { + @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteException { return ggfs.open(path, bufSize); } @@ -168,118 +168,118 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid } /** {@inheritDoc} */ - @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteCheckedException { + @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteException { return ggfs.summary(path); } /** {@inheritDoc} */ - @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteCheckedException { + @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteException { return ggfs.create(path, overwrite); } /** {@inheritDoc} */ @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite, int replication, - long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException { + long blockSize, @Nullable Map<String, String> props) throws IgniteException { return ggfs.create(path, bufSize, overwrite, replication, blockSize, props); } /** {@inheritDoc} */ @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite, @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props) - throws IgniteCheckedException { + throws IgniteException { return ggfs.create(path, bufSize, overwrite, affKey, replication, blockSize, props); } /** {@inheritDoc} */ - @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteCheckedException { + @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteException { return ggfs.append(path, create); } /** {@inheritDoc} */ @Override public IgniteFsOutputStream append(IgniteFsPath path, int bufSize, boolean create, - @Nullable Map<String, String> props) throws IgniteCheckedException { + @Nullable Map<String, String> props) throws IgniteException { return ggfs.append(path, bufSize, create, props); } /** {@inheritDoc} */ - @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws IgniteCheckedException { + @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws IgniteException { ggfs.setTimes(path, accessTime, modificationTime); } /** {@inheritDoc} */ @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len) - throws IgniteCheckedException { + throws IgniteException { return ggfs.affinity(path, start, len); } /** {@inheritDoc} */ @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len, long maxLen) - throws IgniteCheckedException { + throws IgniteException { return ggfs.affinity(path, start, len, maxLen); } /** {@inheritDoc} */ - @Override public IgniteFsMetrics metrics() throws IgniteCheckedException { + @Override public IgniteFsMetrics metrics() throws IgniteException { return ggfs.metrics(); } /** {@inheritDoc} */ - @Override public void resetMetrics() throws IgniteCheckedException { + @Override public void resetMetrics() throws IgniteException { ggfs.resetMetrics(); } /** {@inheritDoc} */ - @Override public long size(IgniteFsPath path) throws IgniteCheckedException { + @Override public long size(IgniteFsPath path) throws IgniteException { return ggfs.size(path); } /** {@inheritDoc} */ - @Override public boolean exists(IgniteFsPath path) throws IgniteCheckedException { + @Override public boolean exists(IgniteFsPath path) throws IgniteException { return ggfs.exists(path); } /** {@inheritDoc} */ - @Nullable @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteCheckedException { + @Nullable @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteException { return ggfs.update(path, props); } /** {@inheritDoc} */ - @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteCheckedException { + @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteException { ggfs.rename(src, dest); } /** {@inheritDoc} */ - @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteCheckedException { + @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteException { return ggfs.delete(path, recursive); } /** {@inheritDoc} */ - @Override public void mkdirs(IgniteFsPath path) throws IgniteCheckedException { + @Override public void mkdirs(IgniteFsPath path) throws IgniteException { ggfs.mkdirs(path); } /** {@inheritDoc} */ - @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteCheckedException { + @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteException { ggfs.mkdirs(path, props); } /** {@inheritDoc} */ - @Override public Collection<IgniteFsPath> listPaths(IgniteFsPath path) throws IgniteCheckedException { + @Override public Collection<IgniteFsPath> listPaths(IgniteFsPath path) throws IgniteException { return ggfs.listPaths(path); } /** {@inheritDoc} */ - @Override public Collection<IgniteFsFile> listFiles(IgniteFsPath path) throws IgniteCheckedException { + @Override public Collection<IgniteFsFile> listFiles(IgniteFsPath path) throws IgniteException { return ggfs.listFiles(path); } /** {@inheritDoc} */ - @Nullable @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteCheckedException { + @Nullable @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteException { return ggfs.info(path); } /** {@inheritDoc} */ - @Override public long usedSpaceSize() throws IgniteCheckedException { + @Override public long usedSpaceSize() throws IgniteException { return ggfs.usedSpaceSize(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java index a42f530..1c259ad 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java @@ -40,10 +40,10 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.*; import java.util.concurrent.locks.*; +import static org.apache.ignite.events.IgniteEventType.*; import static org.gridgain.grid.cache.GridCacheAtomicityMode.*; import static org.gridgain.grid.cache.GridCacheTxConcurrency.*; import static org.gridgain.grid.cache.GridCacheTxIsolation.*; -import static org.apache.ignite.events.IgniteEventType.*; import static org.gridgain.grid.kernal.GridTopic.*; import static org.gridgain.grid.kernal.managers.communication.GridIoPolicy.*; import static org.gridgain.grid.kernal.processors.cache.GridCacheUtils.*; @@ -127,12 +127,7 @@ public class GridGgfsDataManager extends GridGgfsManager { */ void awaitInit() { if (!dataCacheStartFut.isDone()) { - try { - dataCacheStartFut.get(); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } + dataCacheStartFut.get(); } } @@ -644,7 +639,7 @@ public class GridGgfsDataManager extends GridGgfsManager { } } } - catch (IgniteCheckedException e) { + catch (IgniteException e) { log.error("Failed to clean up file range [fileInfo=" + fileInfo + ", range=" + range + ']', e); } } @@ -1028,7 +1023,7 @@ public class GridGgfsDataManager extends GridGgfsManager { try { ggfsCtx.send(nodeId, topic, msg, SYSTEM_POOL); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { completionFut.onError(nodeId, e); } @@ -1046,7 +1041,7 @@ public class GridGgfsDataManager extends GridGgfsManager { completionFut.onWriteAck(nodeId, batchId); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { completionFut.onError(nodeId, e); } } @@ -1274,12 +1269,12 @@ public class GridGgfsDataManager extends GridGgfsManager { private void processBlocksMessage(final UUID nodeId, final GridGgfsBlocksMessage blocksMsg) { storeBlocksAsync(blocksMsg.blocks()).listenAsync(new CI1<IgniteFuture<?>>() { @Override public void apply(IgniteFuture<?> fut) { - IgniteCheckedException err = null; + IgniteException err = null; try { fut.get(); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { err = e; } @@ -1715,7 +1710,7 @@ public class GridGgfsDataManager extends GridGgfsManager { catch (GridInterruptedException ignored) { // Ignore interruption during shutdown. } - catch (IgniteCheckedException e) { + catch (IgniteException e) { log.error("Failed to remove file contents: " + fileInfo, e); } finally { @@ -1726,14 +1721,14 @@ public class GridGgfsDataManager extends GridGgfsManager { ldr.removeData(new GridGgfsBlockKey(fileId, fileInfo.affinityKey(), fileInfo.evictExclude(), block)); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { log.error("Failed to remove file contents: " + fileInfo, e); } finally { try { ldr.close(isCancelled()); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { log.error("Failed to stop data loader while shutting down ggfs async delete thread.", e); } finally { @@ -1828,7 +1823,7 @@ public class GridGgfsDataManager extends GridGgfsManager { * @param nodeId Node ID. * @param e Caught exception. */ - private void onError(UUID nodeId, IgniteCheckedException e) { + private void onError(UUID nodeId, IgniteException e) { Set<Long> reqIds = pendingAcks.get(nodeId); // If waiting for ack from this node. @@ -1844,7 +1839,7 @@ public class GridGgfsDataManager extends GridGgfsManager { /** * @param e Error. */ - private void onLocalError(IgniteCheckedException e) { + private void onLocalError(IgniteException e) { if (e instanceof IgniteFsOutOfSpaceException) onDone(new IgniteCheckedException("Failed to write data (not enough space on node): " + ggfsCtx.kernalContext().localNodeId(), e)); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java index 2e78702..8b3152b 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java @@ -40,13 +40,13 @@ public interface GridGgfsEx extends IgniteFs { /** {@inheritDoc} */ @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize, int seqReadsBeforePrefetch) - throws IgniteCheckedException; + throws IgniteException; /** {@inheritDoc} */ - @Override GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException; + @Override GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteException; /** {@inheritDoc} */ - @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException; + @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteException; /** * Gets global space counters. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java index 44e110f..214191e 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java @@ -461,7 +461,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public boolean exists(IgniteFsPath path) throws IgniteCheckedException { + @Override public boolean exists(IgniteFsPath path) throws IgniteException { A.notNull(path, "path"); if (log.isDebugEnabled()) @@ -497,7 +497,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteCheckedException { + @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -526,7 +526,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteCheckedException { + @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -554,7 +554,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteCheckedException { + @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -610,7 +610,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteCheckedException { + @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteException { if (enterBusy()) { try { A.notNull(src, "src"); @@ -719,7 +719,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteCheckedException { + @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -815,12 +815,12 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public void mkdirs(IgniteFsPath path) throws IgniteCheckedException { + @Override public void mkdirs(IgniteFsPath path) throws IgniteException { mkdirs(path, null); } /** {@inheritDoc} */ - @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteCheckedException { + @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -907,7 +907,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public Collection<IgniteFsPath> listPaths(final IgniteFsPath path) throws IgniteCheckedException { + @Override public Collection<IgniteFsPath> listPaths(final IgniteFsPath path) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -959,7 +959,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public Collection<IgniteFsFile> listFiles(final IgniteFsPath path) throws IgniteCheckedException { + @Override public Collection<IgniteFsFile> listFiles(final IgniteFsPath path) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -1026,7 +1026,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public long usedSpaceSize() throws IgniteCheckedException { + @Override public long usedSpaceSize() throws IgniteException { return metrics().localSpaceSize(); } @@ -1036,18 +1036,18 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException { + @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteException { return open(path, cfg.getStreamBufferSize(), cfg.getSequentialReadsBeforePrefetch()); } /** {@inheritDoc} */ - @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException { + @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteException { return open(path, bufSize, cfg.getSequentialReadsBeforePrefetch()); } /** {@inheritDoc} */ @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize, int seqReadsBeforePrefetch) - throws IgniteCheckedException { + throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -1107,20 +1107,20 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteCheckedException { + @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteException { return create0(path, cfg.getStreamBufferSize(), overwrite, null, 0, null, true); } /** {@inheritDoc} */ @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite, int replication, - long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException { + long blockSize, @Nullable Map<String, String> props) throws IgniteException { return create0(path, bufSize, overwrite, null, replication, props, false); } /** {@inheritDoc} */ @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite, @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props) - throws IgniteCheckedException { + throws IgniteException { return create0(path, bufSize, overwrite, affKey, replication, props, false); } @@ -1248,13 +1248,13 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteCheckedException { + @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteException { return append(path, cfg.getStreamBufferSize(), create, null); } /** {@inheritDoc} */ @Override public IgniteFsOutputStream append(final IgniteFsPath path, final int bufSize, boolean create, - @Nullable Map<String, String> props) throws IgniteCheckedException { + @Nullable Map<String, String> props) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -1333,7 +1333,7 @@ public final class GridGgfsImpl implements GridGgfsEx { /** {@inheritDoc} */ @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) - throws IgniteCheckedException { + throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -1380,13 +1380,13 @@ public final class GridGgfsImpl implements GridGgfsEx { /** {@inheritDoc} */ @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len) - throws IgniteCheckedException { + throws IgniteException { return affinity(path, start, len, 0L); } /** {@inheritDoc} */ @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len, long maxLen) - throws IgniteCheckedException { + throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -1431,7 +1431,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public IgniteFsMetrics metrics() throws IgniteCheckedException { + @Override public IgniteFsMetrics metrics() throws IgniteException { if (enterBusy()) { try { IgniteFsPathSummary sum = new IgniteFsPathSummary(); @@ -1482,7 +1482,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public long size(IgniteFsPath path) throws IgniteCheckedException { + @Override public long size(IgniteFsPath path) throws IgniteException { if (enterBusy()) { try { A.notNull(path, "path"); @@ -1534,7 +1534,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public void format() throws IgniteCheckedException { + @Override public void format() throws IgniteException { formatAsync().get(); } @@ -1668,27 +1668,27 @@ public final class GridGgfsImpl implements GridGgfsEx { /** {@inheritDoc} */ @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr, - Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException { + Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException { return executeAsync(task, rslvr, paths, arg).get(); } /** {@inheritDoc} */ @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg) - throws IgniteCheckedException { + throws IgniteException { return executeAsync(task, rslvr, paths, skipNonExistentFiles, maxRangeLen, arg).get(); } /** {@inheritDoc} */ @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls, - @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException { + @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException { return executeAsync(taskCls, rslvr, paths, arg).get(); } /** {@inheritDoc} */ @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls, @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles, - long maxRangeSize, @Nullable T arg) throws IgniteCheckedException { + long maxRangeSize, @Nullable T arg) throws IgniteException { return executeAsync(taskCls, rslvr, paths, skipNonExistentFiles, maxRangeSize, arg).get(); } @@ -1996,7 +1996,7 @@ public final class GridGgfsImpl implements GridGgfsEx { /** {@inheritDoc} */ @Override protected Collection<? extends ComputeJob> split(int gridSize, Object arg) - throws IgniteCheckedException { + throws IgniteException { Collection<ComputeJob> res = new ArrayList<>(gridSize); for (int i = 0; i < gridSize; i++) { @@ -2005,7 +2005,7 @@ public final class GridGgfsImpl implements GridGgfsEx { @IgniteInstanceResource private Ignite g; - @Nullable @Override public IgniteBiTuple<Long, Long> execute() throws IgniteCheckedException { + @Nullable @Override public IgniteBiTuple<Long, Long> execute() throws IgniteException { IgniteFs ggfs = ((GridKernal)g).context().ggfs().ggfs(ggfsName); if (ggfs == null) @@ -2024,7 +2024,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Nullable @Override public IgniteBiTuple<Long, Long> reduce(List<ComputeJobResult> results) throws IgniteCheckedException { + @Nullable @Override public IgniteBiTuple<Long, Long> reduce(List<ComputeJobResult> results) throws IgniteException { long used = 0; long max = 0; @@ -2041,7 +2041,7 @@ public final class GridGgfsImpl implements GridGgfsEx { } /** {@inheritDoc} */ - @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteCheckedException { + @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException { // Never failover. return ComputeJobResultPolicy.WAIT; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java index 26ffe06..dccee38 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java @@ -71,7 +71,7 @@ public class GridGgfsJobImpl implements ComputeJob, GridInternalWrapper<IgniteFs } /** {@inheritDoc} */ - @Override public Object execute() throws IgniteCheckedException { + @Override public Object execute() throws IgniteException { IgniteFs ggfs = ignite.fileSystem(ggfsName); try (IgniteFsInputStream in = ggfs.open(path)) { @@ -93,7 +93,7 @@ public class GridGgfsJobImpl implements ComputeJob, GridInternalWrapper<IgniteFs return job.execute(ggfs, new IgniteFsFileRange(path, split.start(), split.length()), in); } catch (IOException e) { - throw new IgniteCheckedException("Failed to execute GGFS job for file split [ggfsName=" + ggfsName + + throw new IgniteException("Failed to execute GGFS job for file split [ggfsName=" + ggfsName + ", path=" + path + ", start=" + start + ", len=" + len + ']', e); } }