You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/05/13 11:46:50 UTC

[1/6] incubator-ignite git commit: #ignite-373: Revert TopologyVersionAwareJob.localExecute semantic.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-373 83550f8e0 -> b8b8cf6d8


#ignite-373: Revert TopologyVersionAwareJob.localExecute semantic.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/3dd16ba3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/3dd16ba3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/3dd16ba3

Branch: refs/heads/ignite-373
Commit: 3dd16ba345bdaeb47eb9bf65481503619f234085
Parents: 83550f8
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 12:09:23 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 12:09:23 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java       | 19 ++++++++-----------
 .../distributed/GridDistributedCacheAdapter.java |  2 +-
 2 files changed, 9 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3dd16ba3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 0b7fa91..3826bfa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -4842,9 +4842,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public Object localExecute() {
-            IgniteInternalCache cache = ((IgniteKernal)ignite).context().cache().cache(cacheName);
-
+        @Nullable @Override public Object localExecute(@Nullable IgniteInternalCache cache) {
             if (cache != null)
                 cache.clearLocally();
 
@@ -4882,9 +4880,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public Object localExecute() {
-            IgniteInternalCache cache = ((IgniteKernal)ignite).context().cache().cache(cacheName);
-
+        @Nullable @Override public Object localExecute(@Nullable IgniteInternalCache cache) {
             if (cache != null)
                 cache.clearLocallyAll(keys);
 
@@ -4922,9 +4918,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public Object localExecute() {
-            IgniteInternalCache cache = ((IgniteKernal)ignite).context().cache().cache(cacheName);
-
+        @Nullable @Override public Object localExecute(@Nullable IgniteInternalCache cache) {
             if (cache == null)
                 return 0;
 
@@ -5547,13 +5541,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         @Nullable @Override public final Object execute() {
             waitAffinityReadyFuture();
 
-            return localExecute();
+            IgniteInternalCache cache = ((IgniteKernal)ignite).context().cache().cache(cacheName);
+
+            return localExecute(cache);
         }
 
         /**
+         * @param cache Cache.
          * @return Local execution result.
          */
-        @Nullable protected abstract Object localExecute();
+        @Nullable protected abstract Object localExecute(@Nullable IgniteInternalCache cache);
 
         /**
          * Holds (suspends) job execution until our cache version becomes equal to remote cache's version.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3dd16ba3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 941a775..3878c67 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -316,7 +316,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public Object localExecute() {
+        @Nullable @Override public Object localExecute(@Nullable IgniteInternalCache cache0) {
             GridCacheAdapter cache = ((IgniteKernal) ignite).context().cache().internalCache(cacheName);
 
             if (cache == null)


[6/6] incubator-ignite git commit: #ignite-373: Remote jobs clear, size, removeAll should rethrow all exceptions except ClusterTopologyException.

Posted by sb...@apache.org.
#ignite-373: Remote jobs clear, size, removeAll  should rethrow all exceptions except ClusterTopologyException.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/b8b8cf6d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/b8b8cf6d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/b8b8cf6d

Branch: refs/heads/ignite-373
Commit: b8b8cf6d8b599e7719b0d6571c6c103f43d37e86
Parents: 525d42a
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 12:44:53 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 12:44:53 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java        | 18 ++++++++++++++++++
 .../distributed/GridDistributedCacheAdapter.java  |  9 +++++++++
 2 files changed, 27 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b8b8cf6d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 624c528..4106cb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -5591,6 +5591,15 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /** {@inheritDoc} */
         @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
+            IgniteException e = res.getException();
+
+            if (e != null) {
+                if (e instanceof ClusterTopologyException)
+                    return ComputeJobResultPolicy.WAIT;
+
+                throw new IgniteException("Remote job threw exception.", e);
+            }
+
             return ComputeJobResultPolicy.WAIT;
         }
 
@@ -5650,6 +5659,15 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /** {@inheritDoc} */
         @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
+            IgniteException e = res.getException();
+
+            if (e != null) {
+                if (e instanceof ClusterTopologyException)
+                    return ComputeJobResultPolicy.WAIT;
+
+                throw new IgniteException("Remote job threw exception.", e);
+            }
+
             return ComputeJobResultPolicy.WAIT;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b8b8cf6d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index b7ac198..56f8e1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -281,6 +281,15 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
 
         /** {@inheritDoc} */
         @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
+            IgniteException e = res.getException();
+
+            if (e != null) {
+                if (e instanceof ClusterTopologyException)
+                    return ComputeJobResultPolicy.WAIT;
+
+                throw new IgniteException("Remote job threw exception.", e);
+            }
+
             return ComputeJobResultPolicy.WAIT;
         }
 


[2/6] incubator-ignite git commit: #ignite-373: Change flag removeAll to retry in DistributedCacheAdapter.removeAll

Posted by sb...@apache.org.
#ignite-373: Change flag removeAll to retry in DistributedCacheAdapter.removeAll


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/48fdafac
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/48fdafac
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/48fdafac

Branch: refs/heads/ignite-373
Commit: 48fdafac9aac6b47c091d587da0f74f7fad8b015
Parents: 3dd16ba
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 12:12:16 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 12:12:16 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/GridDistributedCacheAdapter.java  | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/48fdafac/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 3878c67..6f939e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -143,10 +143,10 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         try {
             AffinityTopologyVersion topVer;
 
-            Boolean rmvAll;
+            boolean retry = false;
 
             do {
-                rmvAll = true;
+                retry = false;
 
                 topVer = ctx.affinity().affinityTopologyVersion();
 
@@ -156,10 +156,10 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
                 if (!nodes.isEmpty()) {
                     ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
-                    rmvAll = ctx.kernalContext().task().execute(new RemoveAllTask(ctx), null).get();
+                    retry = !ctx.kernalContext().task().execute(new RemoveAllTask(ctx), null).get();
                 }
             }
-            while (ctx.affinity().affinityTopologyVersion().compareTo(topVer) != 0 || rmvAll == null || !rmvAll);
+            while (ctx.affinity().affinityTopologyVersion().compareTo(topVer) != 0 || retry);
         }
         catch (ClusterGroupEmptyCheckedException ignore) {
             if (log.isDebugEnabled())
@@ -193,11 +193,11 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
             rmvAll.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
                 @Override public void apply(IgniteInternalFuture<Boolean> fut) {
                     try {
-                        Boolean res = fut.get();
+                        boolean retry = !fut.get();
 
                         AffinityTopologyVersion topVer0 = ctx.affinity().affinityTopologyVersion();
 
-                        if (topVer0.equals(topVer) && res != null && res)
+                        if (topVer0.equals(topVer) && !retry)
                             opFut.onDone();
                         else
                             removeAllAsync(opFut, topVer0);


[3/6] incubator-ignite git commit: #ignite-373: Use one topology version for all nodes in clear(), removeAll() and size().

Posted by sb...@apache.org.
#ignite-373: Use one topology version for all nodes in clear(), removeAll() and size().


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/f670f400
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/f670f400
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/f670f400

Branch: refs/heads/ignite-373
Commit: f670f4001517754eed1e22f7ef333fb1a21ce93a
Parents: 48fdafa
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 12:31:44 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 12:31:44 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      | 47 +++++++++++------
 .../GridDistributedCacheAdapter.java            | 54 +++++++++++++-------
 2 files changed, 65 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f670f400/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 3826bfa..76ad7a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1133,7 +1133,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
             ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
-            ctx.kernalContext().task().execute(new ClearTask(ctx, keys), null).get();
+            ctx.kernalContext().task().execute(
+                new ClearTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), keys), null).get();
         }
     }
 
@@ -1152,7 +1153,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (!nodes.isEmpty()) {
             ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
-            return ctx.kernalContext().task().execute(new ClearTask(ctx, keys), null);
+            return ctx.kernalContext().task().execute(
+                new ClearTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), keys), null);
         }
         else
             return new GridFinishedFuture<>();
@@ -3571,7 +3573,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
-        return ctx.kernalContext().task().execute(new SizeTask(ctx, peekModes), null);
+        return ctx.kernalContext().task().execute(
+            new SizeTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes), null);
     }
 
     /** {@inheritDoc} */
@@ -5583,8 +5586,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         /** */
         private static final long serialVersionUID = 0L;
 
-        /** Cache context. */
-        private GridCacheContext ctx;
+        /** Cache name. */
+        private String cacheName;
+
+        /** Affinity topology version. */
+        private AffinityTopologyVersion topVer;
 
         /** Peek modes. */
         private CachePeekMode[] peekModes;
@@ -5597,10 +5603,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /**
-         * @param ctx Cache context.
+         * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
+         * @param peekModes Cache peek modes.
          */
-        public SizeTask(GridCacheContext ctx, CachePeekMode[] peekModes) {
-            this.ctx = ctx;
+        public SizeTask(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes) {
+            this.cacheName = cacheName;
+            this.topVer = topVer;
             this.peekModes = peekModes;
         }
 
@@ -5610,7 +5619,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             Map<ComputeJob, ClusterNode> jobs = new HashMap();
 
             for (ClusterNode node : subgrid)
-                jobs.put(new SizeJob(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes), node);
+                jobs.put(new SizeJob(cacheName, topVer, peekModes), node);
 
             return jobs;
         }
@@ -5640,8 +5649,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         /** */
         private static final long serialVersionUID = 0L;
 
-        /** Cache context. */
-        private GridCacheContext ctx;
+        /** Cache name. */
+        private String cacheName;
+
+        /** Affinity topology version. */
+        private AffinityTopologyVersion topVer;
 
         /** Keys to clear. */
         private Set<? extends K> keys;
@@ -5654,11 +5666,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /**
-         * @param ctx Cache context.
+         * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
          * @param keys Keys to clear.
          */
-        public ClearTask(GridCacheContext ctx, Set<? extends K> keys) {
-            this.ctx = ctx;
+        public ClearTask(String cacheName, AffinityTopologyVersion topVer, Set<? extends K> keys) {
+            this.cacheName = cacheName;
+            this.topVer = topVer;
             this.keys = keys;
         }
 
@@ -5668,9 +5682,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             Map<ComputeJob, ClusterNode> jobs = new HashMap();
 
             for (ClusterNode node : subgrid) {
-                jobs.put(keys == null ?
-                        new GlobalClearAllJob(ctx.name(), ctx.affinity().affinityTopologyVersion()) :
-                        new GlobalClearKeySetJob<K>(ctx.name(), ctx.affinity().affinityTopologyVersion(), keys),
+                jobs.put(keys == null ? new GlobalClearAllJob(cacheName, topVer) :
+                        new GlobalClearKeySetJob<K>(cacheName, topVer, keys),
                     node);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f670f400/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 6f939e1..c172a87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -143,7 +143,11 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         try {
             AffinityTopologyVersion topVer;
 
-            boolean retry = false;
+            boolean retry;
+
+            CacheOperationContext opCtx = ctx.operationContextPerCall();
+
+            boolean skipStore = opCtx != null && opCtx.skipStore();
 
             do {
                 retry = false;
@@ -156,7 +160,8 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
                 if (!nodes.isEmpty()) {
                     ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
-                    retry = !ctx.kernalContext().task().execute(new RemoveAllTask(ctx), null).get();
+                    retry = !ctx.kernalContext().task().execute(
+                        new RemoveAllTask(ctx.name(), topVer, skipStore), null).get();
                 }
             }
             while (ctx.affinity().affinityTopologyVersion().compareTo(topVer) != 0 || retry);
@@ -173,7 +178,11 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
 
         AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
 
-        removeAllAsync(opFut, topVer);
+        CacheOperationContext opCtx = ctx.operationContextPerCall();
+
+        boolean skipStore = opCtx != null && opCtx.skipStore();
+
+        removeAllAsync(opFut, topVer, skipStore);
 
         return opFut;
     }
@@ -181,14 +190,17 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
     /**
      * @param opFut Future.
      * @param topVer Topology version.
+     * @param skipStore Skip store flag.
      */
-    private void removeAllAsync(final GridFutureAdapter<Void> opFut, final AffinityTopologyVersion topVer) {
+    private void removeAllAsync(final GridFutureAdapter<Void> opFut, final AffinityTopologyVersion topVer,
+        final boolean skipStore) {
         Collection<ClusterNode> nodes = ctx.grid().cluster().forDataNodes(name()).nodes();
 
         if (!nodes.isEmpty()) {
             ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
-            IgniteInternalFuture<Boolean> rmvAll = ctx.kernalContext().task().execute(new RemoveAllTask(ctx), null);
+            IgniteInternalFuture<Boolean> rmvAll = ctx.kernalContext().task().execute(
+                new RemoveAllTask(ctx.name(), topVer, skipStore), null);
 
             rmvAll.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
                 @Override public void apply(IgniteInternalFuture<Boolean> fut) {
@@ -200,7 +212,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
                         if (topVer0.equals(topVer) && !retry)
                             opFut.onDone();
                         else
-                            removeAllAsync(opFut, topVer0);
+                            removeAllAsync(opFut, topVer0, skipStore);
                     }
                     catch (ClusterGroupEmptyCheckedException ignore) {
                         if (log.isDebugEnabled())
@@ -236,8 +248,14 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         /** */
         private static final long serialVersionUID = 0L;
 
-        /** Cache context. */
-        private GridCacheContext ctx;
+        /** Cache name. */
+        private String cacheName;
+
+        /** Affinity topology version. */
+        private AffinityTopologyVersion topVer;
+
+        /** Skip store flag. */
+        private boolean skipStore;
 
         /**
          * Empty constructor for serialization.
@@ -247,10 +265,14 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         }
 
         /**
-         * @param ctx Cache context.
+         * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
+         * @param skipStore Skip store flag.
          */
-        public RemoveAllTask(GridCacheContext ctx) {
-            this.ctx = ctx;
+        public RemoveAllTask(String cacheName, AffinityTopologyVersion topVer, boolean skipStore) {
+            this.cacheName = cacheName;
+            this.topVer = topVer;
+            this.skipStore = skipStore;
         }
 
         /** {@inheritDoc} */
@@ -258,14 +280,8 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
             @Nullable Object arg) throws IgniteException {
             Map<ComputeJob, ClusterNode> jobs = new HashMap();
 
-            CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-            boolean skipStore = opCtx != null && opCtx.skipStore();
-
-            for (ClusterNode node : subgrid) {
-                jobs.put(new GlobalRemoveAllJob(ctx.name(), ctx.affinity().affinityTopologyVersion(), skipStore),
-                    node);
-            }
+            for (ClusterNode node : subgrid)
+                jobs.put(new GlobalRemoveAllJob(cacheName, topVer, skipStore), node);
 
             return jobs;
         }


[4/6] incubator-ignite git commit: #ignite-373: Serializable does not require empty constructor.

Posted by sb...@apache.org.
#ignite-373: Serializable does not require empty constructor.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/a0161d90
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a0161d90
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a0161d90

Branch: refs/heads/ignite-373
Commit: a0161d9037a8cedf46df7c55238188b8226e3cdc
Parents: f670f40
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 12:40:16 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 12:40:16 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      | 62 ++++----------------
 .../GridDistributedCacheAdapter.java            | 22 ++-----
 2 files changed, 14 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a0161d90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 76ad7a1..624c528 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -4830,13 +4830,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         private static final long serialVersionUID = 0L;
 
         /**
-         * Empty constructor for serialization.
-         */
-        public GlobalClearAllJob() {
-            // No-op.
-        }
-
-        /**
          * @param cacheName Cache name.
          * @param topVer Affinity topology version.
          */
@@ -4862,14 +4855,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         private static final long serialVersionUID = 0L;
 
         /** Keys to remove. */
-        private Set<? extends K> keys;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public GlobalClearKeySetJob() {
-            // No-op.
-        }
+        private final Set<? extends K> keys;
 
         /**
          * @param cacheName Cache name.
@@ -4900,14 +4886,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         private static final long serialVersionUID = 0L;
 
         /** Peek modes. */
-        private CachePeekMode[] peekModes;
-
-        /**
-         * Required by {@link Externalizable}.
-         */
-        public SizeJob() {
-            // No-op.
-        }
+        private final CachePeekMode[] peekModes;
 
         /**
          * @param cacheName Cache name.
@@ -5517,17 +5496,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         protected Ignite ignite;
 
         /** Affinity topology version. */
-        protected AffinityTopologyVersion topVer;
+        protected final AffinityTopologyVersion topVer;
 
         /** Cache name. */
-        protected String cacheName;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public TopologyVersionAwareJob() {
-            // No-op.
-        }
+        protected final String cacheName;
 
         /**
          * @param cacheName Cache name.
@@ -5587,20 +5559,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         private static final long serialVersionUID = 0L;
 
         /** Cache name. */
-        private String cacheName;
+        private final String cacheName;
 
         /** Affinity topology version. */
-        private AffinityTopologyVersion topVer;
+        private final AffinityTopologyVersion topVer;
 
         /** Peek modes. */
-        private CachePeekMode[] peekModes;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public SizeTask() {
-            // No-op.
-        }
+        private final CachePeekMode[] peekModes;
 
         /**
          * @param cacheName Cache name.
@@ -5650,20 +5615,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         private static final long serialVersionUID = 0L;
 
         /** Cache name. */
-        private String cacheName;
+        private final String cacheName;
 
         /** Affinity topology version. */
-        private AffinityTopologyVersion topVer;
+        private final AffinityTopologyVersion topVer;
 
         /** Keys to clear. */
-        private Set<? extends K> keys;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public ClearTask() {
-            // No-op.
-        }
+        private final Set<? extends K> keys;
 
         /**
          * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a0161d90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index c172a87..385c95e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -249,20 +249,13 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         private static final long serialVersionUID = 0L;
 
         /** Cache name. */
-        private String cacheName;
+        private final String cacheName;
 
         /** Affinity topology version. */
-        private AffinityTopologyVersion topVer;
+        private final AffinityTopologyVersion topVer;
 
         /** Skip store flag. */
-        private boolean skipStore;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public RemoveAllTask(){
-            // No-op.
-        }
+        private final boolean skipStore;
 
         /**
          * @param cacheName Cache name.
@@ -311,14 +304,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         private static final long serialVersionUID = 0L;
 
         /** Skip store flag. */
-        private boolean skipStore;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public GlobalRemoveAllJob() {
-            // No-op.
-        }
+        private final boolean skipStore;
 
         /**
          * @param cacheName Cache name.


[5/6] incubator-ignite git commit: #ignite-373: If cache is not configured on node return true in removeAll.

Posted by sb...@apache.org.
#ignite-373: If cache is not configured on node return true in removeAll.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/525d42a9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/525d42a9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/525d42a9

Branch: refs/heads/ignite-373
Commit: 525d42a95c6735fa6595471868f1228c9e883c9e
Parents: a0161d9
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 12:41:12 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 12:41:12 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/distributed/GridDistributedCacheAdapter.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/525d42a9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 385c95e..b7ac198 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -322,7 +322,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
             GridCacheAdapter cache = ((IgniteKernal) ignite).context().cache().internalCache(cacheName);
 
             if (cache == null)
-                return false;
+                return true;
 
             final GridCacheContext<K, V> ctx = cache.context();