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/09/02 13:03:31 UTC

[1/7] ignite git commit: ignite-1334 Fixed concurrent destroyCache/node stop. Check initFut result in GridDhtPartitionsExchangeFuture.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.4 ba756cdd6 -> ba4a582ce


ignite-1334 Fixed concurrent destroyCache/node stop. Check initFut result in GridDhtPartitionsExchangeFuture.


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

Branch: refs/heads/ignite-1.4
Commit: a61cf24b741daa679a66fc904960b74005d3e442
Parents: 612a6b8
Author: sboikov <sb...@gridgain.com>
Authored: Mon Aug 31 03:13:52 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Mon Aug 31 13:01:42 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 25 ++++--
 .../GridDhtPartitionsExchangeFuture.java        | 92 +++++++++++--------
 .../cache/IgniteDynamicCacheAndNodeStop.java    | 95 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |  2 +
 4 files changed, 173 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 8e2b20e..f13af23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -91,6 +91,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** */
     private final Map<String, GridCacheAdapter<?, ?>> caches;
 
+    /** Caches stopped from onKernalStop callback. */
+    private final Map<String, GridCacheAdapter> stoppedCaches = new ConcurrentHashMap<>();
+
     /** Map of proxies. */
     private final Map<String, IgniteCacheProxy<?, ?>> jCacheProxies;
 
@@ -854,14 +857,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = stoppedCaches.remove(maskNull(cacheName));
 
             if (cache != null)
                 stopCache(cache, cancel);
         }
 
-        for (GridCacheAdapter<?, ?> cache : caches.values())
-            stopCache(cache, cancel);
+        for (GridCacheAdapter<?, ?> cache : stoppedCaches.values()) {
+            if (cache == stoppedCaches.remove(maskNull(cache.name())))
+                stopCache(cache, cancel);
+        }
 
         List<? extends GridCacheSharedManager<?, ?>> mgrs = sharedCtx.managers();
 
@@ -893,15 +898,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cacheStartedLatch.countDown();
 
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.get(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
+
+            if (cache != null) {
+                stoppedCaches.put(maskNull(cacheName), cache);
 
-            if (cache != null)
                 onKernalStop(cache, cancel);
+            }
         }
 
         for (Map.Entry<String, GridCacheAdapter<?, ?>> entry : caches.entrySet()) {
-            if (!stopSeq.contains(entry.getKey()))
+            GridCacheAdapter<?, ?> cache = entry.getValue();
+
+            if (cache == caches.remove(entry.getKey())) {
+                stoppedCaches.put(entry.getKey(), cache);
+
                 onKernalStop(entry.getValue(), cancel);
+            }
         }
 
         List<? extends GridCacheSharedManager<?, ?>> sharedMgrs = sharedCtx.managers();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 4971ca6..b70fbc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1136,52 +1136,54 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         }
         else {
             initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-                @Override public void apply(IgniteInternalFuture<Boolean> t) {
+                @Override public void apply(IgniteInternalFuture<Boolean> f) {
                     try {
-                        if (!t.get()) // Just to check if there was an error.
+                        if (!f.get())
                             return;
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to initialize exchange future: " + this, e);
 
-                        ClusterNode loc = cctx.localNode();
+                        return;
+                    }
 
-                        singleMsgs.put(nodeId, msg);
+                    ClusterNode loc = cctx.localNode();
 
-                        boolean match = true;
+                    singleMsgs.put(nodeId, msg);
 
-                        // Check if oldest node has changed.
-                        if (!oldestNode.get().equals(loc)) {
-                            match = false;
+                    boolean match = true;
 
-                            synchronized (mux) {
-                                // Double check.
-                                if (oldestNode.get().equals(loc))
-                                    match = true;
-                            }
+                    // Check if oldest node has changed.
+                    if (!oldestNode.get().equals(loc)) {
+                        match = false;
+
+                        synchronized (mux) {
+                            // Double check.
+                            if (oldestNode.get().equals(loc))
+                                match = true;
                         }
+                    }
 
-                        if (match) {
-                            boolean allReceived;
+                    if (match) {
+                        boolean allReceived;
 
-                            synchronized (rcvdIds) {
-                                if (rcvdIds.add(nodeId))
-                                    updatePartitionSingleMap(msg);
+                        synchronized (rcvdIds) {
+                            if (rcvdIds.add(nodeId))
+                                updatePartitionSingleMap(msg);
 
-                                allReceived = allReceived();
-                            }
+                            allReceived = allReceived();
+                        }
 
-                            // If got all replies, and initialization finished, and reply has not been sent yet.
-                            if (allReceived && ready.get() && replied.compareAndSet(false, true)) {
-                                spreadPartitions();
+                        // If got all replies, and initialization finished, and reply has not been sent yet.
+                        if (allReceived && ready.get() && replied.compareAndSet(false, true)) {
+                            spreadPartitions();
 
-                                onDone(exchId.topologyVersion());
-                            }
-                            else if (log.isDebugEnabled())
-                                log.debug("Exchange future full map is not sent [allReceived=" + allReceived() +
-                                    ", ready=" + ready + ", replied=" + replied.get() + ", init=" + init.get() +
-                                    ", fut=" + this + ']');
+                            onDone(exchId.topologyVersion());
                         }
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to initialize exchange future: " + this, e);
+                        else if (log.isDebugEnabled())
+                            log.debug("Exchange future full map is not sent [allReceived=" + allReceived() +
+                                ", ready=" + ready + ", replied=" + replied.get() + ", init=" + init.get() +
+                                ", fut=" + this + ']');
                     }
                 }
             });
@@ -1244,7 +1246,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         assert exchId.topologyVersion().equals(msg.topologyVersion());
 
         initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-            @Override public void apply(IgniteInternalFuture<Boolean> t) {
+            @Override public void apply(IgniteInternalFuture<Boolean> f) {
+                try {
+                    if (!f.get())
+                        return;
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to initialize exchange future: " + this, e);
+
+                    return;
+                }
+
                 ClusterNode curOldest = oldestNode.get();
 
                 if (!nodeId.equals(curOldest.id())) {
@@ -1333,8 +1345,18 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
         try {
             // Wait for initialization part of this future to complete.
-            initFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> f) {
+            initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+                @Override public void apply(IgniteInternalFuture<Boolean> f) {
+                    try {
+                        if (!f.get())
+                            return;
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to initialize exchange future: " + this, e);
+
+                        return;
+                    }
+
                     if (isDone())
                         return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
new file mode 100644
index 0000000..a389e1f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheAndNodeStop extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheAndNodeStop() throws Exception {
+        final Ignite ignite = startGrid(0);
+
+        for (int i = 0; i < 3; i++) {
+            log.info("Iteration: " + i);
+
+            startGrid(1);
+
+            final  CacheConfiguration ccfg = new CacheConfiguration();
+
+            ignite.createCache(ccfg);
+
+            final CyclicBarrier barrier = new CyclicBarrier(2);
+
+            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    barrier.await();
+
+                    ignite.destroyCache(null);
+
+                    return null;
+                }
+            });
+
+            IgniteInternalFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    barrier.await();
+
+                    stopGrid(1);
+
+                    return null;
+                }
+            });
+
+            fut1.get();
+            fut2.get();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a61cf24b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 495719f..4258733 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -143,6 +143,8 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteCacheEntryProcessorNodeJoinTest.class));
         suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
         suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
+        suite.addTest(new TestSuite(CrossCacheTxRandomOperationsTest.class));
+        suite.addTest(new TestSuite(IgniteDynamicCacheAndNodeStop.class));
 
         return suite;
     }


[6/7] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4


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

Branch: refs/heads/ignite-1.4
Commit: 7809d262931bf6231b6a96ec2bf247a05cb4cecf
Parents: ba756cd 81e0c40
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 13:59:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 13:59:44 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  52 ++++--
 .../cache/DynamicCacheChangeRequest.java        |  19 +-
 .../cache/DynamicCacheDescriptor.java           |  16 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    | 187 +++++++++----------
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   4 +-
 6 files changed, 142 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index c5f8168,687c987..57c1977
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@@ -1837,22 -1673,16 +1802,20 @@@ public class GridCacheProcessor extend
          Collection<DynamicCacheChangeRequest> reqs =
              new ArrayList<>(registeredCaches.size() + registeredTemplates.size());
  
 -        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
 +        boolean reconnect = ctx.localNodeId().equals(nodeId) && cachesOnDisconnect != null;
 +
 +        Map<String, DynamicCacheDescriptor> descs = reconnect ? cachesOnDisconnect : registeredCaches;
 +
 +        for (DynamicCacheDescriptor desc : descs.values()) {
-             if (!desc.cancelled()) {
-                 DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+             DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
  
-                 req.startCacheConfiguration(desc.cacheConfiguration());
+             req.startCacheConfiguration(desc.cacheConfiguration());
  
-                 req.cacheType(desc.cacheType());
+             req.cacheType(desc.cacheType());
  
-                 req.deploymentId(desc.deploymentId());
+             req.deploymentId(desc.deploymentId());
  
-                 reqs.add(req);
-             }
+             reqs.add(req);
          }
  
          for (DynamicCacheDescriptor desc : registeredTemplates.values()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7809d262/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------


[5/7] ignite git commit: Merge branch 'ignite-993' into ignite-1.3.3-p7

Posted by sb...@apache.org.
Merge branch 'ignite-993' into ignite-1.3.3-p7


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

Branch: refs/heads/ignite-1.4
Commit: 81e0c401fb5d7a1731621d60e60d5ac4b03435d9
Parents: dadf4bd 3321645
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Sep 1 14:44:47 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Sep 1 14:44:47 2015 -0700

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  50 +++--
 .../cache/DynamicCacheChangeRequest.java        |  17 ++
 .../cache/DynamicCacheDescriptor.java           |  14 --
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../processors/cache/GridCacheProcessor.java    | 187 +++++++++----------
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   2 -
 6 files changed, 137 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/81e0c401/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------


[7/7] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1.3.3-p7' into ignite-1.4


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

Branch: refs/heads/ignite-1.4
Commit: ba4a582ce19c548475e0f87f5bdedab0d6f4c66c
Parents: 7809d26
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 2 14:01:21 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 2 14:01:21 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ba4a582c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 57c1977..cf8c8d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1871,7 +1871,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     if (!sysCache) {
                         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
 
-                        if (desc != null && !desc.cancelled() && desc.deploymentId().equals(req.deploymentId())) {
+                        if (desc != null && desc.deploymentId().equals(req.deploymentId())) {
                             Map<UUID, Boolean> nodes = batch.clientNodes().get(name);
 
                             assert nodes != null : req;


[2/7] ignite git commit: 1.3.3-p7-SNAPSHOT

Posted by sb...@apache.org.
1.3.3-p7-SNAPSHOT


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

Branch: refs/heads/ignite-1.4
Commit: c1b537fa5b8d29048745b8aa59c52a583e9e4931
Parents: a61cf24
Author: Ignite Teamcity <ig...@apache.org>
Authored: Mon Aug 31 23:08:47 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Mon Aug 31 23:08:47 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                   | 2 +-
 modules/aop/pom.xml                                | 2 +-
 modules/apache-license-gen/pom.xml                 | 2 +-
 modules/aws/pom.xml                                | 2 +-
 modules/clients/pom.xml                            | 2 +-
 modules/cloud/pom.xml                              | 2 +-
 modules/codegen/pom.xml                            | 2 +-
 modules/core/pom.xml                               | 2 +-
 modules/core/src/main/resources/ignite.properties  | 2 +-
 modules/extdata/p2p/pom.xml                        | 2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml | 2 +-
 modules/extdata/uri/pom.xml                        | 2 +-
 modules/gce/pom.xml                                | 2 +-
 modules/geospatial/pom.xml                         | 2 +-
 modules/hadoop/pom.xml                             | 2 +-
 modules/hibernate/pom.xml                          | 2 +-
 modules/indexing/pom.xml                           | 2 +-
 modules/jcl/pom.xml                                | 2 +-
 modules/jta/pom.xml                                | 2 +-
 modules/kafka/pom.xml                              | 2 +-
 modules/log4j/pom.xml                              | 2 +-
 modules/mesos/pom.xml                              | 2 +-
 modules/rest-http/pom.xml                          | 2 +-
 modules/scalar-2.10/pom.xml                        | 2 +-
 modules/scalar/pom.xml                             | 2 +-
 modules/schedule/pom.xml                           | 2 +-
 modules/schema-import/pom.xml                      | 2 +-
 modules/slf4j/pom.xml                              | 2 +-
 modules/spark-2.10/pom.xml                         | 2 +-
 modules/spark/pom.xml                              | 2 +-
 modules/spring/pom.xml                             | 2 +-
 modules/ssh/pom.xml                                | 2 +-
 modules/tools/pom.xml                              | 2 +-
 modules/urideploy/pom.xml                          | 2 +-
 modules/visor-console-2.10/pom.xml                 | 2 +-
 modules/visor-console/pom.xml                      | 2 +-
 modules/visor-plugins/pom.xml                      | 2 +-
 modules/web/pom.xml                                | 2 +-
 modules/yardstick/pom.xml                          | 2 +-
 modules/yarn/pom.xml                               | 2 +-
 pom.xml                                            | 2 +-
 41 files changed, 41 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index c4445fa..6abf904 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 13915f1..d10e9d4 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 144cb45..bbfe24d 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,5 +31,5 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 60aaa17..e52d6dc 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 8f44d23..58d32ef 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 6204d35..ac1858f 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 7ada8fc..85897d5 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 1539550..d093d92 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 8e38b9e..a8e23fb 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.3.3-p6-SNAPSHOT
+ignite.version=1.3.3-p7-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 97a3efd..76b2a1b 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/extdata/uri/modules/uri-dependency/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml
index aee39b4..c147131 100644
--- a/modules/extdata/uri/modules/uri-dependency/pom.xml
+++ b/modules/extdata/uri/modules/uri-dependency/pom.xml
@@ -27,7 +27,7 @@
     <artifactId>ignite-extdata-uri-dep</artifactId>
     <packaging>jar</packaging>
 
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index ad45998..cc337eb 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index c399c02..ab3c14e 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index d5d2b17..dd9c6d5 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 2dfae51..4097e9f 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 7ed8bf7..121738b 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index b05c963..12010d9 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 5b48182..7d349d4 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 52f7717..d66d9b5 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index 691e445..576d75d 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 7b52dc0..fcf4143 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index cb40805..202af29 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <mesos.version>0.22.0</mesos.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 2ae943a..4a0bd77 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 86b80e6..b6d6b6a 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index c853592..91e5736 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index b748944..a3b9657 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 4b5ae9f..7e26a63 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index e00a38a..0b45a4e 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index cae8086..6a743a6 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index f61b3f9..3222c6a 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 4e38ddc..f3f4e76 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index b25822f..17c7362 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 67940d6..7d446d1 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index d08a5fb..832fe8f 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index c43e452..b652c11 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index db33d7b..e520f5e 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index a1a67ea..56ea80d 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 1f930af..6316425 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index f48ce2e..43626a6 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index ebac7b2..d75c104 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
 
     <properties>
         <hadoop.version>2.7.0</hadoop.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b537fa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 642e344..40612b6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.3.3-p6-SNAPSHOT</version>
+    <version>1.3.3-p7-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[4/7] ignite git commit: Fixed cache stop procedure

Posted by sb...@apache.org.
Fixed cache stop procedure


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

Branch: refs/heads/ignite-1.4
Commit: dadf4bd3beb5fde31b89b0c2a0fbfb3aa5b46e3a
Parents: c1b537f
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Sep 1 14:44:37 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Sep 1 14:44:37 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dadf4bd3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index f13af23..c08f1d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1683,7 +1683,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCacheIoManager ioMgr = new GridCacheIoManager();
 
         CacheJtaManagerAdapter jta = JTA.createOptional();
-        
+
         return new GridCacheSharedContext(
             kernalCtx,
             tm,
@@ -2875,7 +2875,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Cancel all user operations.
      */
     public void cancelUserOperations() {
-        for (GridCacheAdapter<?, ?> cache : caches.values())
+        for (GridCacheAdapter<?, ?> cache : stoppedCaches.values())
             cache.ctx.mvcc().cancelClientFutures();
     }
 


[3/7] ignite git commit: ignite-993 Update registeredCaches map only from discovery thread

Posted by sb...@apache.org.
ignite-993 Update registeredCaches map only from discovery thread


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

Branch: refs/heads/ignite-1.4
Commit: 3321645392e25a2d0ed5e469de57d6d1a2dc173d
Parents: c1b537f
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 1 15:02:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 1 15:21:54 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  50 +++--
 .../cache/DynamicCacheChangeRequest.java        |  17 ++
 .../cache/DynamicCacheDescriptor.java           |  14 --
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../processors/cache/GridCacheProcessor.java    | 187 +++++++++----------
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   2 -
 6 files changed, 137 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index fac6f6d..ce5aca9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -245,7 +245,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param cacheName Cache name.
      */
     public void removeCacheFilter(String cacheName) {
-        registeredCaches.remove(cacheName);
+        CachePredicate p = registeredCaches.remove(cacheName);
+
+        assert p != null : cacheName;
     }
 
     /**
@@ -254,12 +256,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param cacheName Cache name.
      * @param clientNodeId Near node ID.
      * @param nearEnabled Near enabled flag.
+     * @return {@code True} if new node ID was added.
      */
-    public void addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) {
-        CachePredicate pred = registeredCaches.get(cacheName);
+    public boolean addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) {
+        CachePredicate p = registeredCaches.get(cacheName);
 
-        if (pred != null)
-            pred.addClientNode(clientNodeId, nearEnabled);
+        assert p != null : cacheName;
+
+        return p.addClientNode(clientNodeId, nearEnabled);
     }
 
     /**
@@ -267,12 +271,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      *
      * @param cacheName Cache name.
      * @param clientNodeId Near node ID.
+     * @return {@code True} if existing node ID was removed.
      */
-    public void onClientCacheClose(String cacheName, UUID clientNodeId) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+    public boolean onClientCacheClose(String cacheName, UUID clientNodeId) {
+        CachePredicate p = registeredCaches.get(cacheName);
+
+        assert p != null : cacheName;
 
-        if (predicate != null)
-            predicate.onNodeLeft(clientNodeId);
+        return p.onNodeLeft(clientNodeId);
     }
 
     /**
@@ -2621,16 +2627,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      */
     private static class CachePredicate {
         /** Cache filter. */
-        private IgnitePredicate<ClusterNode> cacheFilter;
+        private final IgnitePredicate<ClusterNode> cacheFilter;
 
         /** If near cache is enabled on data nodes. */
-        private boolean nearEnabled;
+        private final boolean nearEnabled;
 
         /** Flag indicating if cache is local. */
-        private boolean loc;
+        private final boolean loc;
 
         /** Collection of client near nodes. */
-        private ConcurrentHashMap<UUID, Boolean> clientNodes;
+        private final ConcurrentHashMap<UUID, Boolean> clientNodes;
 
         /**
          * @param cacheFilter Cache filter.
@@ -2650,16 +2656,26 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /**
          * @param nodeId Near node ID to add.
          * @param nearEnabled Near enabled flag.
+         * @return {@code True} if new node ID was added.
          */
-        public void addClientNode(UUID nodeId, boolean nearEnabled) {
-            clientNodes.putIfAbsent(nodeId, nearEnabled);
+        public boolean addClientNode(UUID nodeId, boolean nearEnabled) {
+            assert nodeId != null;
+
+            Boolean old = clientNodes.putIfAbsent(nodeId, nearEnabled);
+
+            return old == null;
         }
 
         /**
          * @param leftNodeId Left node ID.
+         * @return {@code True} if existing node ID was removed.
          */
-        public void onNodeLeft(UUID leftNodeId) {
-            clientNodes.remove(leftNodeId);
+        public boolean onNodeLeft(UUID leftNodeId) {
+            assert leftNodeId != null;
+
+            Boolean old = clientNodes.remove(leftNodeId);
+
+            return old != null;
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
index 7af1572..2029a95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
@@ -66,6 +66,9 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** Template configuration flag. */
     private boolean template;
 
+    /** */
+    private transient boolean exchangeNeeded;
+
     /**
      * Constructor creates cache stop request.
      *
@@ -78,6 +81,20 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
+     * @return {@code True} if request should trigger partition exchange.
+     */
+    public boolean exchangeNeeded() {
+        return exchangeNeeded;
+    }
+
+    /**
+     * @param exchangeNeeded {@code True} if request should trigger partition exchange.
+     */
+    public void exchangeNeeded(boolean exchangeNeeded) {
+        this.exchangeNeeded = exchangeNeeded;
+    }
+
+    /**
      * @param template {@code True} if this is request for adding template configuration.
      */
     public void template(boolean template) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 9c6cc43..f68e920 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -168,20 +168,6 @@ public class DynamicCacheDescriptor {
     }
 
     /**
-     * Sets cancelled flag.
-     */
-    public void onCancelled() {
-        cancelled = true;
-    }
-
-    /**
-     * @return Cancelled flag.
-     */
-    public boolean cancelled() {
-        return cancelled;
-    }
-
-    /**
      * @param nodeId Remote node ID.
      * @return Configuration.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 4398b4c..5b6f750 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -156,7 +156,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                         // Validate requests to check if event should trigger partition exchange.
                         for (DynamicCacheChangeRequest req : batch.requests()) {
-                            if (cctx.cache().exchangeNeeded(req))
+                            if (req.exchangeNeeded())
                                 valid.add(req);
                             else
                                 cctx.cache().completeStartFuture(req);

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index f13af23..c1d0d17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1405,36 +1405,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param req Request to check.
-     * @return {@code True} if change request was registered to apply.
-     */
-    @SuppressWarnings("IfMayBeConditional")
-    public boolean exchangeNeeded(DynamicCacheChangeRequest req) {
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
-
-        if (desc != null) {
-            if (req.close()) {
-                assert req.initiatingNodeId() != null : req;
-
-                return true;
-            }
-
-            if (desc.deploymentId().equals(req.deploymentId())) {
-                if (req.start())
-                    return !desc.cancelled();
-                else
-                    return desc.cancelled();
-            }
-
-            // If client requested cache start
-            if (req.initiatingNodeId() != null)
-                return true;
-        }
-
-        return false;
-    }
-
-    /**
      * @param reqs Requests to start.
      * @param topVer Topology version.
      * @throws IgniteCheckedException If failed to start cache.
@@ -1622,11 +1592,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     stopGateway(req);
 
                     prepareCacheStop(req);
-
-                    DynamicCacheDescriptor desc = registeredCaches.get(masked);
-
-                    if (desc != null && desc.cancelled() && desc.deploymentId().equals(req.deploymentId()))
-                        registeredCaches.remove(masked, desc);
                 }
                 else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) {
                     IgniteCacheProxy<?, ?> proxy = jCacheProxies.remove(masked);
@@ -1709,17 +1674,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             new ArrayList<>(registeredCaches.size() + registeredTemplates.size());
 
         for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-            if (!desc.cancelled()) {
-                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
 
-                req.startCacheConfiguration(desc.cacheConfiguration());
+            req.startCacheConfiguration(desc.cacheConfiguration());
 
-                req.cacheType(desc.cacheType());
+            req.cacheType(desc.cacheType());
 
-                req.deploymentId(desc.deploymentId());
+            req.deploymentId(desc.deploymentId());
 
-                reqs.add(req);
-            }
+            reqs.add(req);
         }
 
         for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
@@ -1980,10 +1943,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 return new GridFinishedFuture<>(e);
             }
 
-            if (desc != null && !desc.cancelled()) {
-                if (failIfExists)
+            if (desc != null) {
+                if (failIfExists) {
                     return new GridFinishedFuture<>(new CacheExistsException("Failed to start cache " +
                         "(a cache with the same name is already started): " + cacheName));
+                }
                 else {
                     CacheConfiguration descCfg = desc.cacheConfiguration();
 
@@ -2029,7 +1993,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         else {
             req.clientStartOnly(true);
 
-            if (desc != null && !desc.cancelled())
+            if (desc != null)
                 ccfg = desc.cacheConfiguration();
 
             if (ccfg == null) {
@@ -2212,83 +2176,104 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
 
-            if (req.start()) {
-                CacheConfiguration ccfg = req.startCacheConfiguration();
+            boolean needExchange = false;
 
-                DynamicCacheStartFuture startFut = (DynamicCacheStartFuture)pendingFuts.get(
-                    maskNull(ccfg.getName()));
+            DynamicCacheStartFuture fut = null;
 
-                // Check if cache with the same name was concurrently started form different node.
-                if (desc != null) {
-                    if (!req.clientStartOnly() && req.failIfExists()) {
-                        // If local node initiated start, fail the start future.
-                        if (startFut != null && startFut.deploymentId().equals(req.deploymentId())) {
-                            startFut.onDone(new CacheExistsException("Failed to start cache " +
-                                "(a cache with the same name is already started): " + U.maskName(ccfg.getName())));
-                        }
+            if (ctx.localNodeId().equals(req.initiatingNodeId())) {
+                fut = (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName()));
 
-                        return;
+                if (!req.deploymentId().equals(fut.deploymentId()))
+                    fut = null;
+            }
+
+            if (req.start()) {
+                if (desc == null) {
+                    if (req.clientStartOnly()) {
+                        if (fut != null)
+                            fut.onDone(new IgniteCheckedException("Failed to start client cache " +
+                                "(a cache with the given name is not started): " + U.maskName(req.cacheName())));
                     }
+                    else {
+                        CacheConfiguration ccfg = req.startCacheConfiguration();
+
+                        assert req.cacheType() != null : req;
+                        assert F.eq(ccfg.getName(), req.cacheName()) : req;
 
-                    req.clientStartOnly(true);
+                        DynamicCacheDescriptor startDesc =
+                            new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, req.deploymentId());
+
+                        DynamicCacheDescriptor old = registeredCaches.put(maskNull(ccfg.getName()), startDesc);
+
+                        assert old == null :
+                            "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
+
+                        ctx.discovery().setCacheFilter(
+                            ccfg.getName(),
+                            ccfg.getNodeFilter(),
+                            ccfg.getNearConfiguration() != null,
+                            ccfg.getCacheMode() == LOCAL);
+
+                        ctx.discovery().addClientNode(req.cacheName(),
+                            req.initiatingNodeId(),
+                            req.nearCacheConfiguration() != null);
+
+                        needExchange = true;
+                    }
                 }
                 else {
                     if (req.clientStartOnly()) {
-                        if (startFut != null && startFut.deploymentId().equals(req.deploymentId())) {
-                            startFut.onDone(new IgniteCheckedException("Failed to start client cache " +
-                                "(a cache with the given name is not started): " + U.maskName(ccfg.getName())));
-                        }
+                        assert req.initiatingNodeId() != null : req;
 
-                        return;
+                        needExchange = ctx.discovery().addClientNode(req.cacheName(),
+                            req.initiatingNodeId(),
+                            req.nearCacheConfiguration() != null);
                     }
-                }
-
-                if (!req.clientStartOnly() && desc == null) {
-                    assert req.cacheType() != null : req;
-
-                    DynamicCacheDescriptor startDesc =
-                        new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, req.deploymentId());
+                    else {
+                        if (req.failIfExists() ) {
+                            if (fut != null)
+                                fut.onDone(new CacheExistsException("Failed to start cache " +
+                                    "(a cache with the same name is already started): " + U.maskName(req.cacheName())));
+                        }
+                        else {
+                            // Cache already exists, exchange is needed only if client cache should be created.
+                            ClusterNode node = ctx.discovery().node(req.initiatingNodeId());
 
-                    DynamicCacheDescriptor old = registeredCaches.put(maskNull(ccfg.getName()), startDesc);
+                            boolean clientReq = node != null &&
+                                !ctx.discovery().cacheAffinityNode(node, req.cacheName());
 
-                    assert old == null :
-                        "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
+                            needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
+                                req.initiatingNodeId(),
+                                req.nearCacheConfiguration() != null);
 
-                    ctx.discovery().setCacheFilter(
-                        ccfg.getName(),
-                        ccfg.getNodeFilter(),
-                        ccfg.getNearConfiguration() != null,
-                        ccfg.getCacheMode() == LOCAL);
+                            if (needExchange)
+                                req.clientStartOnly(true);
+                        }
+                    }
                 }
-
-                ctx.discovery().addClientNode(req.cacheName(),
-                    req.initiatingNodeId(),
-                    req.nearCacheConfiguration() != null);
             }
             else {
-                assert req.stop() || req.close() : req;
+                assert req.stop() ^ req.close() : req;
 
-                if (desc == null) {
-                    // If local node initiated start, finish future.
-                    DynamicCacheStartFuture changeFut =
-                        (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName()));
+                if (desc != null) {
+                    if (req.stop()) {
+                        DynamicCacheDescriptor old = registeredCaches.remove(maskNull(req.cacheName()));
 
-                    if (changeFut != null && changeFut.deploymentId().equals(req.deploymentId())) {
-                        // No-op.
-                        changeFut.onDone();
-                    }
+                        assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
 
-                    return;
-                }
+                        ctx.discovery().removeCacheFilter(req.cacheName());
 
-                if (req.stop()) {
-                    desc.onCancelled();
+                        needExchange = true;
+                    }
+                    else {
+                        assert req.close() : req;
 
-                    ctx.discovery().removeCacheFilter(req.cacheName());
+                        needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
+                    }
                 }
-                else
-                    ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
             }
+
+            req.exchangeNeeded(needExchange);
         }
     }
 
@@ -2711,7 +2696,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(name));
 
-        if (desc == null || desc.cancelled())
+        if (desc == null)
             throw new IllegalArgumentException("Cache is not started: " + name);
 
         if (!desc.cacheType().userCache())
@@ -2779,7 +2764,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public CacheConfiguration cacheConfiguration(String name) {
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(name));
 
-        if (desc == null || desc.cancelled())
+        if (desc == null)
             throw new IllegalStateException("Cache doesn't exist: " + name);
         else
             return desc.cacheConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/33216453/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index d1f8016..95f7701 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -1118,8 +1118,6 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStartStopSameCacheMultinode() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-993");
-
         final AtomicInteger idx = new AtomicInteger();
 
         IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {