http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java index 97316e3..d09f126 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java @@ -9,6 +9,7 @@ package org.gridgain.grid.kernal.processors.interop.os; +import org.apache.ignite.*; import org.gridgain.grid.*; import org.gridgain.grid.kernal.*; import org.gridgain.grid.kernal.processors.interop.*; @@ -36,12 +37,12 @@ public class GridOsInteropProcessor extends GridInteropProcessorAdapter { } /** {@inheritDoc} */ - @Override public void awaitStart() throws GridException { + @Override public void awaitStart() throws IgniteCheckedException { throw new UnsupportedOperationException(ERR_MSG); } /** {@inheritDoc} */ - @Override public long environmentPointer() throws GridException { + @Override public long environmentPointer() throws IgniteCheckedException { throw new UnsupportedOperationException(ERR_MSG); } @@ -56,12 +57,12 @@ public class GridOsInteropProcessor extends GridInteropProcessorAdapter { } /** {@inheritDoc} */ - @Override public GridInteropTarget projection() throws GridException { + @Override public GridInteropTarget projection() throws IgniteCheckedException { throw new UnsupportedOperationException(ERR_MSG); } /** {@inheritDoc} */ - @Override public GridInteropTarget cache(@Nullable String name) throws GridException { + @Override public GridInteropTarget cache(@Nullable String name) throws IgniteCheckedException { throw new UnsupportedOperationException(ERR_MSG); } }
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java index 9a4a21e..3f7d4e1 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java @@ -9,6 +9,7 @@ package org.gridgain.grid.kernal.processors.job; +import org.apache.ignite.*; import org.apache.ignite.cluster.*; import org.apache.ignite.compute.*; import org.apache.ignite.events.*; @@ -176,9 +177,9 @@ public class GridJobProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void start() throws GridException { + @Override public void start() throws IgniteCheckedException { if (metricsUpdateFreq < -1) - throw new GridException("Invalid value for 'metricsUpdateFrequency' configuration property " + + throw new IgniteCheckedException("Invalid value for 'metricsUpdateFrequency' configuration property " + "(should be greater than or equals to -1): " + metricsUpdateFreq); if (metricsUpdateFreq == -1) @@ -291,7 +292,7 @@ public class GridJobProcessor extends GridProcessorAdapter { * @param sndReply {@code True} to send reply. */ private void rejectJob(GridJobWorker job, boolean sndReply) { - GridException e = new ComputeExecutionRejectedException("Job was cancelled before execution [taskSesId=" + + IgniteCheckedException e = new ComputeExecutionRejectedException("Job was cancelled before execution [taskSesId=" + job.getSession().getId() + ", jobId=" + job.getJobId() + ", job=" + job.getJob() + ']'); job.finishJob(null, e, sndReply); @@ -325,9 +326,9 @@ public class GridJobProcessor extends GridProcessorAdapter { /** * @param ses Session. * @param attrs Attributes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void setAttributes(GridJobSessionImpl ses, Map<?, ?> attrs) throws GridException { + public void setAttributes(GridJobSessionImpl ses, Map<?, ?> attrs) throws IgniteCheckedException { assert ses.isFullSupport(); long timeout = ses.getEndTime() - U.currentTimeMillis(); @@ -344,7 +345,7 @@ public class GridJobProcessor extends GridProcessorAdapter { ClusterNode taskNode = ctx.discovery().node(ses.getTaskNodeId()); if (taskNode == null) - throw new GridException("Node that originated task execution has left grid: " + + throw new IgniteCheckedException("Node that originated task execution has left grid: " + ses.getTaskNodeId()); boolean loc = ctx.localNodeId().equals(taskNode.id()) && !ctx.config().isMarshalLocalJobs(); @@ -368,9 +369,9 @@ public class GridJobProcessor extends GridProcessorAdapter { /** * @param ses Session. * @return Siblings. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public Collection<ComputeJobSibling> requestJobSiblings(final ComputeTaskSession ses) throws GridException { + public Collection<ComputeJobSibling> requestJobSiblings(final ComputeTaskSession ses) throws IgniteCheckedException { assert ses != null; final UUID taskNodeId = ses.getTaskNodeId(); @@ -378,7 +379,7 @@ public class GridJobProcessor extends GridProcessorAdapter { ClusterNode taskNode = ctx.discovery().node(taskNodeId); if (taskNode == null) - throw new GridException("Node that originated task execution has left grid: " + taskNodeId); + throw new IgniteCheckedException("Node that originated task execution has left grid: " + taskNodeId); // Tuple: error message-response. final IgniteBiTuple<String, GridJobSiblingsResponse> t = F.t2(); @@ -404,7 +405,7 @@ public class GridJobProcessor extends GridProcessorAdapter { try { res.unmarshalSiblings(marsh); } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(log, "Failed to unmarshal job siblings.", e); err = e.getMessage(); @@ -473,7 +474,7 @@ public class GridJobProcessor extends GridProcessorAdapter { taskNode = ctx.discovery().node(taskNodeId); if (taskNode == null) - throw new GridException("Node that originated task execution has left grid: " + taskNodeId); + throw new IgniteCheckedException("Node that originated task execution has left grid: " + taskNodeId); // 6. Wait for result. lock.lock(); @@ -485,18 +486,18 @@ public class GridJobProcessor extends GridProcessorAdapter { cond.await(netTimeout, MILLISECONDS); if (t.isEmpty()) - throw new GridException("Timed out waiting for job siblings (consider increasing" + + throw new IgniteCheckedException("Timed out waiting for job siblings (consider increasing" + "'networkTimeout' configuration property) [ses=" + ses + ", netTimeout=" + netTimeout + ']'); // Error is set? if (t.get1() != null) - throw new GridException(t.get1()); + throw new IgniteCheckedException(t.get1()); else // Return result return t.get2().jobSiblings(); } catch (InterruptedException e) { - throw new GridException("Interrupted while waiting for job siblings response: " + ses, e); + throw new IgniteCheckedException("Interrupted while waiting for job siblings response: " + ses, e); } finally { lock.unlock(); @@ -1009,8 +1010,8 @@ public class GridJobProcessor extends GridProcessorAdapter { jobCtx = new GridJobContextImpl(ctx, req.getJobId(), jobAttrs); } - catch (GridException e) { - GridException ex = new GridException("Failed to deserialize task attributes [taskName=" + + catch (IgniteCheckedException e) { + IgniteCheckedException ex = new IgniteCheckedException("Failed to deserialize task attributes [taskName=" + req.getTaskName() + ", taskClsName=" + req.getTaskClassName() + ", codeVer=" + req.getUserVersion() + ", taskClsLdr=" + dep.classLoader() + ']', e); @@ -1090,7 +1091,7 @@ public class GridJobProcessor extends GridProcessorAdapter { } else { // Deployment is null. - GridException ex = new GridDeploymentException("Task was not deployed or was redeployed since " + + IgniteCheckedException ex = new GridDeploymentException("Task was not deployed or was redeployed since " + "task execution [taskName=" + req.getTaskName() + ", taskClsName=" + req.getTaskClassName() + ", codeVer=" + req.getUserVersion() + ", clsLdrId=" + req.getClassLoaderId() + ", seqNum=" + req.getClassLoaderId().localId() + ", depMode=" + req.getDeploymentMode() + @@ -1173,7 +1174,7 @@ public class GridJobProcessor extends GridProcessorAdapter { // Even if job has been removed from another thread, we need to reject it // here since job has never been executed. - GridException e2 = new ComputeExecutionRejectedException( + IgniteCheckedException e2 = new ComputeExecutionRejectedException( "Job was cancelled before execution [jobSes=" + jobWorker. getSession() + ", job=" + jobWorker.getJob() + ']'); @@ -1221,7 +1222,7 @@ public class GridJobProcessor extends GridProcessorAdapter { // Even if job was removed from another thread, we need to reject it // here since job has never been executed. - GridException e2 = new ComputeExecutionRejectedException("Job has been rejected " + + IgniteCheckedException e2 = new ComputeExecutionRejectedException("Job has been rejected " + "[jobSes=" + jobWorker.getSession() + ", job=" + jobWorker.getJob() + ']', e); if (metricsUpdateFreq > -1L) @@ -1241,7 +1242,7 @@ public class GridJobProcessor extends GridProcessorAdapter { * @param ex Exception that happened. * @param endTime Job end time. */ - private void handleException(ClusterNode node, GridJobExecuteRequest req, GridException ex, long endTime) { + private void handleException(ClusterNode node, GridJobExecuteRequest req, IgniteCheckedException ex, long endTime) { UUID locNodeId = ctx.localNodeId(); ClusterNode sndNode = ctx.discovery().node(node.id()); @@ -1319,7 +1320,7 @@ public class GridJobProcessor extends GridProcessorAdapter { // Send response to common topic as unordered message. ctx.io().send(sndNode, TOPIC_TASK, jobRes, req.isInternal() ? MANAGEMENT_POOL : SYSTEM_POOL); } - catch (GridException e) { + catch (IgniteCheckedException e) { // The only option here is to log, as we must assume that resending will fail too. if (isDeadNode(node.id())) // Avoid stack trace for left nodes. @@ -1399,7 +1400,7 @@ public class GridJobProcessor extends GridProcessorAdapter { ses.setInternal(attrs); } } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(log, "Failed to deserialize session attributes.", e); } finally { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java index abbd45e..9d100e3 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java @@ -365,7 +365,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { boolean initialize(GridDeployment dep, Class<?> taskCls) { assert dep != null; - GridException ex = null; + IgniteCheckedException ex = null; try { if (job == null) { @@ -381,7 +381,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { if (!internal && ctx.event().isRecordable(EVT_JOB_QUEUED)) recordEvent(EVT_JOB_QUEUED, "Job got queued for computation."); } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(log, "Failed to initialize job [jobId=" + ses.getJobId() + ", ses=" + ses + ']', e); ex = e; @@ -452,7 +452,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { Object res = null; - GridException ex = null; + IgniteCheckedException ex = null; try { ctx.job().currentTaskSession(ses); @@ -463,7 +463,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { sndRes = false; else { res = U.wrapThreadLoader(dep.classLoader(), new Callable<Object>() { - @Nullable @Override public Object call() throws GridException { + @Nullable @Override public Object call() throws IgniteCheckedException { try { if (internal && ctx.config().isPeerClassLoadingEnabled()) ctx.job().internal(true); @@ -481,7 +481,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { log.debug("Job execution has successfully finished [job=" + job + ", res=" + res + ']'); } } - catch (GridException e) { + catch (IgniteCheckedException e) { if (sysStopping && e.hasCause(GridInterruptedException.class, InterruptedException.class)) { ex = handleThrowable(e); @@ -530,10 +530,10 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { * @param e Exception. * @return Wrapped exception. */ - private GridException handleThrowable(Throwable e) { + private IgniteCheckedException handleThrowable(Throwable e) { String msg = null; - GridException ex = null; + IgniteCheckedException ex = null; // Special handling for weird interrupted exception which // happens due to JDk 1.5 bug. @@ -541,7 +541,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { msg = "Failed to execute job due to interrupted exception."; // Turn interrupted exception into checked exception. - ex = new GridException(msg, e); + ex = new IgniteCheckedException(msg, e); } // Special 'NoClassDefFoundError' handling if P2P is on. We had many questions // about this exception and decided to change error message. @@ -640,7 +640,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { * @param ex Error. * @param sndReply If {@code true}, reply will be sent. */ - void finishJob(@Nullable Object res, @Nullable GridException ex, boolean sndReply) { + void finishJob(@Nullable Object res, @Nullable IgniteCheckedException ex, boolean sndReply) { // Avoid finishing a job more than once from different threads. if (!finishing.compareAndSet(false, true)) return; @@ -735,7 +735,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { // Send response to common topic as unordered message. ctx.io().send(sndNode, TOPIC_TASK, jobRes, internal ? MANAGEMENT_POOL : SYSTEM_POOL); } - catch (GridException e) { + catch (IgniteCheckedException e) { // Log and invoke the master-leave callback. if (isDeadNode(taskNode.id())) { onMasterNodeLeft(); @@ -811,7 +811,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject { log.debug("Successfully executed GridComputeJobMasterLeaveAware.onMasterNodeLeft() callback " + "[nodeId=" + taskNode.id() + ", jobId=" + ses.getJobId() + ", job=" + job + ']'); } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(log, "Failed to execute GridComputeJobMasterLeaveAware.onMasterNodeLeft() callback " + "[nodeId=" + taskNode.id() + ", jobId=" + ses.getJobId() + ", job=" + job + ']', e); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java index 1edfd9b..857bf7c 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java @@ -9,8 +9,8 @@ package org.gridgain.grid.kernal.processors.jobmetrics; +import org.apache.ignite.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.kernal.*; import org.gridgain.grid.kernal.processors.*; import org.gridgain.grid.util.typedef.*; @@ -114,12 +114,12 @@ public class GridJobMetricsProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void start() throws GridException { + @Override public void start() throws IgniteCheckedException { assertParameter(histSize > 0, "metricsHistorySize > 0"); assertParameter(expireTime > 0, "metricsExpireTime > 0"); if (metrics.snapshotsQueues == null) - throw new GridException("Invalid concurrency level configured " + + throw new IgniteCheckedException("Invalid concurrency level configured " + "(is 'GRIDGAIN_JOBS_METRICS_CONCURRENCY_LEVEL' system property properly set?)."); if (log.isDebugEnabled()) @@ -129,7 +129,7 @@ public class GridJobMetricsProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void stop(boolean cancel) throws GridException { + @Override public void stop(boolean cancel) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Job metrics processor stopped."); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java index 11d0ca8..10c699a 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java @@ -9,9 +9,9 @@ package org.gridgain.grid.kernal.processors.offheap; +import org.apache.ignite.*; import org.apache.ignite.lang.*; import org.apache.ignite.marshaller.*; -import org.gridgain.grid.*; import org.gridgain.grid.kernal.*; import org.gridgain.grid.kernal.processors.*; import org.gridgain.grid.util.*; @@ -65,7 +65,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void stop(boolean cancel) throws GridException { + @Override public void stop(boolean cancel) throws IgniteCheckedException { super.stop(cancel); for (GridOffHeapPartitionedMap m : offheap.values()) @@ -89,9 +89,9 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param key Key. * @param keyBytes Optional key bytes. * @return Key bytes - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - private byte[] keyBytes(Object key, @Nullable byte[] keyBytes) throws GridException { + private byte[] keyBytes(Object key, @Nullable byte[] keyBytes) throws IgniteCheckedException { assert key != null; return keyBytes != null ? keyBytes : marsh.marshal(key); @@ -118,9 +118,9 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param key Key. * @param keyBytes Key bytes. * @return {@code true} If offheap space contains value for the given key. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public boolean contains(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws GridException { + public boolean contains(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); return m != null && m.contains(part, U.hash(key), keyBytes(key, keyBytes)); @@ -134,9 +134,9 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param key Key. * @param keyBytes Key bytes. * @return Value bytes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - @Nullable public byte[] get(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws GridException { + @Nullable public byte[] get(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); return m == null ? null : m.get(part, U.hash(key), keyBytes(key, keyBytes)); @@ -152,10 +152,10 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param key Key. * @param keyBytes Key bytes. * @return Tuple where first value is pointer and second is value size. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable String spaceName, int part, Object key, - byte[] keyBytes) throws GridException { + byte[] keyBytes) throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); return m == null ? null : m.valuePointer(part, U.hash(key), keyBytes(key, keyBytes)); @@ -168,9 +168,9 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param part Partition. * @param key Key. * @param keyBytes Key bytes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void enableEviction(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws GridException { + public void enableEviction(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); if (m != null) @@ -186,10 +186,10 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param keyBytes Key bytes. * @param ldr Class loader. * @return Value bytes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ @Nullable public <T> T getValue(@Nullable String spaceName, int part, Object key, byte[] keyBytes, - @Nullable ClassLoader ldr) throws GridException { + @Nullable ClassLoader ldr) throws IgniteCheckedException { byte[] valBytes = get(spaceName, part, key, keyBytes); if (valBytes == null) @@ -206,9 +206,9 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param key Key. * @param keyBytes Key bytes. * @return Value bytes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - @Nullable public byte[] remove(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws GridException { + @Nullable public byte[] remove(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); return m == null ? null : m.remove(part, U.hash(key), keyBytes(key, keyBytes)); @@ -222,14 +222,14 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param key Key. * @param keyBytes Key bytes. * @param valBytes Value bytes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public void put(@Nullable String spaceName, int part, Object key, byte[] keyBytes, byte[] valBytes) - throws GridException { + throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); if (m == null) - throw new GridException("Failed to write data to off-heap space, no space registered for name: " + + throw new IgniteCheckedException("Failed to write data to off-heap space, no space registered for name: " + spaceName); m.put(part, U.hash(key), keyBytes(key, keyBytes), valBytes); @@ -243,9 +243,9 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @param key Key. * @param keyBytes Key bytes. * @return {@code true} If succeeded. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public boolean removex(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws GridException { + public boolean removex(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes)); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java index 4b80cd3..0a7146c 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java @@ -9,6 +9,7 @@ package org.gridgain.grid.kernal.processors.port; +import org.apache.ignite.*; import org.apache.ignite.spi.*; import org.gridgain.grid.*; import org.gridgain.grid.kernal.*; @@ -44,13 +45,13 @@ public class GridPortProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void start() throws GridException { + @Override public void start() throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Started port processor."); } /** {@inheritDoc} */ - @Override public void stop(boolean cancel) throws GridException { + @Override public void stop(boolean cancel) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Stopped port processor."); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java index e22bfd4..18b2832 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java @@ -9,9 +9,9 @@ package org.gridgain.grid.kernal.processors.query; +import org.apache.ignite.*; import org.apache.ignite.lang.*; import org.apache.ignite.spi.indexing.*; -import org.gridgain.grid.*; import org.gridgain.grid.kernal.*; import org.gridgain.grid.util.lang.*; import org.jetbrains.annotations.*; @@ -26,16 +26,16 @@ public interface GridQueryIndexing { * Starts indexing. * * @param ctx Context. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void start(GridKernalContext ctx) throws GridException; + public void start(GridKernalContext ctx) throws IgniteCheckedException; /** * Stops indexing. * - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void stop() throws GridException; + public void stop() throws IgniteCheckedException; /** * Queries individual fields (generally used by JDBC drivers). @@ -45,10 +45,10 @@ public interface GridQueryIndexing { * @param params Query parameters. * @param filters Space name and key filters. * @return Query result. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public <K, V> GridQueryFieldsResult queryFields(@Nullable String spaceName, String qry, - Collection<Object> params, GridIndexingQueryFilter filters) throws GridException; + Collection<Object> params, GridIndexingQueryFilter filters) throws IgniteCheckedException; /** * Executes regular query. @@ -59,10 +59,10 @@ public interface GridQueryIndexing { * @param type Query return type. * @param filters Space name and key filters. * @return Queried rows. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(@Nullable String spaceName, String qry, - Collection<Object> params, GridQueryTypeDescriptor type, GridIndexingQueryFilter filters) throws GridException; + Collection<Object> params, GridQueryTypeDescriptor type, GridIndexingQueryFilter filters) throws IgniteCheckedException; /** * Executes text query. @@ -72,10 +72,10 @@ public interface GridQueryIndexing { * @param type Query return type. * @param filters Space name and key filter. * @return Queried rows. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(@Nullable String spaceName, String qry, - GridQueryTypeDescriptor type, GridIndexingQueryFilter filters) throws GridException; + GridQueryTypeDescriptor type, GridIndexingQueryFilter filters) throws IgniteCheckedException; /** * Gets size of index for given type or -1 if it is a unknown type. @@ -84,29 +84,29 @@ public interface GridQueryIndexing { * @param desc Type descriptor. * @param filters Filters. * @return Objects number. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public long size(@Nullable String spaceName, GridQueryTypeDescriptor desc, GridIndexingQueryFilter filters) - throws GridException; + throws IgniteCheckedException; /** * Registers type if it was not known before or updates it otherwise. * * @param spaceName Space name. * @param desc Type descriptor. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. * @return {@code True} if type was registered, {@code false} if for some reason it was rejected. */ - public boolean registerType(@Nullable String spaceName, GridQueryTypeDescriptor desc) throws GridException; + public boolean registerType(@Nullable String spaceName, GridQueryTypeDescriptor desc) throws IgniteCheckedException; /** * Unregisters type and removes all corresponding data. * * @param spaceName Space name. * @param type Type descriptor. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void unregisterType(@Nullable String spaceName, GridQueryTypeDescriptor type) throws GridException; + public void unregisterType(@Nullable String spaceName, GridQueryTypeDescriptor type) throws IgniteCheckedException; /** * Updates index. Note that key is unique for space, so if space contains multiple indexes @@ -118,28 +118,28 @@ public interface GridQueryIndexing { * @param val Value. * @param ver Version. * @param expirationTime Expiration time or 0 if never expires. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public void store(@Nullable String spaceName, GridQueryTypeDescriptor type, Object key, Object val, byte[] ver, - long expirationTime) throws GridException; + long expirationTime) throws IgniteCheckedException; /** * Removes index entry by key. * * @param spaceName Space name. * @param key Key. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void remove(@Nullable String spaceName, Object key) throws GridException; + public void remove(@Nullable String spaceName, Object key) throws IgniteCheckedException; /** * Will be called when entry with given key is swapped. * * @param spaceName Space name. * @param key Key. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void onSwap(@Nullable String spaceName, Object key) throws GridException; + public void onSwap(@Nullable String spaceName, Object key) throws IgniteCheckedException; /** * Will be called when entry with given key is unswapped. @@ -148,9 +148,9 @@ public interface GridQueryIndexing { * @param key Key. * @param val Value. * @param valBytes Value bytes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void onUnswap(@Nullable String spaceName, Object key, Object val, byte[] valBytes) throws GridException; + public void onUnswap(@Nullable String spaceName, Object key, Object val, byte[] valBytes) throws IgniteCheckedException; /** * Rebuilds all indexes of given type. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java index 94db08d..6bc0235 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java @@ -9,6 +9,7 @@ package org.gridgain.grid.kernal.processors.query; +import org.apache.ignite.*; import org.apache.ignite.lang.*; import org.apache.ignite.portables.*; import org.apache.ignite.spi.indexing.*; @@ -71,7 +72,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** * @param ctx Kernal context. */ - public GridQueryProcessor(GridKernalContext ctx) throws GridException { + public GridQueryProcessor(GridKernalContext ctx) throws IgniteCheckedException { super(ctx); if (idxCls != null) { @@ -84,7 +85,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void start() throws GridException { + @Override public void start() throws IgniteCheckedException { super.start(); if (idx != null) { @@ -118,7 +119,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void stop(boolean cancel) throws GridException { + @Override public void stop(boolean cancel) throws IgniteCheckedException { super.stop(cancel); if (idx != null) @@ -131,9 +132,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param space Space. * @param valType Value type. * @return Objects number or -1 if this type is unknown for given SPI and space. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public long size(@Nullable String space, Class<?> valType) throws GridException { + public long size(@Nullable String space, Class<?> valType) throws IgniteCheckedException { checkEnabled(); if (!busyLock.enterBusy()) @@ -178,7 +179,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { */ private IgniteFuture<?> rebuildIndexes(@Nullable final String space, @Nullable final TypeDescriptor desc) { if (idx == null) - return new GridFinishedFuture<>(ctx, new GridException("Indexing is disabled.")); + return new GridFinishedFuture<>(ctx, new IgniteCheckedException("Indexing is disabled.")); if (desc == null || !desc.registered()) return new GridFinishedFuture<Void>(ctx); @@ -245,11 +246,11 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param valBytes Byte array with value data. * @param ver Cache entry version. * @param expirationTime Expiration time or 0 if never expires. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ @SuppressWarnings("unchecked") public <K, V> void store(final String space, final K key, @Nullable byte[] keyBytes, final V val, - @Nullable byte[] valBytes, byte[] ver, long expirationTime) throws GridException { + @Nullable byte[] valBytes, byte[] ver, long expirationTime) throws IgniteCheckedException { assert key != null; assert val != null; @@ -376,7 +377,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { return; if (!desc.valueClass().equals(valCls)) - throw new GridException("Failed to update index due to class name conflict" + + throw new IgniteCheckedException("Failed to update index due to class name conflict" + "(multiple classes with same simple name are stored in the same cache) " + "[expCls=" + desc.valueClass().getName() + ", actualCls=" + valCls.getName() + ']'); @@ -388,11 +389,11 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - private void checkEnabled() throws GridException { + private void checkEnabled() throws IgniteCheckedException { if (idx == null) - throw new GridException("Indexing is disabled."); + throw new IgniteCheckedException("Indexing is disabled."); } /** @@ -402,12 +403,12 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param resType Result type. * @param filters Filters. * @return Key/value rows. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(String space, String clause, Collection<Object> params, String resType, GridIndexingQueryFilter filters) - throws GridException { + throws IgniteCheckedException { checkEnabled(); if (!busyLock.enterBusy()) @@ -429,10 +430,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** * @param space Space. * @param key Key. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ @SuppressWarnings("unchecked") - public void remove(String space, Object key) throws GridException { + public void remove(String space, Object key) throws IgniteCheckedException { assert key != null; ctx.indexing().remove(space, key); @@ -546,11 +547,11 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param <K> Key type. * @param <V> Value type. * @return Key/value rows. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(String space, String clause, String resType, - GridIndexingQueryFilter filters) throws GridException { + GridIndexingQueryFilter filters) throws IgniteCheckedException { checkEnabled(); if (!busyLock.enterBusy()) @@ -575,10 +576,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param params Parameters collection. * @param filters Key and value filters. * @return Field rows. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public <K, V> GridQueryFieldsResult queryFields(@Nullable String space, String clause, Collection<Object> params, - GridIndexingQueryFilter filters) throws GridException { + GridIndexingQueryFilter filters) throws IgniteCheckedException { checkEnabled(); if (!busyLock.enterBusy()) @@ -597,9 +598,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { * * @param spaceName Space name. * @param key key. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void onSwap(String spaceName, Object key) throws GridException { + public void onSwap(String spaceName, Object key) throws IgniteCheckedException { ctx.indexing().onSwap(spaceName, key); if (idx == null) @@ -623,10 +624,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param key Key. * @param val Value. * @param valBytes Value bytes. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public void onUnswap(String spaceName, Object key, Object val, byte[] valBytes) - throws GridException { + throws IgniteCheckedException { ctx.indexing().onUnswap(spaceName, key, val); if (idx == null) @@ -648,9 +649,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { * * @param space Space name. * @param ldr Class loader to undeploy. - * @throws GridException If undeploy failed. + * @throws IgniteCheckedException If undeploy failed. */ - public void onUndeploy(@Nullable String space, ClassLoader ldr) throws GridException { + public void onUndeploy(@Nullable String space, ClassLoader ldr) throws IgniteCheckedException { if (idx == null) return; @@ -687,15 +688,15 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param cls Class. * @param type Type descriptor. * @param parent Parent in case of embeddable. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ static void processAnnotationsInClass(boolean key, Class<?> cls, TypeDescriptor type, - @Nullable ClassProperty parent) throws GridException { + @Nullable ClassProperty parent) throws IgniteCheckedException { if (U.isJdk(cls)) return; if (parent != null && parent.knowsClass(cls)) - throw new GridException("Recursive reference found in type: " + cls.getName()); + throw new IgniteCheckedException("Recursive reference found in type: " + cls.getName()); if (parent == null) { // Check class annotation at top level only. GridCacheQueryTextField txtAnnCls = cls.getAnnotation(GridCacheQueryTextField.class); @@ -738,7 +739,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { if (sqlAnn != null || txtAnn != null) { if (mtd.getParameterTypes().length != 0) - throw new GridException("Getter with GridCacheQuerySqlField " + + throw new IgniteCheckedException("Getter with GridCacheQuerySqlField " + "annotation cannot have parameters: " + mtd); ClassProperty prop = new ClassProperty(mtd); @@ -762,10 +763,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param cls Class of field or return type for method. * @param prop Current property. * @param desc Class description. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ static void processAnnotation(boolean key, GridCacheQuerySqlField sqlAnn, GridCacheQueryTextField txtAnn, - Class<?> cls, ClassProperty prop, TypeDescriptor desc) throws GridException { + Class<?> cls, ClassProperty prop, TypeDescriptor desc) throws IgniteCheckedException { if (sqlAnn != null) { processAnnotationsInClass(key, cls, desc, prop); @@ -802,10 +803,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param cls Class to process. * @param meta Type metadata. * @param d Type descriptor. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ static void processClassMeta(boolean key, Class<?> cls, GridCacheQueryTypeMetadata meta, TypeDescriptor d) - throws GridException { + throws IgniteCheckedException { for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) { ClassProperty prop = buildClassProperty(cls, entry.getKey(), entry.getValue()); @@ -875,10 +876,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param key Key or value flag. * @param meta Declared metadata. * @param d Type descriptor. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ static void processPortableMeta(boolean key, GridCacheQueryTypeMetadata meta, TypeDescriptor d) - throws GridException { + throws IgniteCheckedException { for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) { PortableProperty prop = buildPortableProperty(entry.getKey(), entry.getValue()); @@ -967,9 +968,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param pathStr String representing path to the property. May contains dots '.' to identify nested fields. * @param resType Expected result type. * @return Property instance corresponding to the given path. - * @throws GridException If property cannot be created. + * @throws IgniteCheckedException If property cannot be created. */ - static ClassProperty buildClassProperty(Class<?> cls, String pathStr, Class<?> resType) throws GridException { + static ClassProperty buildClassProperty(Class<?> cls, String pathStr, Class<?> resType) throws IgniteCheckedException { String[] path = pathStr.split("\\."); ClassProperty res = null; @@ -991,7 +992,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { tmp = new ClassProperty(cls.getDeclaredField(prop)); } catch (NoSuchFieldException ignored) { - throw new GridException("Failed to find getter method or field for property named " + + throw new IgniteCheckedException("Failed to find getter method or field for property named " + "'" + prop + "': " + cls.getName()); } } @@ -1004,7 +1005,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { } if (!U.box(resType).isAssignableFrom(U.box(res.type()))) - throw new GridException("Failed to create property for given path (actual property type is not assignable" + + throw new IgniteCheckedException("Failed to create property for given path (actual property type is not assignable" + " to declared type [path=" + pathStr + ", actualType=" + res.type().getName() + ", declaredType=" + resType.getName() + ']'); @@ -1037,13 +1038,13 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param space Space name. * @param typeName Type name. * @return Type. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public GridQueryTypeDescriptor type(@Nullable String space, String typeName) throws GridException { + public GridQueryTypeDescriptor type(@Nullable String space, String typeName) throws IgniteCheckedException { TypeDescriptor type = typesByName.get(new TypeName(space, typeName)); if (type == null || !type.registered()) - throw new GridException("Failed to find type descriptor for type name: " + typeName); + throw new IgniteCheckedException("Failed to find type descriptor for type name: " + typeName); return type; } @@ -1051,9 +1052,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** * @param cls Field type. * @return {@code True} if given type is a spatial geometry type based on {@code com.vividsolutions.jts} library. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - private static boolean isGeometryClass(Class<?> cls) throws GridException { // TODO optimize + private static boolean isGeometryClass(Class<?> cls) throws IgniteCheckedException { // TODO optimize Class<?> dataTypeCls; try { @@ -1069,7 +1070,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { return (Boolean)method.invoke(null, cls); } catch (Exception e) { - throw new GridException("Failed to invoke 'org.h2.value.DataType.isGeometryClass' method.", e); + throw new IgniteCheckedException("Failed to invoke 'org.h2.value.DataType.isGeometryClass' method.", e); } } @@ -1082,9 +1083,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { * * @param x Object with this property. * @return Property value. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public abstract Object value(Object x) throws GridException; + public abstract Object value(Object x) throws IgniteCheckedException; /** * @return Property name. @@ -1130,7 +1131,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public Object value(Object x) throws GridException { + @Override public Object value(Object x) throws IgniteCheckedException { if (parent != null) x = parent.value(x); @@ -1150,7 +1151,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { } } catch (Exception e) { - throw new GridException(e); + throw new IgniteCheckedException(e); } } @@ -1219,7 +1220,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public Object value(Object obj) throws GridException { + @Override public Object value(Object obj) throws IgniteCheckedException { if (parent != null) obj = parent.value(obj); @@ -1227,7 +1228,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { return null; if (!(obj instanceof PortableObject)) - throw new GridException("Non-portable object received as a result of property extraction " + + throw new IgniteCheckedException("Non-portable object received as a result of property extraction " + "[parent=" + parent + ", propName=" + propName + ", obj=" + obj + ']'); return ((PortableObject)obj).field(propName); @@ -1287,9 +1288,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** * @param c Initialization callable. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ - void init(Callable<Void> c) throws GridException { + void init(Callable<Void> c) throws IgniteCheckedException { initializer.init(c); } @@ -1347,14 +1348,14 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public <T> T value(Object obj, String field) throws GridException { + @Override public <T> T value(Object obj, String field) throws IgniteCheckedException { assert obj != null; assert field != null; Property prop = props.get(field); if (prop == null) - throw new GridException("Failed to find field '" + field + "' in type '" + name + "'."); + throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'."); return (T)prop.value(obj); } @@ -1370,13 +1371,13 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param idxName Index name. * @param type Index type. * @return Index descriptor. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ - public IndexDescriptor addIndex(String idxName, GridQueryIndexType type) throws GridException { + public IndexDescriptor addIndex(String idxName, GridQueryIndexType type) throws IgniteCheckedException { IndexDescriptor idx = new IndexDescriptor(type); if (indexes.put(idxName, idx) != null) - throw new GridException("Index with name '" + idxName + "' already exists."); + throw new IgniteCheckedException("Index with name '" + idxName + "' already exists."); return idx; } @@ -1388,10 +1389,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param field Field name. * @param orderNum Fields order number in index. * @param descending Sorting order. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ public void addFieldToIndex(String idxName, String field, int orderNum, - boolean descending) throws GridException { + boolean descending) throws IgniteCheckedException { IndexDescriptor desc = indexes.get(idxName); if (desc == null) @@ -1449,13 +1450,13 @@ public class GridQueryProcessor extends GridProcessorAdapter { * @param key If given property relates to key. * @param prop Property. * @param failOnDuplicate Fail on duplicate flag. - * @throws GridException In case of error. + * @throws IgniteCheckedException In case of error. */ - public void addProperty(boolean key, Property prop, boolean failOnDuplicate) throws GridException { + public void addProperty(boolean key, Property prop, boolean failOnDuplicate) throws IgniteCheckedException { String name = prop.name(); if (props.put(name, prop) != null && failOnDuplicate) - throw new GridException("Property with name '" + name + "' already exists."); + throw new IgniteCheckedException("Property with name '" + name + "' already exists."); if (key) keyFields.put(name, prop.type()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java index d835349..ae2f01a 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java @@ -9,8 +9,7 @@ package org.gridgain.grid.kernal.processors.query; - -import org.gridgain.grid.*; +import org.apache.ignite.*; import java.util.*; @@ -45,9 +44,9 @@ public interface GridQueryTypeDescriptor { * @param obj Object to get field value from. * @param field Field name. * @return Value for given field. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public <T> T value(Object obj, String field) throws GridException; + public <T> T value(Object obj, String field) throws IgniteCheckedException; /** * Gets indexes for this type. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java index 9ee4734..6c795bd 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java @@ -9,7 +9,7 @@ package org.gridgain.grid.kernal.processors.resource; -import org.gridgain.grid.*; +import org.apache.ignite.*; import org.gridgain.grid.kernal.managers.deployment.*; import org.gridgain.grid.util.typedef.internal.*; @@ -41,13 +41,13 @@ class GridResourceBasicInjector<T> implements GridResourceInjector { /** {@inheritDoc} */ @Override public void inject(GridResourceField field, Object target, Class<?> depCls, GridDeployment dep) - throws GridException { + throws IgniteCheckedException { GridResourceUtils.inject(field.getField(), target, rsrc); } /** {@inheritDoc} */ @Override public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, GridDeployment dep) - throws GridException { + throws IgniteCheckedException { GridResourceUtils.inject(mtd.getMethod(), target, rsrc); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java index 2da1ceb..dfd1884 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java @@ -256,7 +256,7 @@ class GridResourceCustomInjector implements GridResourceInjector { } } } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(log, "Failed to find finalizers for resource: " + rsrc, e); } @@ -283,14 +283,14 @@ class GridResourceCustomInjector implements GridResourceInjector { try { ioc.inject(rsrc.getResource(), annCls, nullInjector, null, null); } - catch (GridException e) { + catch (IgniteCheckedException e) { U.error(log, "Failed to clean up resource [ann=" + annCls + ", rsrc=" + rsrc + ']', e); } } /** {@inheritDoc} */ @Override public void inject(GridResourceField field, Object target, Class<?> depCls, - GridDeployment dep) throws GridException { + GridDeployment dep) throws IgniteCheckedException { assert dep != null; IgniteUserResource ann = (IgniteUserResource)field.getAnnotation(); @@ -298,7 +298,7 @@ class GridResourceCustomInjector implements GridResourceInjector { assert ann != null; if (!Modifier.isTransient(field.getField().getModifiers())) { - throw new GridException("@GridUserResource must only be used with 'transient' fields: " + + throw new IgniteCheckedException("@GridUserResource must only be used with 'transient' fields: " + field.getField()); } @@ -312,13 +312,13 @@ class GridResourceCustomInjector implements GridResourceInjector { /** {@inheritDoc} */ @Override public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, GridDeployment dep) - throws GridException { + throws IgniteCheckedException { assert dep != null; IgniteUserResource ann = (IgniteUserResource)mtd.getAnnotation(); if (mtd.getMethod().getParameterTypes().length != 1) - throw new GridException("Method injection setter must have only one parameter: " + mtd.getMethod()); + throw new IgniteCheckedException("Method injection setter must have only one parameter: " + mtd.getMethod()); Class<?> rsrcCls = !ann.resourceClass().equals(Void.class) ? ann.resourceClass() : mtd.getMethod().getParameterTypes()[0]; @@ -338,10 +338,10 @@ class GridResourceCustomInjector implements GridResourceInjector { * @param rsrcCls Resource class. * @param rsrcName Resource name. * @return Created resource. - * @throws GridException If resource creation failed. + * @throws IgniteCheckedException If resource creation failed. */ private Object getResource(GridDeployment dep, Class<?> depCls, Class<?> rsrcCls, String rsrcName) - throws GridException { + throws IgniteCheckedException { assert dep != null; // For performance reasons we first try to acquire read lock and @@ -434,9 +434,9 @@ class GridResourceCustomInjector implements GridResourceInjector { * @param dep Deployment. * @param depCls Deployed class. * @return Created object with injected resources. - * @throws GridException Thrown in case of any errors during injection. + * @throws IgniteCheckedException Thrown in case of any errors during injection. */ - private CachedResource createResource(Class<?> rsrcCls, GridDeployment dep, Class<?> depCls) throws GridException { + private CachedResource createResource(Class<?> rsrcCls, GridDeployment dep, Class<?> depCls) throws IgniteCheckedException { assert dep != null; try { @@ -466,13 +466,13 @@ class GridResourceCustomInjector implements GridResourceInjector { return new CachedResource(rsrc, dep.classLoader()); } catch (InstantiationException e) { - throw new GridException("Failed to instantiate task shared resource: " + rsrcCls, e); + throw new IgniteCheckedException("Failed to instantiate task shared resource: " + rsrcCls, e); } catch (IllegalAccessException e) { - throw new GridException("Failed to access task shared resource (is class public?): " + rsrcCls, e); + throw new IgniteCheckedException("Failed to access task shared resource (is class public?): " + rsrcCls, e); } catch (InvocationTargetException e) { - throw new GridException("Failed to initialize task shared resource: " + rsrcCls, e); + throw new IgniteCheckedException("Failed to initialize task shared resource: " + rsrcCls, e); } } @@ -482,17 +482,17 @@ class GridResourceCustomInjector implements GridResourceInjector { * @param cls Class in which search for methods. * @param annCls Annotation. * @return Set of methods with given annotations. - * @throws GridException Thrown in case when method contains parameters. + * @throws IgniteCheckedException Thrown in case when method contains parameters. */ private List<Method> getMethodsWithAnnotation(Class<?> cls, Class<? extends Annotation> annCls) - throws GridException { + throws IgniteCheckedException { List<Method> mtds = new ArrayList<>(); for (Class<?> c = cls; !c.equals(Object.class); c = c.getSuperclass()) { for (Method mtd : c.getDeclaredMethods()) { if (mtd.getAnnotation(annCls) != null) { if (mtd.getParameterTypes().length > 0) { - throw new GridException("Task shared resource initialization or finalization method should " + + throw new IgniteCheckedException("Task shared resource initialization or finalization method should " + "not have parameters: " + mtd); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java index b522be7..2a895d2 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java @@ -9,7 +9,7 @@ package org.gridgain.grid.kernal.processors.resource; -import org.gridgain.grid.*; +import org.apache.ignite.*; import org.gridgain.grid.kernal.managers.deployment.*; /** @@ -24,9 +24,9 @@ interface GridResourceInjector { * @param target Target object the field belongs to. * @param depCls Deployed class. * @param dep Deployment. - * @throws GridException If injection failed. + * @throws IgniteCheckedException If injection failed. */ - public void inject(GridResourceField field, Object target, Class<?> depCls, GridDeployment dep) throws GridException; + public void inject(GridResourceField field, Object target, Class<?> depCls, GridDeployment dep) throws IgniteCheckedException; /** * Injects resource with a setter method. Caches injected resource with the given key if needed. @@ -35,9 +35,9 @@ interface GridResourceInjector { * @param target Target object the field belongs to. * @param depCls Deployed class. * @param dep Deployment. - * @throws GridException If injection failed. + * @throws IgniteCheckedException If injection failed. */ - public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, GridDeployment dep) throws GridException; + public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, GridDeployment dep) throws IgniteCheckedException; /** * Gracefully cleans all resources associated with deployment. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java index f1a134a..ef7c508 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java @@ -9,7 +9,7 @@ package org.gridgain.grid.kernal.processors.resource; -import org.gridgain.grid.*; +import org.apache.ignite.*; import org.gridgain.grid.kernal.managers.deployment.*; import org.gridgain.grid.util.*; import org.gridgain.grid.util.typedef.*; @@ -90,10 +90,10 @@ class GridResourceIoc { * @param injector Resource to inject. * @param dep Deployment. * @param depCls Deployment class. - * @throws GridException Thrown in case of any errors during injection. + * @throws IgniteCheckedException Thrown in case of any errors during injection. */ void inject(Object target, Class<? extends Annotation> annCls, GridResourceInjector injector, - @Nullable GridDeployment dep, @Nullable Class<?> depCls) throws GridException { + @Nullable GridDeployment dep, @Nullable Class<?> depCls) throws IgniteCheckedException { assert target != null; assert annCls != null; assert injector != null; @@ -110,10 +110,10 @@ class GridResourceIoc { * @param dep Deployment. * @param depCls Deployment class. * @param checkedObjs Set of already inspected objects to avoid indefinite recursion. - * @throws GridException Thrown in case of any errors during injection. + * @throws IgniteCheckedException Thrown in case of any errors during injection. */ private void injectInternal(Object target, Class<? extends Annotation> annCls, GridResourceInjector injector, - @Nullable GridDeployment dep, @Nullable Class<?> depCls, Set<Object> checkedObjs) throws GridException { + @Nullable GridDeployment dep, @Nullable Class<?> depCls, Set<Object> checkedObjs) throws IgniteCheckedException { assert target != null; assert annCls != null; assert injector != null; @@ -149,7 +149,7 @@ class GridResourceIoc { injectInternal(obj, annCls, injector, dep, depCls, checkedObjs); } catch (IllegalAccessException e) { - throw new GridException("Failed to inject resource [field=" + f.getName() + + throw new IgniteCheckedException("Failed to inject resource [field=" + f.getName() + ", target=" + target + ']', e); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java index abc2861..bdd89d8 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java @@ -9,8 +9,8 @@ package org.gridgain.grid.kernal.processors.resource; +import org.apache.ignite.*; import org.apache.ignite.compute.*; -import org.gridgain.grid.*; import org.gridgain.grid.kernal.managers.deployment.*; /** @@ -28,7 +28,7 @@ public class GridResourceJobContextInjector extends GridResourceBasicInjector<Co /** {@inheritDoc} */ @Override public void inject(GridResourceField field, Object target, Class<?> depCls, GridDeployment dep) - throws GridException { + throws IgniteCheckedException { assert target != null; if (!(target instanceof ComputeTask)) @@ -37,7 +37,7 @@ public class GridResourceJobContextInjector extends GridResourceBasicInjector<Co /** {@inheritDoc} */ @Override public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, GridDeployment dep) - throws GridException { + throws IgniteCheckedException { assert target != null; if (!(target instanceof ComputeTask)) http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java index 7034257..b4ba0c7 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java @@ -27,13 +27,13 @@ public class GridResourceLoggerInjector extends GridResourceBasicInjector<Ignite /** {@inheritDoc} */ @Override public void inject(GridResourceField field, Object target, Class<?> depCls, GridDeployment dep) - throws GridException { + throws IgniteCheckedException { GridResourceUtils.inject(field.getField(), target, resource((IgniteLoggerResource)field.getAnnotation(), target)); } /** {@inheritDoc} */ @Override public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, GridDeployment dep) - throws GridException { + throws IgniteCheckedException { GridResourceUtils.inject(mtd.getMethod(), target, resource((IgniteLoggerResource)mtd.getAnnotation(), target)); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java index 89548f1..be23993 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java @@ -145,7 +145,7 @@ public class GridResourceProcessor extends GridProcessorAdapter { } /** {@inheritDoc} */ - @Override public void start() throws GridException { + @Override public void start() throws IgniteCheckedException { customInjector = new GridResourceCustomInjector(log, ioc); customInjector.setExecutorInjector(execInjector); @@ -199,10 +199,10 @@ public class GridResourceProcessor extends GridProcessorAdapter { * @param dep Deployment. * @param target Target object. * @param annCls Annotation class. - * @throws GridException If failed to execute annotated methods. + * @throws IgniteCheckedException If failed to execute annotated methods. */ public void invokeAnnotated(GridDeployment dep, Object target, Class<? extends Annotation> annCls) - throws GridException { + throws IgniteCheckedException { if (target != null) { Collection<Method> mtds = getMethodsWithAnnotation(dep, target.getClass(), annCls); @@ -214,7 +214,7 @@ public class GridResourceProcessor extends GridProcessorAdapter { mtd.invoke(target); } catch (IllegalArgumentException | InvocationTargetException | IllegalAccessException e) { - throw new GridException("Failed to invoke annotated method [job=" + target + ", mtd=" + mtd + + throw new IgniteCheckedException("Failed to invoke annotated method [job=" + target + ", mtd=" + mtd + ", ann=" + annCls + ']', e); } } @@ -228,9 +228,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * @param dep Deployment. * @param depCls Deployed class. * @param target Target instance to inject into. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ - public void inject(GridDeployment dep, Class<?> depCls, Object target) throws GridException { + public void inject(GridDeployment dep, Class<?> depCls, Object target) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Injecting resources: " + target); @@ -259,9 +259,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * * @param obj Object. * @param cacheName Cache name to inject. - * @throws GridException If failed to inject. + * @throws IgniteCheckedException If failed to inject. */ - public void injectCacheName(Object obj, String cacheName) throws GridException { + public void injectCacheName(Object obj, String cacheName) throws IgniteCheckedException { assert obj != null; if (log.isDebugEnabled()) @@ -275,9 +275,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { /** * @param obj Object to inject. - * @throws GridException If failed to inject. + * @throws IgniteCheckedException If failed to inject. */ - public void injectGeneric(Object obj) throws GridException { + public void injectGeneric(Object obj) throws IgniteCheckedException { assert obj != null; if (log.isDebugEnabled()) @@ -303,9 +303,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { /** * @param obj Object. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void cleanupGeneric(Object obj) throws GridException { + public void cleanupGeneric(Object obj) throws IgniteCheckedException { if (obj != null) { if (log.isDebugEnabled()) log.debug("Cleaning up resources: " + obj); @@ -337,10 +337,10 @@ public class GridResourceProcessor extends GridProcessorAdapter { * @param job Grid job to inject resources to. * @param ses Current task session. * @param jobCtx Job context. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ public void inject(GridDeployment dep, Class<?> taskCls, ComputeJob job, ComputeTaskSession ses, - GridJobContextImpl jobCtx) throws GridException { + GridJobContextImpl jobCtx) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Injecting resources: " + job); @@ -365,10 +365,10 @@ public class GridResourceProcessor extends GridProcessorAdapter { * @param job Job. * @param ses Session. * @param jobCtx Job context. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ private void injectToJob(GridDeployment dep, Class<?> taskCls, Object job, ComputeTaskSession ses, - GridJobContextImpl jobCtx) throws GridException { + GridJobContextImpl jobCtx) throws IgniteCheckedException { Class<? extends Annotation>[] filtered = ioc.filter(dep, job, JOB_INJECTIONS); if (filtered.length > 0) { @@ -432,10 +432,10 @@ public class GridResourceProcessor extends GridProcessorAdapter { * @param ses Grid task session. * @param balancer Load balancer. * @param mapper Continuous task mapper. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ public void inject(GridDeployment dep, ComputeTask<?, ?> task, GridTaskSessionImpl ses, - ComputeLoadBalancer balancer, ComputeTaskContinuousMapper mapper) throws GridException { + ComputeLoadBalancer balancer, ComputeTaskContinuousMapper mapper) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Injecting resources: " + task); @@ -504,9 +504,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * Injects held resources into given SPI implementation. * * @param spi SPI implementation. - * @throws GridException Throw in case of any errors. + * @throws IgniteCheckedException Throw in case of any errors. */ - public void inject(IgniteSpi spi) throws GridException { + public void inject(IgniteSpi spi) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Injecting resources: " + spi); @@ -533,9 +533,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * method injects {@code null}s into SPI implementation. * * @param spi SPI implementation. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ - public void cleanup(IgniteSpi spi) throws GridException { + public void cleanup(IgniteSpi spi) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Cleaning up resources: " + spi); @@ -560,9 +560,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * Injects held resources into given lifecycle bean. * * @param lifecycleBean Lifecycle bean. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ - public void inject(LifecycleBean lifecycleBean) throws GridException { + public void inject(LifecycleBean lifecycleBean) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Injecting resources: " + lifecycleBean); @@ -589,9 +589,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * method injects {@code null}s into lifecycle bean. * * @param lifecycleBean Lifecycle bean. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ - public void cleanup(LifecycleBean lifecycleBean) throws GridException { + public void cleanup(LifecycleBean lifecycleBean) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Cleaning up resources: " + lifecycleBean); @@ -617,9 +617,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * Injects resources into service. * * @param svc Service to inject. - * @throws GridException If failed. + * @throws IgniteCheckedException If failed. */ - public void inject(ManagedService svc) throws GridException { + public void inject(ManagedService svc) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Injecting resources: " + svc); @@ -646,9 +646,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * method injects {@code null}s into service bean. * * @param svc Service. - * @throws GridException Thrown in case of any errors. + * @throws IgniteCheckedException Thrown in case of any errors. */ - public void cleanup(ManagedService svc) throws GridException { + public void cleanup(ManagedService svc) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Cleaning up resources: " + svc); @@ -680,10 +680,10 @@ public class GridResourceProcessor extends GridProcessorAdapter { * @param rsrc Resource to inject. * @param dep Deployment. * @param depCls Deployed class. - * @throws GridException If injection failed. + * @throws IgniteCheckedException If injection failed. */ public void injectBasicResource(Object target, Class<? extends Annotation> annCls, Object rsrc, - GridDeployment dep, Class<?> depCls) throws GridException { + GridDeployment dep, Class<?> depCls) throws IgniteCheckedException { // Safety. assert !(rsrc instanceof GridResourceInjector) : "Invalid injection."; @@ -699,10 +699,10 @@ public class GridResourceProcessor extends GridProcessorAdapter { * @param target Target object. * @param annCls Setter annotation. * @param rsrc Resource to inject. - * @throws GridException If injection failed. + * @throws IgniteCheckedException If injection failed. */ public void injectBasicResource(Object target, Class<? extends Annotation> annCls, Object rsrc) - throws GridException { + throws IgniteCheckedException { // Safety. assert !(rsrc instanceof GridResourceInjector) : "Invalid injection."; @@ -762,9 +762,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { * * @param target Target object. * @return Original object wrapped by proxy. - * @throws GridException If unwrap failed. + * @throws IgniteCheckedException If unwrap failed. */ - private Object unwrapTarget(Object target) throws GridException { + private Object unwrapTarget(Object target) throws IgniteCheckedException { return rsrcCtx != null ? rsrcCtx.unwrapTarget(target) : target; }