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 2017/05/02 15:08:02 UTC

[01/26] ignite git commit: cache discovery data refactoring

Repository: ignite
Updated Branches:
  refs/heads/ignite-5075 91452b127 -> a1d410412


http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/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 654d306..3c056fd 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
@@ -540,10 +540,15 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             }
             else {
                 if (discoEvt.type() == EVT_NODE_JOINED) {
-                    Collection<DynamicCacheDescriptor> receivedCaches = cctx.cache().startReceivedCaches(topVer);
+                    if (!discoEvt.eventNode().isLocal()) {
+                        Collection<DynamicCacheDescriptor> receivedCaches = cctx.cache().startReceivedCaches(
+                            discoEvt.eventNode().id(),
+                            topVer);
 
-                    if (!discoEvt.eventNode().isLocal())
                         cctx.affinity().initStartedCaches(crdNode, this, receivedCaches);
+                    }
+                    else
+                        cctx.cache().startCachesOnLocalJoin(topVer);
                 }
 
                 exchange = CU.clientNode(discoEvt.eventNode()) ?


[23/26] ignite git commit: ignite-5075

Posted by sb...@apache.org.
ignite-5075


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

Branch: refs/heads/ignite-5075
Commit: 3e6113bb0c2e9c4fece2e58627c064896ba2d314
Parents: 98b4378
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 15:36:41 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 16:38:54 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       | 34 +++++----
 .../processors/cache/ClusterCachesInfo.java     | 56 +++++++++++----
 .../processors/cache/ExchangeActions.java       | 13 +---
 .../processors/cache/GridCacheProcessor.java    | 74 ++++++++++----------
 .../processors/cache/local/GridLocalCache.java  | 31 +-------
 .../cache/local/GridLocalLockFuture.java        | 41 ++++++++++-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  6 +-
 7 files changed, 141 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3e6113bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 48bc6da..45f463b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -323,12 +323,10 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param exchActions Cache change requests to execte on exchange.
      */
     private void updateCachesInfo(ExchangeActions exchActions) {
-        for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
-            Integer cacheId = CU.cacheId(req.cacheName());
-
-            DynamicCacheDescriptor desc = registeredCaches.remove(cacheId);
+        for (ExchangeActions.ActionData action : exchActions.stopRequests()) {
+            DynamicCacheDescriptor desc = registeredCaches.remove(action.descriptor().cacheId());
 
-            assert desc != null : req.cacheName();
+            assert desc != null : action.request().cacheName();
         }
 
         for (ExchangeActions.ActionData action : exchActions.newAndClientCachesStartRequests()) {
@@ -380,9 +378,9 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         });
 
         for (ExchangeActions.ActionData action : exchActions.newAndClientCachesStartRequests()) {
-            DynamicCacheChangeRequest req = action.request();
+            DynamicCacheDescriptor cacheDesc = action.descriptor();
 
-            Integer cacheId = CU.cacheId(req.cacheName());
+            DynamicCacheChangeRequest req = action.request();
 
             boolean startCache;
 
@@ -399,15 +397,15 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             }
 
             if (startCache)
-                cctx.cache().prepareCacheStart(req, nearCfg, action.descriptor(), fut.topologyVersion());
+                cctx.cache().prepareCacheStart(cacheDesc, nearCfg, fut.topologyVersion());
 
-            if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
+            if (fut.isCacheAdded(cacheDesc.cacheId(), fut.topologyVersion())) {
                 if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
                     U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
             }
 
             if (!crd || !lateAffAssign) {
-                GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+                GridCacheContext cacheCtx = cctx.cacheContext(cacheDesc.cacheId());
 
                 if (cacheCtx != null && !cacheCtx.isLocal()) {
                     boolean clientCacheStarted =
@@ -430,7 +428,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 }
             }
             else
-                initStartedCacheOnCoordinator(fut, cacheId);
+                initStartedCacheOnCoordinator(fut, cacheDesc.cacheId());
         }
 
         for (DynamicCacheChangeRequest req : exchActions.closeRequests(cctx.localNodeId())) {
@@ -459,22 +457,22 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         Set<Integer> stoppedCaches = null;
 
-        for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
-            Integer cacheId = CU.cacheId(req.cacheName());
+        for (ExchangeActions.ActionData action : exchActions.stopRequests()) {
+            DynamicCacheDescriptor desc = action.descriptor();
 
-            cctx.cache().blockGateway(req);
+            cctx.cache().blockGateway(action.request());
 
-            if (crd) {
-                CacheHolder cache = caches.remove(cacheId);
+            if (crd && desc.cacheConfiguration().getCacheMode() != LOCAL) {
+                CacheHolder cache = caches.remove(desc.cacheId());
 
-                assert cache != null : req;
+                assert cache != null : action.request();
 
                 if (stoppedCaches == null)
                     stoppedCaches = new HashSet<>();
 
                 stoppedCaches.add(cache.cacheId());
 
-                cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class);
+                cctx.io().removeHandler(desc.cacheId(), GridDhtAffinityAssignmentResponse.class);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e6113bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index e75f93c..1064de3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -84,6 +84,9 @@ class ClusterCachesInfo {
         this.ctx = ctx;
     }
 
+    /**
+     * @param joinDiscoData Information about configured caches and templates.
+     */
     void onStart(CacheJoinNodeDiscoveryData joinDiscoData) {
         this.joinDiscoData = joinDiscoData;
     }
@@ -317,9 +320,12 @@ class ClusterCachesInfo {
         return incMinorTopVer;
     }
 
+    /**
+     * @param dataBag Discovery data bag.
+     */
     void collectJoiningNodeData(DiscoveryDataBag dataBag) {
         if (!ctx.isDaemon())
-        dataBag.addJoiningNodeData(CACHE_PROC.ordinal(), joinDiscoveryData());
+            dataBag.addJoiningNodeData(CACHE_PROC.ordinal(), joinDiscoveryData());
     }
 
     /**
@@ -411,26 +417,27 @@ class ClusterCachesInfo {
                     assert joinDiscoData != null;
                 }
 
-                processJoiningNode(joinDiscoData, node.id());
-
                 assert locJoinStartCaches == null;
 
                 locJoinStartCaches = new ArrayList<>();
 
-                for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                    CacheConfiguration cfg = desc.cacheConfiguration();
+                if (!disconnectedState()) {
+                    processJoiningNode(joinDiscoData, node.id());
 
-                    CacheJoinNodeDiscoveryData.CacheInfo locCfg = joinDiscoData.caches().get(cfg.getName());
+                    for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                        CacheConfiguration cfg = desc.cacheConfiguration();
 
-                    boolean affNode = CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter());
+                        CacheJoinNodeDiscoveryData.CacheInfo locCfg = joinDiscoData.caches().get(cfg.getName());
 
-                    NearCacheConfiguration nearCfg = (!affNode && locCfg != null) ? locCfg.config().getNearConfiguration() : null;
+                        NearCacheConfiguration nearCfg = locCfg != null ? locCfg.config().getNearConfiguration() :
+                            null;
 
-                    if (locCfg != null || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()))
-                        locJoinStartCaches.add(new T2<>(desc, nearCfg));
-                }
+                        if (locCfg != null || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()))
+                            locJoinStartCaches.add(new T2<>(desc, nearCfg));
+                    }
 
-                joinDiscoData = null;
+                    joinDiscoData = null;
+                }
             }
             else {
                 CacheJoinNodeDiscoveryData discoData = joiningNodesDiscoData.remove(node.id());
@@ -459,6 +466,9 @@ class ClusterCachesInfo {
         }
     }
 
+    /**
+     * @param dataBag Discovery data bag.
+     */
     void collectGridNodeData(DiscoveryDataBag dataBag) {
         if (ctx.isDaemon())
             return;
@@ -515,7 +525,7 @@ class ClusterCachesInfo {
         if (ctx.isDaemon() || data.commonData() == null)
             return;
 
-        assert joinDiscoData != null;
+        assert joinDiscoData != null || disconnectedState();
         assert data.commonData() instanceof CacheNodeCommonDiscoveryData : data;
 
         CacheNodeCommonDiscoveryData cachesData = (CacheNodeCommonDiscoveryData)data.commonData();
@@ -576,6 +586,9 @@ class ClusterCachesInfo {
         gridData = cachesData;
     }
 
+    /**
+     * @param data Joining node data.
+     */
     void onJoiningNodeDataReceived(DiscoveryDataBag.JoiningNodeDiscoveryData data) {
         if (data.hasJoiningNodeData()) {
             Serializable joiningNodeData = data.joiningNodeData();
@@ -610,6 +623,10 @@ class ClusterCachesInfo {
         }
     }
 
+    /**
+     * @param joinData Joined node discovery data.
+     * @param nodeId Joined node ID.
+     */
     private void processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId) {
         for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.templates().values()) {
             CacheConfiguration cfg = cacheInfo.config();
@@ -688,7 +705,7 @@ class ClusterCachesInfo {
      * @return Stopped caches names.
      */
     Set<String> onReconnected() {
-        assert cachesOnDisconnect != null;
+        assert disconnectedState();
 
         Set<String> stoppedCaches = new HashSet<>();
 
@@ -716,6 +733,17 @@ class ClusterCachesInfo {
         return stoppedCaches;
     }
 
+    /**
+     * @return {@code True} if client node is currently in disconnected state.
+     */
+    private boolean disconnectedState() {
+        return cachesOnDisconnect != null;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @return {@code True} if cache with given name if system cache which should always survive client node disconnect.
+     */
     private boolean surviveReconnect(String cacheName) {
         return CU.isUtilityCache(cacheName) || CU.isAtomicsCache(cacheName);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e6113bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index 9be4b6a..bcc77f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -104,17 +104,8 @@ public class ExchangeActions {
     /**
      * @return Stop cache requests.
      */
-    public List<DynamicCacheChangeRequest> stopRequests() {
-        List<DynamicCacheChangeRequest> res = null;
-
-        if (cachesToStop != null) {
-            res = new ArrayList<>(cachesToStop.size());
-
-            for (ActionData req : cachesToStop.values())
-                res.add(req.req);
-        }
-
-        return res != null ? res : Collections.<DynamicCacheChangeRequest>emptyList();
+    Collection<ActionData> stopRequests() {
+        return cachesToStop != null ? cachesToStop.values() : Collections.EMPTY_LIST;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e6113bb/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 b4e4b14..2ed20e4 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
@@ -1338,6 +1338,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         AffinityTopologyVersion cacheStartTopVer,
         AffinityTopologyVersion locStartTopVer,
         CacheObjectContext cacheObjCtx,
+        boolean affNode,
         boolean updatesAllowed)
         throws IgniteCheckedException {
         assert cfg != null;
@@ -1398,8 +1399,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         storeMgr.initialize(cfgStore, sesHolders);
 
-        boolean affNode = cfg.getCacheMode() == LOCAL || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter());
-
         String memPlcName = cfg.getMemoryPolicyName();
 
         MemoryPolicy memPlc = sharedCtx.database().memoryPolicy(memPlcName);
@@ -1710,42 +1709,34 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param req Cache start request.
+     * @param cacheDesc Cache start request.
      * @param nearCfg Near cache configuration.
-     * @param desc Cache descriptor.
      * @param exchTopVer Current exchange version.
      * @throws IgniteCheckedException If failed.
      */
-    void prepareCacheStart(DynamicCacheChangeRequest req,
+    void prepareCacheStart(DynamicCacheDescriptor cacheDesc,
         @Nullable NearCacheConfiguration nearCfg,
-        DynamicCacheDescriptor desc,
         AffinityTopologyVersion exchTopVer)
         throws IgniteCheckedException {
-        assert req.start() : req;
-        assert req.cacheType() != null : req;
-
         prepareCacheStart(
-            req.startCacheConfiguration(),
+            cacheDesc.cacheConfiguration(),
             nearCfg,
-            req.cacheType(),
-            req.deploymentId(),
-            desc.startTopologyVersion(),
+            cacheDesc.cacheType(),
+            cacheDesc.deploymentId(),
+            cacheDesc.startTopologyVersion(),
             exchTopVer,
-            desc.schema()
+            cacheDesc.schema()
         );
     }
 
     /**
      * @param exchTopVer Current exchange version.
      * @throws IgniteCheckedException If failed.
-     * @return Collection of started caches.
      */
-    public List<DynamicCacheDescriptor> startCachesOnLocalJoin(AffinityTopologyVersion exchTopVer) throws IgniteCheckedException {
+    public void startCachesOnLocalJoin(AffinityTopologyVersion exchTopVer) throws IgniteCheckedException {
         List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> caches = cachesInfo.cachesToStartOnLocalJoin();
 
         if (!F.isEmpty(caches)) {
-            List<DynamicCacheDescriptor> started = new ArrayList<>(caches.size());
-
             for (T2<DynamicCacheDescriptor, NearCacheConfiguration> t : caches) {
                 DynamicCacheDescriptor desc = t.get1();
 
@@ -1758,14 +1749,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     exchTopVer,
                     desc.schema()
                 );
-
-                started.add(desc);
             }
-
-            return started;
         }
-        else
-            return Collections.emptyList();
     }
 
     /**
@@ -1802,8 +1787,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param cfg Start configuration.
-     * @param nearCfg Near configuration.
+     * @param startCfg Start configuration.
+     * @param reqNearCfg Near configuration if specified for client cache start request.
      * @param cacheType Cache type.
      * @param deploymentId Deployment ID.
      * @param cacheStartTopVer Cache start topology version.
@@ -1812,29 +1797,42 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private void prepareCacheStart(
-        CacheConfiguration cfg,
-        NearCacheConfiguration nearCfg,
+        CacheConfiguration startCfg,
+        @Nullable NearCacheConfiguration reqNearCfg,
         CacheType cacheType,
         IgniteUuid deploymentId,
         AffinityTopologyVersion cacheStartTopVer,
         AffinityTopologyVersion exchTopVer,
         @Nullable QuerySchema schema
     ) throws IgniteCheckedException {
-        assert !caches.containsKey(cfg.getName()) : cfg.getName();
-
-        CacheConfiguration ccfg = new CacheConfiguration(cfg);
+        assert !caches.containsKey(startCfg.getName()) : startCfg.getName();
 
-        if (nearCfg != null)
-            ccfg.setNearConfiguration(nearCfg);
+        CacheConfiguration ccfg = new CacheConfiguration(startCfg);
 
         CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
+        boolean affNode;
+
+        if (ccfg.getCacheMode() == LOCAL) {
+            affNode = true;
+
+            ccfg.setNearConfiguration(null);
+        }
+        else if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter()))
+            affNode = true;
+        else {
+            affNode = false;
+
+            ccfg.setNearConfiguration(reqNearCfg);
+        }
+
         GridCacheContext cacheCtx = createCache(ccfg,
             null,
             cacheType,
             cacheStartTopVer,
             exchTopVer,
             cacheObjCtx,
+            affNode,
             true);
 
         cacheCtx.dynamicDeploymentId(deploymentId);
@@ -1931,10 +1929,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         if (exchActions != null && err == null) {
-            for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
-                stopGateway(req);
+            for (ExchangeActions.ActionData action : exchActions.stopRequests()) {
+                stopGateway(action.request());
 
-                prepareCacheStop(req);
+                prepareCacheStop(action.request());
             }
 
             for (DynamicCacheChangeRequest req : exchActions.closeRequests(ctx.localNodeId())) {
@@ -1961,7 +1959,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cacheName Cache name.
-     * @param deploymentId
+     * @param deploymentId Future deployment ID.
      */
     void completeTemplateAddFuture(String cacheName, IgniteUuid deploymentId) {
         GridCacheProcessor.TemplateConfigurationFuture fut =
@@ -1975,7 +1973,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param req Request to complete future for.
      * @param err Error if any.
      */
-    public void completeCacheStartFuture(DynamicCacheChangeRequest req, @Nullable Exception err) {
+    void completeCacheStartFuture(DynamicCacheChangeRequest req, @Nullable Exception err) {
         if (req.initiatingNodeId().equals(ctx.localNodeId())) {
             DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e6113bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
index 94f618a..88499c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
@@ -148,35 +148,8 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
         GridLocalLockFuture<K, V> fut = new GridLocalLockFuture<>(ctx, keys, tx, this, timeout, filter);
 
         try {
-            for (KeyCacheObject key : keys) {
-                while (true) {
-                    GridLocalCacheEntry entry = null;
-
-                    try {
-                        entry = entryExx(key);
-
-                        entry.unswap(false);
-
-                        if (!ctx.isAll(entry, filter)) {
-                            fut.onFailed();
-
-                            return fut;
-                        }
-
-                        // Removed exception may be thrown here.
-                        GridCacheMvccCandidate cand = fut.addEntry(entry);
-
-                        if (cand == null && fut.isDone())
-                            return fut;
-
-                        break;
-                    }
-                    catch (GridCacheEntryRemovedException ignored) {
-                        if (log().isDebugEnabled())
-                            log().debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
-                    }
-                }
-            }
+            if (!fut.addEntries(keys))
+                return fut;
 
             if (!ctx.mvcc().addFuture(fut))
                 fut.onError(new IgniteCheckedException("Duplicate future ID (internal error): " + fut));

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e6113bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
index 59d0adb..9641533 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
@@ -144,12 +144,51 @@ public final class GridLocalLockFuture<K, V> extends GridCacheFutureAdapter<Bool
 
         if (log == null)
             log = U.logger(cctx.kernalContext(), logRef, GridLocalLockFuture.class);
+    }
+
+    /**
+     * @param keys Keys.
+     * @return {@code False} in case of error.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean addEntries(Collection<KeyCacheObject> keys) throws IgniteCheckedException {
+        for (KeyCacheObject key : keys) {
+            while (true) {
+                GridLocalCacheEntry entry = null;
+
+                try {
+                    entry = cache.entryExx(key);
+
+                    entry.unswap(false);
+
+                    if (!cctx.isAll(entry, filter)) {
+                        onFailed();
+
+                        return false;
+                    }
+
+                    // Removed exception may be thrown here.
+                    GridCacheMvccCandidate cand = addEntry(entry);
+
+                    if (cand == null && isDone())
+                        return false;
+
+                    break;
+                }
+                catch (GridCacheEntryRemovedException ignored) {
+                    if (log.isDebugEnabled())
+                        log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
+                }
+            }
+        }
 
         if (timeout > 0) {
             timeoutObj = new LockTimeoutObject();
 
             cctx.time().addTimeoutObject(timeoutObj);
         }
+
+        return true;
     }
 
     /** {@inheritDoc} */
@@ -216,7 +255,7 @@ public final class GridLocalLockFuture<K, V> extends GridCacheFutureAdapter<Bool
      * @return Lock candidate.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
-    @Nullable GridCacheMvccCandidate addEntry(GridLocalCacheEntry entry)
+    private @Nullable GridCacheMvccCandidate addEntry(GridLocalCacheEntry entry)
         throws GridCacheEntryRemovedException {
         // Add local lock first, as it may throw GridCacheEntryRemovedException.
         GridCacheMvccCandidate c = entry.addLocal(

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e6113bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
index a4a831f..546ec06 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -44,13 +44,13 @@ public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnma
 
     /** {@inheritDoc} */
     @Override public void testResponseMessageOnUnmarshallingFailed() throws InterruptedException {
-        //GridCacheEvictionRequest unmarshalling failed test
-        readCnt.set(5); //2 for each put
+        //GridCacheEvictionRequest unmarshalling failed test.
+        readCnt.set(5); //2 for each put.
 
         jcache(0).put(new TestKey(String.valueOf(++key)), "");
         jcache(0).put(new TestKey(String.valueOf(++key)), "");
 
-        //Eviction request unmarshalling failed but ioManager does not hangs up.
+        // Eviction request unmarshalling failed but ioManager does not hangs up.
 
         // Wait for eviction complete.
         Thread.sleep(1000);


[06/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java


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

Branch: refs/heads/ignite-5075
Commit: ab92ab2ea8031870fe421bda7d300b81bfd27c42
Parents: 0022f6b d4b6fec
Author: sboikov <sb...@gridgain.com>
Authored: Thu Apr 27 18:01:34 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Apr 27 18:01:34 2017 +0300

----------------------------------------------------------------------
 .../client/memcache/MemcacheRestExample.java    |   2 +-
 .../MemcacheRestExampleNodeStartup.java         |   1 +
 .../ml/math/matrix/CacheMatrixExample.java      |  23 +-
 .../ml/math/matrix/ExampleMatrixStorage.java    |   3 +-
 .../math/matrix/MatrixCustomStorageExample.java |   6 +-
 .../examples/ml/math/matrix/MatrixExample.java  |   4 +-
 .../ml/math/matrix/OffHeapMatrixExample.java    |   6 +-
 .../matrix/SparseDistributedMatrixExample.java  |   8 +-
 .../ml/math/matrix/SparseMatrixExample.java     |   4 +-
 .../examples/ml/math/tracer/TracerExample.java  |   2 +-
 .../ml/math/vector/CacheVectorExample.java      |  19 +-
 .../ml/math/vector/ExampleVectorStorage.java    |   7 +-
 .../ml/math/vector/OffHeapVectorExample.java    |   2 +-
 .../ml/math/vector/SparseVectorExample.java     |   4 -
 .../math/vector/VectorCustomStorageExample.java |   4 -
 .../examples/ml/math/vector/VectorExample.java  |   6 +-
 .../ignite/ignite-cassandra-server-template.xml |   3 -
 .../tests/ignite-cassandra-client-template.xml  |   3 -
 .../client/ClientDefaultCacheSelfTest.java      |   4 +-
 .../JettyRestProcessorAbstractSelfTest.java     | 265 +++++++++++++------
 .../clients/src/test/resources/spring-cache.xml |   9 -
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../configuration/CacheConfiguration.java       |  31 +--
 .../configuration/DataPageEvictionMode.java     |  17 +-
 .../processors/cache/GridCacheAdapter.java      |  19 +-
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../evict/PageAbstractEvictionTracker.java      |   7 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   3 +-
 .../processors/cache/local/GridLocalCache.java  |   3 +-
 .../utils/PlatformConfigurationUtils.java       |   2 -
 .../visor/binary/VisorBinaryMetadata.java       |   8 +-
 .../VisorBinaryMetadataCollectorTask.java       |  16 +-
 .../VisorBinaryMetadataCollectorTaskArg.java    |  71 +++++
 .../VisorBinaryMetadataCollectorTaskResult.java |   4 +-
 .../cache/VisorCacheAffinityConfiguration.java  |   8 +-
 .../visor/cache/VisorCacheClearTask.java        |  19 +-
 .../visor/cache/VisorCacheClearTaskArg.java     |  72 +++++
 .../visor/cache/VisorCacheConfiguration.java    | 162 +++++++++++-
 .../VisorCacheConfigurationCollectorJob.java    |  12 +-
 .../VisorCacheConfigurationCollectorTask.java   |   5 +-
 ...VisorCacheConfigurationCollectorTaskArg.java |  74 ++++++
 .../visor/cache/VisorCacheLoadTask.java         |   5 +-
 .../visor/cache/VisorCacheLoadTaskArg.java      |   2 +-
 .../visor/cache/VisorCacheMetadataTask.java     |  14 +-
 .../visor/cache/VisorCacheMetadataTaskArg.java  |  72 +++++
 .../visor/cache/VisorCacheNodesTask.java        |  12 +-
 .../visor/cache/VisorCacheNodesTaskArg.java     |  72 +++++
 .../cache/VisorCacheRebalanceConfiguration.java |  26 ++
 .../visor/cache/VisorCacheRebalanceTask.java    |  13 +-
 .../visor/cache/VisorCacheRebalanceTaskArg.java |  73 +++++
 .../visor/cache/VisorCacheResetMetricsTask.java |  14 +-
 .../cache/VisorCacheResetMetricsTaskArg.java    |  72 +++++
 .../visor/cache/VisorCacheStartArg.java         | 100 -------
 .../visor/cache/VisorCacheStopTask.java         |  17 +-
 .../visor/cache/VisorCacheStopTaskArg.java      |  72 +++++
 .../cache/VisorCacheStoreConfiguration.java     |  14 +
 .../internal/visor/cache/VisorPartitionMap.java |  24 +-
 .../compute/VisorComputeCancelSessionsTask.java |  13 +-
 .../VisorComputeCancelSessionsTaskArg.java      |  76 ++++++
 .../visor/compute/VisorGatewayTask.java         |  87 ++++--
 .../internal/visor/debug/VisorThreadInfo.java   |  64 ++---
 .../visor/debug/VisorThreadMonitorInfo.java     |   8 +-
 .../internal/visor/file/VisorFileBlockArg.java  | 114 --------
 .../visor/igfs/VisorIgfsFormatTask.java         |  14 +-
 .../visor/igfs/VisorIgfsFormatTaskArg.java      |  72 +++++
 .../visor/igfs/VisorIgfsProfilerClearTask.java  |  24 +-
 .../igfs/VisorIgfsProfilerClearTaskArg.java     |  72 +++++
 .../igfs/VisorIgfsProfilerClearTaskResult.java  |   6 +-
 .../visor/igfs/VisorIgfsProfilerTask.java       |  18 +-
 .../visor/igfs/VisorIgfsProfilerTaskArg.java    |  72 +++++
 .../visor/igfs/VisorIgfsResetMetricsTask.java   |  13 +-
 .../igfs/VisorIgfsResetMetricsTaskArg.java      |  73 +++++
 .../internal/visor/log/VisorLogSearchArg.java   | 114 --------
 .../internal/visor/misc/VisorAckTask.java       |  14 +-
 .../internal/visor/misc/VisorAckTaskArg.java    |  72 +++++
 .../misc/VisorChangeGridActiveStateTask.java    |  12 +-
 .../misc/VisorChangeGridActiveStateTaskArg.java |  71 +++++
 .../visor/node/VisorBasicConfiguration.java     | 211 ++++++++++++++-
 .../visor/node/VisorBinaryConfiguration.java    | 131 +++++++++
 .../node/VisorBinaryTypeConfiguration.java      | 150 +++++++++++
 .../visor/node/VisorCacheKeyConfiguration.java  | 108 ++++++++
 .../visor/node/VisorExecutorConfiguration.java  | 108 ++++++++
 .../node/VisorExecutorServiceConfiguration.java | 115 ++++++++
 .../visor/node/VisorGridConfiguration.java      | 110 ++++++++
 .../visor/node/VisorHadoopConfiguration.java    | 145 ++++++++++
 .../visor/node/VisorIgfsConfiguration.java      |  42 ++-
 .../node/VisorMemoryPolicyConfiguration.java    |  41 +++
 .../internal/visor/node/VisorNodePingTask.java  |  13 +-
 .../visor/node/VisorNodePingTaskArg.java        |  73 +++++
 .../visor/node/VisorNodeSuppressedErrors.java   |   6 +-
 .../node/VisorNodeSuppressedErrorsTask.java     |  12 +-
 .../node/VisorNodeSuppressedErrorsTaskArg.java  |  74 ++++++
 .../visor/node/VisorOdbcConfiguration.java      | 114 ++++++++
 .../visor/node/VisorRestConfiguration.java      | 207 ++++++++++++++-
 .../node/VisorSegmentationConfiguration.java    |  13 +
 .../visor/node/VisorServiceConfiguration.java   | 176 ++++++++++++
 .../internal/visor/query/VisorQueryArg.java     | 155 -----------
 .../visor/query/VisorQueryCancelTask.java       |  12 +-
 .../visor/query/VisorQueryCancelTaskArg.java    |  71 +++++
 .../visor/query/VisorQueryCleanupTask.java      |  10 +-
 .../visor/query/VisorQueryCleanupTaskArg.java   |  75 ++++++
 .../VisorQueryDetailMetricsCollectorTask.java   |  17 +-
 ...VisorQueryDetailMetricsCollectorTaskArg.java |  71 +++++
 .../query/VisorQueryResetDetailMetricsTask.java |   6 +-
 .../visor/query/VisorQueryResetMetricsTask.java |  18 +-
 .../query/VisorQueryResetMetricsTaskArg.java    |  72 +++++
 .../query/VisorRunningQueriesCollectorTask.java |  16 +-
 .../VisorRunningQueriesCollectorTaskArg.java    |  71 +++++
 .../internal/visor/query/VisorScanQueryArg.java | 157 -----------
 .../visor/service/VisorCancelServiceTask.java   |  12 +-
 .../service/VisorCancelServiceTaskArg.java      |  72 +++++
 .../internal/visor/util/VisorTaskUtils.java     |  23 ++
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   8 +-
 .../resources/META-INF/classnames.properties    |  39 ++-
 .../spring-cache-client-benchmark-1.xml         |   3 -
 .../spring-cache-client-benchmark-2.xml         |   3 -
 .../spring-cache-client-benchmark-3.xml         |   3 -
 modules/core/src/test/config/example-cache.xml  |   3 -
 modules/core/src/test/config/igfs-loopback.xml  |  18 --
 modules/core/src/test/config/igfs-shmem.xml     |  18 --
 .../src/test/config/load/cache-benchmark.xml    |   4 -
 .../test/config/load/cache-client-benchmark.xml |   2 -
 .../config/load/dsi-49-server-production.xml    |   2 -
 .../src/test/config/load/dsi-load-client.xml    |   2 -
 .../src/test/config/load/dsi-load-server.xml    |   2 -
 .../core/src/test/config/spring-multicache.xml  |  17 --
 .../config/websession/example-cache-base.xml    |   3 -
 .../IgniteClientReconnectCacheTest.java         |  12 +-
 ...ryConfigurationCustomSerializerSelfTest.java |   4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   2 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |   1 -
 .../cache/GridCachePutAllFailoverSelfTest.java  |   1 -
 .../CacheLateAffinityAssignmentTest.java        |   1 -
 ...GridCachePreloadRestartAbstractSelfTest.java |   1 -
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   1 -
 ...idCachePartitionedHitsAndMissesSelfTest.java |   1 -
 .../GridCachePartitionedNodeRestartTest.java    |   1 -
 ...ePartitionedOptimisticTxNodeRestartTest.java |   1 -
 .../GridCacheReplicatedNodeRestartSelfTest.java |   2 -
 .../cache/eviction/EvictionAbstractTest.java    |   1 -
 .../lru/LruNearEvictionPolicySelfTest.java      |   1 -
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |   1 -
 .../paged/PageEvictionAbstractTest.java         |  13 +-
 ...LruNearEnabledPageEvictionMultinodeTest.java |  28 ++
 ...LruNearEnabledPageEvictionMultinodeTest.java |  28 ++
 ...DeadlockDetectionMessageMarshallingTest.java |   2 +-
 .../IgniteDataStreamerPerformanceTest.java      |   1 -
 .../loadtests/GridCacheMultiNodeLoadTest.java   |   1 -
 .../capacity/spring-capacity-cache.xml          |   3 -
 .../loadtests/colocation/spring-colocation.xml  |   5 -
 .../GridCachePartitionedAtomicLongLoadTest.java |   1 -
 .../GridInternalTasksLoadBalancingSelfTest.java |   7 +-
 .../configvariations/ConfigVariations.java      |   2 -
 .../testframework/junits/GridAbstractTest.java  |   7 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |   4 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   2 +-
 .../webapp/META-INF/ignite-webapp-config.xml    |  12 -
 .../java/org/apache/ignite/ml/math/Tracer.java  |  57 ++--
 .../matrix/SparseDistributedMatrixStorage.java  |   3 -
 .../config/cache-query-continuous-default.xml   |   1 -
 .../Config/ignite-config.xml                    |   1 -
 .../Cache/CacheConfigurationTest.cs             |  10 -
 .../Config/Compute/compute-grid1.xml            |   3 -
 .../Config/Compute/compute-grid2.xml            |   1 -
 .../Config/Dynamic/dynamic-data.xml             |   2 -
 .../Config/cache-query-continuous.xml           |   4 -
 .../Config/native-client-test-cache.xml         |   9 -
 .../IgniteConfigurationSerializerTest.cs        |   1 -
 .../Cache/Configuration/CacheConfiguration.cs   |  12 -
 .../Configuration/NearCacheConfiguration.cs     |   2 +-
 .../IgniteConfigurationSection.xsd              |   5 -
 .../http/jetty/GridJettyObjectMapper.java       |  13 +-
 modules/rocketmq/licenses/apache-2.0.txt        | 202 ++++++++++++++
 .../stream/rocketmq/RocketMQStreamerTest.java   |   8 +-
 .../scalar/src/test/resources/spring-cache.xml  |   3 -
 .../java/org/apache/ignite/internal/cache.xml   |   3 -
 .../apache/ignite/internal/filtered-cache.xml   |   3 -
 .../apache/ignite/internal/invalid-cache.xml    |   6 -
 .../visor/commands/ack/VisorAckCommand.scala    |   5 +-
 .../commands/cache/VisorCacheClearCommand.scala |   5 +-
 .../commands/cache/VisorCacheCommand.scala      |   1 -
 .../commands/cache/VisorCacheResetCommand.scala |   4 +-
 .../commands/cache/VisorCacheStopCommand.scala  |   4 +-
 .../config/VisorConfigurationCommand.scala      |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   5 +-
 .../web-console/backend/app/browsersHandler.js  |   3 +-
 modules/web-console/backend/app/mongo.js        |   5 +-
 .../app/modules/agent/AgentManager.service.js   |   3 +-
 .../generator/ConfigurationGenerator.js         |   9 +-
 .../generator/PlatformGenerator.js              |   7 +-
 .../generator/defaults/Cache.service.js         |   1 -
 .../generator/defaults/Cluster.service.js       |   4 +-
 .../frontend/app/modules/sql/sql.controller.js  |   2 +-
 .../states/configuration/caches/memory.pug      |   9 -
 .../states/configuration/clusters/discovery.pug |   8 +-
 .../demo/service/DemoCachesLoadService.java     |   1 -
 .../service/DemoRandomCacheLoadService.java     |   1 -
 .../webapp2/META-INF/ignite-webapp-config.xml   |  12 -
 .../config/ignite-base-load-config.xml          |  31 +++
 parent/pom.xml                                  |   1 +
 200 files changed, 4554 insertions(+), 1433 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ab92ab2e/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 55690b2,d6225c0..cfc267e
--- 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
@@@ -652,14 -705,11 +652,14 @@@ public class GridCacheProcessor extend
  
      /**
       * @param cfg Cache configuration.
 +     * @param caches Caches map.
 +     * @param templates Templates map.
       * @throws IgniteCheckedException If failed.
       */
 -    private void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
 +    private void registerCache(CacheConfiguration cfg,
 +        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
 +        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates) throws IgniteCheckedException {
-         assert cfg.getName() != null;
+         CU.validateCacheName(cfg.getName());
 -
          cloneCheckSerializable(cfg);
  
          CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);


[25/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-5075-cacheStart' into ignite-5075

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-5075-cacheStart' into ignite-5075

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java


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

Branch: refs/heads/ignite-5075
Commit: 019d2f751223dc50fa5dafa3c816e238f0bb7844
Parents: da62feb 193b805
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 17:04:08 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 17:04:08 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       | 34 ++++-----
 .../processors/cache/ClusterCachesInfo.java     | 74 ++++++++++++++------
 .../processors/cache/ExchangeActions.java       | 13 +---
 .../processors/cache/GridCacheProcessor.java    | 74 ++++++++++----------
 .../processors/cache/local/GridLocalCache.java  | 31 +-------
 .../cache/local/GridLocalLockFuture.java        | 41 ++++++++++-
 ...ityFunctionBackupFilterAbstractSelfTest.java | 13 ++--
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  6 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java | 22 ++++--
 9 files changed, 174 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/019d2f75/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 8cc3a5e,45f463b..64c1fd7
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@@ -403,10 -397,10 +399,10 @@@ public class CacheAffinitySharedManager
              }
  
              if (startCache)
-                 cctx.cache().prepareCacheStart(req, nearCfg, action.descriptor(), fut.topologyVersion());
+                 cctx.cache().prepareCacheStart(cacheDesc, nearCfg, fut.topologyVersion());
  
-             if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
+             if (fut.isCacheAdded(cacheDesc.cacheId(), fut.topologyVersion())) {
 -                if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
 +                if (fut.discoCache().cacheGroupAffinityNodes(desc.groupDescriptor().groupId()).isEmpty())
                      U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
              }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/019d2f75/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 88e15ae,1064de3..e539f79
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@@ -426,20 -421,23 +430,23 @@@ class ClusterCachesInfo 
  
                  locJoinStartCaches = new ArrayList<>();
  
-                 for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                     CacheConfiguration cfg = desc.cacheConfiguration();
+                 if (!disconnectedState()) {
 -                    processJoiningNode(joinDiscoData, node.id());
++                    processJoiningNode(joinDiscoData, node.id(), topVer);
  
-                     CacheJoinNodeDiscoveryData.CacheInfo locCfg = joinDiscoData.caches().get(cfg.getName());
+                     for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                         CacheConfiguration cfg = desc.cacheConfiguration();
  
-                     boolean affNode = CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter());
+                         CacheJoinNodeDiscoveryData.CacheInfo locCfg = joinDiscoData.caches().get(cfg.getName());
  
-                     NearCacheConfiguration nearCfg = (!affNode && locCfg != null) ? locCfg.config().getNearConfiguration() : null;
+                         NearCacheConfiguration nearCfg = locCfg != null ? locCfg.config().getNearConfiguration() :
+                             null;
  
-                     if (locCfg != null || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()))
-                         locJoinStartCaches.add(new T2<>(desc, nearCfg));
-                 }
+                         if (locCfg != null || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()))
+                             locJoinStartCaches.add(new T2<>(desc, nearCfg));
+                     }
  
-                 joinDiscoData = null;
+                     joinDiscoData = null;
+                 }
              }
              else {
                  CacheJoinNodeDiscoveryData discoData = joiningNodesDiscoData.remove(node.id());
@@@ -648,6 -604,6 +652,15 @@@
          }
      }
  
++    private CacheGroupDescriptor groupDescriptor(int grpId) {
++        for (CacheGroupDescriptor desc : registeredCacheGrps.values()) {
++            if (desc.groupId() == grpId)
++                return desc;
++        }
++
++        return null;
++    }
++
      /**
       * @param clientData Discovery data.
       * @param clientNodeId Client node ID.
@@@ -667,7 -623,11 +680,11 @@@
          }
      }
  
+     /**
+      * @param joinData Joined node discovery data.
+      * @param nodeId Joined node ID.
+      */
 -    private void processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId) {
 +    private void processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId, AffinityTopologyVersion topVer) {
          for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.templates().values()) {
              CacheConfiguration cfg = cacheInfo.config();
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/019d2f75/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 4c95a24,2ed20e4..93d12ba
--- 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
@@@ -1824,23 -1786,9 +1809,23 @@@ public class GridCacheProcessor extend
          return started != null ? started : Collections.<DynamicCacheDescriptor>emptyList();
      }
  
 +    private CacheGroupInfrastructure startCacheGroup(CacheConfiguration cfg0, int grpId) throws IgniteCheckedException {
 +        CacheConfiguration ccfg = new CacheConfiguration(cfg0);
 +
 +        CacheGroupInfrastructure grp = new CacheGroupInfrastructure(grpId, sharedCtx, ccfg);
 +
 +        grp.start();
 +
 +        CacheGroupInfrastructure old = cacheGrps.put(grpId, grp);
 +
 +        assert old == null;
 +
 +        return grp;
 +    }
 +
      /**
-      * @param cfg Start configuration.
-      * @param nearCfg Near configuration.
+      * @param startCfg Start configuration.
+      * @param reqNearCfg Near configuration if specified for client cache start request.
       * @param cacheType Cache type.
       * @param deploymentId Deployment ID.
       * @param cacheStartTopVer Cache start topology version.
@@@ -1849,45 -1797,36 +1834,57 @@@
       * @throws IgniteCheckedException If failed.
       */
      private void prepareCacheStart(
-         CacheConfiguration cfg,
-         NearCacheConfiguration nearCfg,
+         CacheConfiguration startCfg,
+         @Nullable NearCacheConfiguration reqNearCfg,
          CacheType cacheType,
 +        int grpId,
          IgniteUuid deploymentId,
          AffinityTopologyVersion cacheStartTopVer,
          AffinityTopologyVersion exchTopVer,
          @Nullable QuerySchema schema
      ) throws IgniteCheckedException {
-         assert !caches.containsKey(cfg.getName()) : cfg.getName();
+         assert !caches.containsKey(startCfg.getName()) : startCfg.getName();
  
 +        String grpName = cfg.getGroupName();
 +
 +        CacheGroupInfrastructure grp = null;
 +
 +        if (grpName != null) {
 +            for (CacheGroupInfrastructure grp0 : cacheGrps.values()) {
 +                if (grp0.sharedGroup() && grpName.equals(grp0.groupName())) {
 +                    grp = grp0;
 +
 +                    break;
 +                }
 +            }
 +
 +            if (grp == null)
 +                grp = startCacheGroup(cfg, grpId);
 +        }
 +        else
 +            grp = startCacheGroup(cfg, grpId);
 +
-         CacheConfiguration ccfg = new CacheConfiguration(cfg);
- 
-         if (nearCfg != null)
-             ccfg.setNearConfiguration(nearCfg);
+         CacheConfiguration ccfg = new CacheConfiguration(startCfg);
  
          CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
  
+         boolean affNode;
+ 
+         if (ccfg.getCacheMode() == LOCAL) {
+             affNode = true;
+ 
+             ccfg.setNearConfiguration(null);
+         }
+         else if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter()))
+             affNode = true;
+         else {
+             affNode = false;
+ 
+             ccfg.setNearConfiguration(reqNearCfg);
+         }
+ 
          GridCacheContext cacheCtx = createCache(ccfg,
 +            grp,
              null,
              cacheType,
              cacheStartTopVer,


[12/26] ignite git commit: Moved hibernate-core to lgpl profile.

Posted by sb...@apache.org.
Moved hibernate-core to lgpl profile.


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

Branch: refs/heads/ignite-5075
Commit: e80773bf271394bf5883eae6f4197df01e0befcd
Parents: 0e8e5dd
Author: oleg-ostanin <oo...@gridgain.com>
Authored: Fri Apr 28 17:05:56 2017 +0300
Committer: oleg-ostanin <oo...@gridgain.com>
Committed: Fri Apr 28 17:26:04 2017 +0300

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e80773bf/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9a672cc..722f5eb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -91,7 +91,6 @@
         <module>modules/flink</module>
         <module>modules/kubernetes</module>
         <module>modules/zeromq</module>
-        <module>modules/hibernate-core</module>
         <module>modules/rocketmq</module>
     </modules>
 
@@ -201,6 +200,7 @@
             <modules>
                 <module>modules/hibernate-4.2</module>
                 <module>modules/hibernate-5.1</module>
+                <module>modules/hibernate-core</module>
                 <module>modules/geospatial</module>
                 <module>modules/schedule</module>
             </modules>


[21/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-5075-cacheStart' into ignite-5075

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-5075-cacheStart' into ignite-5075


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

Branch: refs/heads/ignite-5075
Commit: c055276498e223c80793b840e07be3e619ce331f
Parents: c132480 98b4378
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 13:25:20 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 13:25:20 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheClientReconnectDiscoveryData.java    | 6 ++++++
 .../org/apache/ignite/internal/processors/cache/CacheData.java | 3 +++
 .../internal/processors/cache/CacheJoinNodeDiscoveryData.java  | 6 ++++++
 .../processors/cache/CacheNodeCommonDiscoveryData.java         | 3 +++
 .../ignite/internal/processors/cache/ClusterCachesInfo.java    | 1 -
 5 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[24/26] ignite git commit: ignite-5075

Posted by sb...@apache.org.
ignite-5075


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

Branch: refs/heads/ignite-5075
Commit: 193b80580f2a5bdd84f0e73225afc6c59a5a12bd
Parents: 3e6113b
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 16:47:29 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 16:59:47 2017 +0300

----------------------------------------------------------------------
 ...ityFunctionBackupFilterAbstractSelfTest.java | 13 +++++++-----
 ...ePartitionedBasicStoreMultiNodeSelfTest.java | 22 +++++++++++++++-----
 2 files changed, 25 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/193b8058/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionBackupFilterAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionBackupFilterAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionBackupFilterAbstractSelfTest.java
index 2110c28..99e80ca 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionBackupFilterAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionBackupFilterAbstractSelfTest.java
@@ -78,13 +78,13 @@ public abstract class AffinityFunctionBackupFilterAbstractSelfTest extends GridC
 
                 Map<String, Integer> backupAssignedAttribute = getAttributeStatistic(assigned);
 
-                String nodeAttributeValue = node.attribute(SPLIT_ATTRIBUTE_NAME);
+                String nodeAttributeVal = node.attribute(SPLIT_ATTRIBUTE_NAME);
 
-                if (FIRST_NODE_GROUP.equals(nodeAttributeValue)
+                if (FIRST_NODE_GROUP.equals(nodeAttributeVal)
                     && backupAssignedAttribute.get(FIRST_NODE_GROUP) < 2)
                     return true;
 
-                return backupAssignedAttribute.get(nodeAttributeValue).equals(0);
+                return backupAssignedAttribute.get(nodeAttributeVal).equals(0);
             }
         };
 
@@ -107,10 +107,11 @@ public abstract class AffinityFunctionBackupFilterAbstractSelfTest extends GridC
 
             String val = assignedNode.attribute(SPLIT_ATTRIBUTE_NAME);
 
-            Integer count = backupAssignedAttribute.get(val);
+            Integer cnt = backupAssignedAttribute.get(val);
 
-            backupAssignedAttribute.put(val, count + 1);
+            backupAssignedAttribute.put(val, cnt + 1);
         }
+
         return backupAssignedAttribute;
     }
 
@@ -157,6 +158,7 @@ public abstract class AffinityFunctionBackupFilterAbstractSelfTest extends GridC
      */
     public void testPartitionDistribution() throws Exception {
         backups = 1;
+
         try {
             for (int i = 0; i < 3; i++) {
                 splitAttrVal = "A";
@@ -205,6 +207,7 @@ public abstract class AffinityFunctionBackupFilterAbstractSelfTest extends GridC
      */
     public void testPartitionDistributionWithAffinityBackupFilter() throws Exception {
         backups = 3;
+
         try {
             for (int i = 0; i < 2; i++) {
                 splitAttrVal = FIRST_NODE_GROUP;

http://git-wip-us.apache.org/repos/asf/ignite/blob/193b8058/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
index 372da32..a7128e0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
@@ -22,7 +22,9 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import javax.cache.configuration.Factory;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
@@ -102,11 +104,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setBackups(1);
 
-        GridCacheTestStore store = new GridCacheTestStore();
-
-        stores.add(store);
-
-        cc.setCacheStoreFactory(singletonFactory(store));
+        cc.setCacheStoreFactory(new StoreFactory());
         cc.setReadThrough(true);
         cc.setWriteThrough(true);
         cc.setLoadPreviousValue(true);
@@ -269,4 +267,18 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
         assertEquals(expPutAll, putAll);
         assertEquals(expTxs, txs);
     }
+
+    /**
+     *
+     */
+    static class StoreFactory implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            GridCacheTestStore store = new GridCacheTestStore();
+
+            stores.add(store);
+
+            return store;
+        }
+    }
 }
\ No newline at end of file


[18/26] ignite git commit: ignite-5075 cache start

Posted by sb...@apache.org.
ignite-5075 cache start


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

Branch: refs/heads/ignite-5075
Commit: bf2360d5ffd645bbfb015123987fb949be48685d
Parents: e2cf06a
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 10:37:40 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 13:00:05 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       | 45 +++++----
 .../CacheClientReconnectDiscoveryData.java      | 34 ++++++-
 .../internal/processors/cache/CacheData.java    | 19 +++-
 .../cache/CacheJoinNodeDiscoveryData.java       | 39 +++++++-
 .../cache/CacheNodeCommonDiscoveryData.java     | 24 ++++-
 .../processors/cache/ClusterCachesInfo.java     | 13 ++-
 .../cache/DynamicCacheChangeBatch.java          |  4 +-
 .../cache/DynamicCacheChangeRequest.java        | 11 +++
 .../processors/cache/ExchangeActions.java       | 68 +++++++++++---
 .../processors/cache/GridCacheContext.java      | 27 +++---
 .../GridCachePartitionExchangeManager.java      | 20 ++--
 .../processors/cache/GridCacheProcessor.java    | 98 ++++++++++++--------
 .../dht/GridDhtPartitionTopologyImpl.java       | 10 +-
 .../GridDhtPartitionsExchangeFuture.java        |  4 +-
 .../cache/query/GridCacheQueryManager.java      |  3 +-
 .../ignite/spi/discovery/DiscoveryDataBag.java  | 22 ++---
 .../processors/cache/IgniteCacheStartTest.java  | 17 +++-
 .../CacheLateAffinityAssignmentTest.java        |  4 +-
 .../loadtests/hashmap/GridCacheTestContext.java |  3 +
 19 files changed, 329 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 6b99e22..48bc6da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -319,30 +319,35 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         }
     }
 
+    /**
+     * @param exchActions Cache change requests to execte on exchange.
+     */
     private void updateCachesInfo(ExchangeActions exchActions) {
         for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
             Integer cacheId = CU.cacheId(req.cacheName());
 
             DynamicCacheDescriptor desc = registeredCaches.remove(cacheId);
 
-            assert desc != null : cacheId;
+            assert desc != null : req.cacheName();
         }
 
-        for (ExchangeActions.ActionData action : exchActions.newCachesStartRequests()) {
+        for (ExchangeActions.ActionData action : exchActions.newAndClientCachesStartRequests()) {
             DynamicCacheChangeRequest req = action.request();
 
-            Integer cacheId = CU.cacheId(req.cacheName());
+            if (!req.clientStartOnly()) {
+                Integer cacheId = CU.cacheId(req.cacheName());
 
-            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(),
-                req.startCacheConfiguration(),
-                req.cacheType(),
-                false,
-                req.deploymentId(),
-                req.schema());
+                DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(),
+                    req.startCacheConfiguration(),
+                    req.cacheType(),
+                    false,
+                    req.deploymentId(),
+                    req.schema());
 
-            DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc);
+                DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc);
 
-            assert old == null : old;
+                assert old == null : old;
+            }
         }
     }
 
@@ -411,7 +416,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                     if (clientCacheStarted)
                         initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign);
                     else if (!req.clientStartOnly()) {
-                        assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion());
+                        assert fut.topologyVersion().equals(cacheCtx.cacheStartTopologyVersion());
 
                         GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache();
 
@@ -600,7 +605,11 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
                 assert affTopVer.topologyVersion() > 0 : affTopVer;
 
-                IgniteUuid deploymentId = registeredCaches.get(aff.cacheId()).deploymentId();
+                DynamicCacheDescriptor desc = registeredCaches.get(aff.cacheId());
+
+                assert desc != null : aff.cacheName();
+
+                IgniteUuid deploymentId = desc.deploymentId();
 
                 if (!deploymentId.equals(deploymentIds.get(aff.cacheId()))) {
                     aff.clientEventTopologyChange(exchFut.discoveryEvent(), topVer);
@@ -804,12 +813,10 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      */
     public void initStartedCaches(boolean crd,
         final GridDhtPartitionsExchangeFuture fut,
-        @Nullable Collection<DynamicCacheDescriptor> descs) throws IgniteCheckedException {
-        if (descs != null) {
-            for (DynamicCacheDescriptor desc : descs) {
-                if (!registeredCaches.containsKey(desc.cacheId()))
-                    registeredCaches.put(desc.cacheId(), desc);
-            }
+        Collection<DynamicCacheDescriptor> descs) throws IgniteCheckedException {
+        for (DynamicCacheDescriptor desc : descs) {
+            if (!registeredCaches.containsKey(desc.cacheId()))
+                registeredCaches.put(desc.cacheId(), desc);
         }
 
         if (crd && lateAffAssign) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
index b791e35..172547d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
@@ -18,13 +18,14 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 
 import java.io.Serializable;
 import java.util.Map;
 
 /**
- *
+ * Discovery data sent from client reconnecting to cluster.
  */
 public class CacheClientReconnectDiscoveryData implements Serializable {
     /** */
@@ -60,9 +61,16 @@ public class CacheClientReconnectDiscoveryData implements Serializable {
         /** */
         private final boolean nearCache;
 
-        /** */
+        /** Flags added for future usage. */
         private final byte flags;
 
+        /**
+         * @param ccfg Cache configuration.
+         * @param cacheType Cache type.
+         * @param deploymentId Cache deployment ID.
+         * @param nearCache Near cache flag.
+         * @param flags Flags (for future usage).
+         */
         public CacheInfo(CacheConfiguration ccfg,
             CacheType cacheType,
             IgniteUuid deploymentId,
@@ -79,20 +87,42 @@ public class CacheClientReconnectDiscoveryData implements Serializable {
             this.flags = flags;
         }
 
+        /**
+         * @return Cache configuraiton.
+         */
         CacheConfiguration config() {
             return ccfg;
         }
 
+        /**
+         * @return Cache type.
+         */
         CacheType cacheType() {
             return cacheType;
         }
 
+        /**
+         * @return Cache deployment ID.
+         */
         IgniteUuid deploymentId() {
             return deploymentId;
         }
 
+        /**
+         * @return Near cache flag.
+         */
         boolean nearCache() {
             return nearCache;
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(CacheInfo.class, this);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheClientReconnectDiscoveryData.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
index b38e03f..43422da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -56,6 +56,21 @@ public class CacheData implements Serializable {
     /** */
     private final boolean template;
 
+    /** Flags added for future usage. */
+    private final byte flags;
+
+    /**
+     * @param cacheCfg Cache configuration.
+     * @param cacheId Cache ID.
+     * @param cacheType Cache ID.
+     * @param startTopVer Topology version when cache was started.
+     * @param deploymentId Cache deployment ID.
+     * @param schema Query schema.
+     * @param rcvdFrom Node ID cache was started from.
+     * @param staticCfg {@code True} if cache was statically configured.
+     * @param template {@code True} if this is cache template.
+     * @param flags Flags (added for future usage).
+     */
     CacheData(CacheConfiguration cacheCfg,
         int cacheId,
         CacheType cacheType,
@@ -64,7 +79,8 @@ public class CacheData implements Serializable {
         QuerySchema schema,
         UUID rcvdFrom,
         boolean staticCfg,
-        boolean template) {
+        boolean template,
+        byte flags) {
         assert cacheCfg != null;
         assert rcvdFrom != null : cacheCfg.getName();
         assert startTopVer != null : cacheCfg.getName();
@@ -80,6 +96,7 @@ public class CacheData implements Serializable {
         this.rcvdFrom = rcvdFrom;
         this.staticCfg = staticCfg;
         this.template = template;
+        this.flags = flags;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
index e04e739..94e5736 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.processors.cache;
 import java.io.Serializable;
 import java.util.Map;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 
 /**
@@ -27,12 +29,15 @@ import org.apache.ignite.lang.IgniteUuid;
  */
 class CacheJoinNodeDiscoveryData implements Serializable {
     /** */
+    @GridToStringInclude
     private final Map<String, CacheInfo> caches;
 
     /** */
+    @GridToStringInclude
     private final Map<String, CacheInfo> templates;
 
     /** */
+    @GridToStringInclude
     private final IgniteUuid cacheDeploymentId;
 
     /**
@@ -49,14 +54,23 @@ class CacheJoinNodeDiscoveryData implements Serializable {
         this.templates = templates;
     }
 
+    /**
+     * @return Deployment ID assigned on joining node.
+     */
     IgniteUuid cacheDeploymentId() {
         return cacheDeploymentId;
     }
 
+    /**
+     * @return Templates configured on joining node.
+     */
     Map<String, CacheInfo> templates() {
         return templates;
     }
 
+    /**
+     * @return Caches configured on joining node.
+     */
     Map<String, CacheInfo> caches() {
         return caches;
     }
@@ -66,26 +80,49 @@ class CacheJoinNodeDiscoveryData implements Serializable {
      */
     static class CacheInfo implements Serializable {
         /** */
+        @GridToStringInclude
         private final CacheConfiguration ccfg;
 
         /** */
+        @GridToStringInclude
         private final CacheType cacheType;
 
-        /** */
+        /** Flags added for future usage. */
         private final byte flags;
 
+        /**
+         * @param ccfg Cache configuration.
+         * @param cacheType Cache type.
+         * @param flags Flags (for future usage).
+         */
         CacheInfo(CacheConfiguration ccfg, CacheType cacheType, byte flags) {
             this.ccfg = ccfg;
             this.cacheType = cacheType;
             this.flags = flags;
         }
 
+        /**
+         * @return Cache configuration.
+         */
         CacheConfiguration config() {
             return ccfg;
         }
 
+        /**
+         * @return Cache type.
+         */
         CacheType cacheType() {
             return cacheType;
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(CacheInfo.class, this);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheJoinNodeDiscoveryData.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
index a2a7b30..0109b3b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
@@ -20,24 +20,28 @@ package org.apache.ignite.internal.processors.cache;
 import java.io.Serializable;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * Cache information sent in discovery data to joining node.
  */
 class CacheNodeCommonDiscoveryData implements Serializable {
     /** */
+    @GridToStringInclude
     private final Map<String, CacheData> caches;
 
     /** */
+    @GridToStringInclude
     private final Map<String, CacheData> templates;
 
     /** */
     private final Map<String, Map<UUID, Boolean>> clientNodesMap;
 
     /**
-     * @param caches
-     * @param templates
-     * @param clientNodesMap
+     * @param caches Started caches.
+     * @param templates Configured templates.
+     * @param clientNodesMap Information about cache client nodes.
      */
     CacheNodeCommonDiscoveryData(Map<String, CacheData> caches,
         Map<String, CacheData> templates,
@@ -47,15 +51,29 @@ class CacheNodeCommonDiscoveryData implements Serializable {
         this.clientNodesMap = clientNodesMap;
     }
 
+    /**
+     * @return Started caches.
+     */
     Map<String, CacheData> caches() {
         return caches;
     }
 
+    /**
+     * @return Configured templates.
+     */
     Map<String, CacheData> templates() {
         return templates;
     }
 
+    /**
+     * @return Information about cache client nodes.
+     */
     Map<String, Map<UUID, Boolean>> clientNodesMap() {
         return clientNodesMap;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheNodeCommonDiscoveryData.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 6cc09a0..645881f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -378,15 +378,12 @@ class ClusterCachesInfo {
 
         List<DynamicCacheDescriptor> started = null;
 
-        if (!ctx.clientNode() && !ctx.isDaemon()) {
+        if (!ctx.isDaemon()) {
             for (DynamicCacheDescriptor desc : registeredCaches.values()) {
                 if (desc.staticallyConfigured()) {
                     assert desc.receivedFrom() != null : desc;
 
-                    IgnitePredicate<ClusterNode> filter = desc.cacheConfiguration().getNodeFilter();
-
-                    if (joinedNodeId.equals(desc.receivedFrom()) &&
-                        CU.affinityNode(ctx.discovery().localNode(), filter)) {
+                    if (joinedNodeId.equals(desc.receivedFrom())) {
                         if (started == null)
                             started = new ArrayList<>();
 
@@ -486,7 +483,8 @@ class ClusterCachesInfo {
                 desc.schema(),
                 desc.receivedFrom(),
                 desc.staticallyConfigured(),
-                false);
+                false,
+                (byte)0);
 
             caches.put(desc.cacheName(), cacheData);
         }
@@ -502,7 +500,8 @@ class ClusterCachesInfo {
                 desc.schema(),
                 desc.receivedFrom(),
                 desc.staticallyConfigured(),
-                true);
+                true,
+                (byte)0);
 
             templates.put(desc.cacheName(), cacheData);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index e27d5af..3c65326 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -32,7 +32,7 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Custom message ID. */
+    /** Discovery custom message ID. */
     private IgniteUuid id = IgniteUuid.randomUuid();
 
     /** Change requests. */
@@ -91,7 +91,7 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
      * @param exchangeActions Cache updates to be executed on exchange.
      */
     void exchangeActions(ExchangeActions exchangeActions) {
-        assert !exchangeActions.empty() : exchangeActions;
+        assert exchangeActions != null && !exchangeActions.empty() : exchangeActions;
 
         this.exchangeActions = exchangeActions;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/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 e4c95a7..8fa763c 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
@@ -136,6 +136,11 @@ public class DynamicCacheChangeRequest implements Serializable {
         return req;
     }
 
+    /**
+     * @param ctx Context.
+     * @param cacheName Cache name.
+     * @return Request to close client cache.
+     */
     static DynamicCacheChangeRequest closeRequest(GridKernalContext ctx, String cacheName) {
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 
@@ -144,6 +149,12 @@ public class DynamicCacheChangeRequest implements Serializable {
         return req;
     }
 
+    /**
+     * @param ctx Context.
+     * @param cacheName Cache name.
+     * @param destroy Destroy flag.
+     * @return Cache stop request.
+     */
     static DynamicCacheChangeRequest stopRequest(GridKernalContext ctx, String cacheName, boolean destroy) {
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index 6de02b8..9be4b6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -31,7 +31,7 @@ import java.util.List;
 import org.jetbrains.annotations.Nullable;
 
 /**
- *
+ * Cache change requests to execute on request.
  */
 public class ExchangeActions {
     /** */
@@ -52,6 +52,9 @@ public class ExchangeActions {
     /** */
     private ClusterState newState;
 
+    /**
+     * @return {@code True} if server nodes should not participate in exchange.
+     */
     public boolean clientOnlyExchange() {
         return F.isEmpty(cachesToStart) &&
             F.isEmpty(cachesToStop) &&
@@ -99,13 +102,6 @@ public class ExchangeActions {
     }
 
     /**
-     * @return Start cache requests.
-     */
-    Collection<ActionData> newCachesStartRequests() {
-        return cachesToStart != null ? cachesToStart.values() : Collections.<ActionData>emptyList();
-    }
-
-    /**
      * @return Stop cache requests.
      */
     public List<DynamicCacheChangeRequest> stopRequests() {
@@ -132,6 +128,10 @@ public class ExchangeActions {
         completeRequestFutures(cachesToResetLostParts, ctx);
     }
 
+    /**
+     * @param map Actions map.
+     * @param ctx Context.
+     */
     private void completeRequestFutures(Map<String, ActionData> map, GridCacheSharedContext ctx) {
         if (map != null) {
             for (ActionData req : map.values())
@@ -147,7 +147,7 @@ public class ExchangeActions {
     }
 
     /**
-     * @return
+     * @return Caches to reset lost partitions for.
      */
     public Set<String> cachesToResetLostPartitions() {
         Set<String> caches = null;
@@ -157,7 +157,11 @@ public class ExchangeActions {
 
         return caches != null ? caches : Collections.<String>emptySet();
     }
-    
+
+    /**
+     * @param cacheId Cache ID.
+     * @return {@code True} if cache stop was requested.
+     */
     public boolean cacheStopped(int cacheId) {
         if (cachesToStop != null) {
             for (ActionData cache : cachesToStop.values()) {
@@ -169,6 +173,10 @@ public class ExchangeActions {
         return false;
     }
 
+    /**
+     * @param cacheId Cache ID.
+     * @return {@code True} if cache start was requested.
+     */
     public boolean cacheStarted(int cacheId) {
         if (cachesToStart != null) {
             for (ActionData cache : cachesToStart.values()) {
@@ -202,7 +210,15 @@ public class ExchangeActions {
         return newState;
     }
 
-    private Map<String, ActionData> add(Map<String, ActionData> map, DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+    /**
+     * @param map Actions map.
+     * @param req Request.
+     * @param desc Cache descriptor.
+     * @return Actions map.
+     */
+    private Map<String, ActionData> add(Map<String, ActionData> map,
+        DynamicCacheChangeRequest req,
+        DynamicCacheDescriptor desc) {
         assert req != null;
         assert desc != null;
 
@@ -216,30 +232,50 @@ public class ExchangeActions {
         return map;
     }
 
+    /**
+     * @param req Request.
+     * @param desc Cache descriptor.
+     */
     void addCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
         assert req.start() : req;
 
         cachesToStart = add(cachesToStart, req, desc);
     }
 
+    /**
+     * @param req Request.
+     * @param desc Cache descriptor.
+     */
     void addClientCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
         assert req.start() : req;
 
         clientCachesToStart = add(clientCachesToStart, req, desc);
     }
 
+    /**
+     * @param req Request.
+     * @param desc Cache descriptor.
+     */
     void addCacheToStop(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
         assert req.stop() : req;
 
         cachesToStop = add(cachesToStop, req, desc);
     }
 
+    /**
+     * @param req Request.
+     * @param desc Cache descriptor.
+     */
     void addCacheToClose(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
         assert req.close() : req;
 
         cachesToClose = add(cachesToClose, req, desc);
     }
 
+    /**
+     * @param req Request.
+     * @param desc Cache descriptor.
+     */
     void addCacheToResetLostPartitions(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
         assert req.resetLostPartitions() : req;
 
@@ -267,6 +303,10 @@ public class ExchangeActions {
         /** */
         private DynamicCacheDescriptor desc;
 
+        /**
+         * @param req Request.
+         * @param desc Cache descriptor.
+         */
         ActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
             assert req != null;
             assert desc != null;
@@ -275,10 +315,16 @@ public class ExchangeActions {
             this.desc = desc;
         }
 
+        /**
+         * @return Request.
+         */
         public DynamicCacheChangeRequest request() {
             return req;
         }
 
+        /**
+         * @return Cache descriptor.
+         */
         public DynamicCacheDescriptor descriptor() {
             return desc;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 72adeaf..6cb8907 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -235,7 +235,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     private CountDownLatch startLatch = new CountDownLatch(1);
 
     /** Topology version when cache was started on local node. */
-    private AffinityTopologyVersion startTopVer;
+    private AffinityTopologyVersion locStartTopVer;
 
     /** Global cache start topology version. */
     private AffinityTopologyVersion cacheStartTopVer;
@@ -292,6 +292,8 @@ public class GridCacheContext<K, V> implements Externalizable {
         GridCacheSharedContext sharedCtx,
         CacheConfiguration cacheCfg,
         CacheType cacheType,
+        AffinityTopologyVersion cacheStartTopVer,
+        AffinityTopologyVersion locStartTopVer,
         boolean affNode,
         boolean updatesAllowed,
         MemoryPolicy memPlc,
@@ -319,6 +321,8 @@ public class GridCacheContext<K, V> implements Externalizable {
         assert ctx != null;
         assert sharedCtx != null;
         assert cacheCfg != null;
+        assert cacheStartTopVer != null : cacheCfg.getName();
+        assert locStartTopVer != null : cacheCfg.getName();
 
         assert evtMgr != null;
         assert storeMgr != null;
@@ -336,6 +340,8 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.sharedCtx = sharedCtx;
         this.cacheCfg = cacheCfg;
         this.cacheType = cacheType;
+        this.locStartTopVer = locStartTopVer;
+        this.cacheStartTopVer = cacheStartTopVer;
         this.affNode = affNode;
         this.updatesAllowed = updatesAllowed;
         this.depEnabled = ctx.deploy().enabled() && !cacheObjects().isBinaryEnabled(cacheCfg);
@@ -448,24 +454,21 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
-     * @return Start topology version.
+     * @return Topology version when cache was started on local node.
      */
     public AffinityTopologyVersion startTopologyVersion() {
-        return startTopVer;
-    }
+        assert locStartTopVer != null : name();
 
-    /**
-     * @param startTopVer Start topology version.
-     */
-    public void startTopologyVersion(AffinityTopologyVersion startTopVer) {
-        this.startTopVer = startTopVer;
+        return locStartTopVer;
     }
 
     /**
-     * @param cacheStartTopVer Global cache start topology version.
+     * @return Cache start topology version.
      */
-    public void cacheStartTopologyVersion(AffinityTopologyVersion cacheStartTopVer) {
-        this.cacheStartTopVer = cacheStartTopVer;
+    public AffinityTopologyVersion cacheStartTopologyVersion() {
+        assert cacheStartTopVer != null : name();
+
+        return cacheStartTopVer;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/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 c2b0e27..8f52ae6 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
@@ -363,10 +363,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         assert startTime > 0;
 
         // Generate dummy discovery event for local node joining.
-        T2<DiscoveryEvent, DiscoCache> localJoin = cctx.discovery().localJoin();
+        T2<DiscoveryEvent, DiscoCache> locJoin = cctx.discovery().localJoin();
 
-        DiscoveryEvent discoEvt = localJoin.get1();
-        DiscoCache discoCache = localJoin.get2();
+        DiscoveryEvent discoEvt = locJoin.get1();
+        DiscoCache discoCache = locJoin.get2();
 
         GridDhtPartitionExchangeId exchId = initialExchangeId();
 
@@ -466,8 +466,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 }
             }
 
+            AffinityTopologyVersion nodeStartVer = new AffinityTopologyVersion(discoEvt.topologyVersion(), 0);
+
             for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                if (cacheCtx.startTopologyVersion() == null)
+                if (nodeStartVer.equals(cacheCtx.startTopologyVersion()))
                     cacheCtx.preloader().onInitialExchangeComplete(null);
             }
 
@@ -893,9 +895,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     boolean ready;
 
                     if (exchId != null) {
-                        AffinityTopologyVersion startTopVer = cacheCtx.startTopologyVersion();
+                        AffinityTopologyVersion startTopVer = cacheCtx.cacheStartTopologyVersion();
 
-                        ready = startTopVer == null || startTopVer.compareTo(exchId.topologyVersion()) <= 0;
+                        ready = startTopVer.compareTo(exchId.topologyVersion()) <= 0;
                     }
                     else
                         ready = cacheCtx.started();
@@ -1298,10 +1300,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                     GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
 
-                    if (cacheCtx != null && cacheCtx.startTopologyVersion() != null &&
-                        entry.getValue() != null &&
-                        entry.getValue().topologyVersion() != null && // Backward compatibility.
-                        cacheCtx.startTopologyVersion().compareTo(entry.getValue().topologyVersion()) > 0)
+                    if (cacheCtx != null &&
+                        cacheCtx.cacheStartTopologyVersion().compareTo(entry.getValue().topologyVersion()) > 0)
                         continue;
 
                     GridDhtPartitionTopology top = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/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 0951676..b4e4b14 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
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -26,7 +25,6 @@ import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.IdentityHashMap;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.ListIterator;
@@ -60,13 +58,6 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.query.QuerySchema;
-import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.processors.query.schema.message.SchemaAbstractDiscoveryMessage;
-import org.apache.ignite.internal.processors.query.schema.SchemaExchangeWorkerTask;
-import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask;
-import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage;
-import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -109,7 +100,14 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
+import org.apache.ignite.internal.processors.query.QuerySchema;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.schema.SchemaExchangeWorkerTask;
+import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask;
+import org.apache.ignite.internal.processors.query.schema.message.SchemaAbstractDiscoveryMessage;
+import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -842,7 +840,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         ctx.service().onUtilityCacheStarted();
 
-        AffinityTopologyVersion startTopVer = new AffinityTopologyVersion(locNode.order(), 0);
+        AffinityTopologyVersion startTopVer =
+            new AffinityTopologyVersion(ctx.discovery().localJoinEvent().topologyVersion(), 0);
 
         for (GridCacheAdapter cache : caches.values()) {
             CacheConfiguration cfg = cache.configuration();
@@ -1336,6 +1335,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private GridCacheContext createCache(CacheConfiguration<?, ?> cfg,
         @Nullable CachePluginManager pluginMgr,
         CacheType cacheType,
+        AffinityTopologyVersion cacheStartTopVer,
+        AffinityTopologyVersion locStartTopVer,
         CacheObjectContext cacheObjCtx,
         boolean updatesAllowed)
         throws IgniteCheckedException {
@@ -1410,6 +1411,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             sharedCtx,
             cfg,
             cacheType,
+            cacheStartTopVer,
+            locStartTopVer,
             affNode,
             updatesAllowed,
             memPlc,
@@ -1541,6 +1544,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 sharedCtx,
                 cfg,
                 cacheType,
+                cacheStartTopVer,
+                locStartTopVer,
                 affNode,
                 true,
                 memPlc,
@@ -1733,23 +1738,34 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * @param exchTopVer Current exchange version.
      * @throws IgniteCheckedException If failed.
+     * @return Collection of started caches.
      */
-    public void startCachesOnLocalJoin(AffinityTopologyVersion exchTopVer) throws IgniteCheckedException {
+    public List<DynamicCacheDescriptor> startCachesOnLocalJoin(AffinityTopologyVersion exchTopVer) throws IgniteCheckedException {
         List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> caches = cachesInfo.cachesToStartOnLocalJoin();
 
-        for (T2<DynamicCacheDescriptor, NearCacheConfiguration> t : caches) {
-            DynamicCacheDescriptor desc = t.get1();
-
-            prepareCacheStart(
-                desc.cacheConfiguration(),
-                t.get2(),
-                desc.cacheType(),
-                desc.deploymentId(),
-                desc.startTopologyVersion(),
-                exchTopVer,
-                desc.schema()
-            );
+        if (!F.isEmpty(caches)) {
+            List<DynamicCacheDescriptor> started = new ArrayList<>(caches.size());
+
+            for (T2<DynamicCacheDescriptor, NearCacheConfiguration> t : caches) {
+                DynamicCacheDescriptor desc = t.get1();
+
+                prepareCacheStart(
+                    desc.cacheConfiguration(),
+                    t.get2(),
+                    desc.cacheType(),
+                    desc.deploymentId(),
+                    desc.startTopologyVersion(),
+                    exchTopVer,
+                    desc.schema()
+                );
+
+                started.add(desc);
+            }
+
+            return started;
         }
+        else
+            return Collections.emptyList();
     }
 
     /**
@@ -1766,19 +1782,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         if (started != null) {
             for (DynamicCacheDescriptor desc : started) {
-                prepareCacheStart(
-                    desc.cacheConfiguration(),
-                    null,
-                    desc.cacheType(),
-                    desc.deploymentId(),
-                    desc.startTopologyVersion(),
-                    exchTopVer,
-                    desc.schema()
-                );
+                IgnitePredicate<ClusterNode> filter = desc.cacheConfiguration().getNodeFilter();
+
+                if (CU.affinityNode(ctx.discovery().localNode(), filter)) {
+                    prepareCacheStart(
+                        desc.cacheConfiguration(),
+                        null,
+                        desc.cacheType(),
+                        desc.deploymentId(),
+                        desc.startTopologyVersion(),
+                        exchTopVer,
+                        desc.schema()
+                    );
+                }
             }
         }
 
-        return started;
+        return started != null ? started : Collections.<DynamicCacheDescriptor>emptyList();
     }
 
     /**
@@ -1809,11 +1829,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-        GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx, true);
-
-        cacheCtx.startTopologyVersion(exchTopVer);
-
-        cacheCtx.cacheStartTopologyVersion(cacheStartTopVer);
+        GridCacheContext cacheCtx = createCache(ccfg,
+            null,
+            cacheType,
+            cacheStartTopVer,
+            exchTopVer,
+            cacheObjCtx,
+            true);
 
         cacheCtx.dynamicDeploymentId(deploymentId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 38cbb8c..5b3dfc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -393,7 +393,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         int num = cctx.affinity().partitions();
 
         if (cctx.rebalanceEnabled()) {
-            boolean added = exchFut.isCacheAdded(cctx.cacheId(), exchId.topologyVersion());
+            boolean added = exchId.topologyVersion().equals(cctx.cacheStartTopologyVersion());
 
             boolean first = (loc.equals(oldest) && loc.id().equals(exchId.nodeId()) && exchId.isJoined()) || added;
 
@@ -1156,10 +1156,8 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     // If for some nodes current partition has a newer map,
                     // then we keep the newer value.
                     if (newPart != null &&
-                        (newPart.updateSequence() < part.updateSequence() || (
-                            cctx.startTopologyVersion() != null &&
-                                newPart.topologyVersion() != null && // Backward compatibility.
-                                cctx.startTopologyVersion().compareTo(newPart.topologyVersion()) > 0))
+                        (newPart.updateSequence() < part.updateSequence() ||
+                        (cctx.cacheStartTopologyVersion().compareTo(newPart.topologyVersion()) > 0))
                         ) {
                         if (log.isDebugEnabled())
                             log.debug("Overriding partition map in full update map [exchId=" + exchId + ", curPart=" +
@@ -1169,7 +1167,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     }
                 }
 
-                //remove entry if node left
+                // Remove entry if node left.
                 for (Iterator<UUID> it = partMap.keySet().iterator(); it.hasNext(); ) {
                     UUID nodeId = it.next();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/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 320480c..bcfd8e0 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
@@ -634,8 +634,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             GridDhtPartitionTopology top = cacheCtx.topology();
 
             if (crd) {
-                boolean updateTop = !cacheCtx.isLocal() &&
-                    exchId.topologyVersion().equals(cacheCtx.startTopologyVersion());
+                boolean updateTop = exchId.topologyVersion().equals(cacheCtx.startTopologyVersion());
 
                 if (updateTop && clientTop != null)
                     top.update(exchId, clientTop.partitionMap(true), clientTop.updateCounters(false));
@@ -737,7 +736,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     private void clientOnlyExchange() throws IgniteCheckedException {
         clientOnlyExchange = true;
 
-        //todo checl invoke on client
         if (crd != null) {
             if (crd.isLocal()) {
                 for (GridCacheContext cacheCtx : cctx.cacheContexts()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 0f7b0df..fcf534c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -272,8 +272,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         qryTopVer = cctx.startTopologyVersion();
 
-        if (qryTopVer == null)
-            qryTopVer = new AffinityTopologyVersion(cctx.localNode().order(), 0);
+        assert qryTopVer != null : cctx.name();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
index 1ca4ac7..3737d69 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
@@ -188,14 +188,15 @@ public class DiscoveryDataBag {
     }
 
     /**
-     *
+     * @return ID on joining node.
      */
     public UUID joiningNodeId() {
         return joiningNodeId;
     }
 
     /**
-     * @param cmpId component ID.
+     * @param cmpId Component ID.
+     * @return Discovery data for given component.
      */
     public GridDiscoveryData gridDiscoveryData(int cmpId) {
         if (gridData == null)
@@ -207,7 +208,8 @@ public class DiscoveryDataBag {
     }
 
     /**
-     * @param cmpId component ID.
+     * @param cmpId Component ID.
+     * @return Joining node discovery data.
      */
     public JoiningNodeDiscoveryData newJoinerDiscoveryData(int cmpId) {
         if (newJoinerData == null)
@@ -218,12 +220,8 @@ public class DiscoveryDataBag {
         return newJoinerData;
     }
 
-    void collectJoiningNodeData(DiscoveryDataBag dataBag) {
-
-    }
-
     /**
-     * @param cmpId component ID.
+     * @param cmpId Component ID.
      * @param data Data.
      */
     public void addJoiningNodeData(Integer cmpId, Serializable data) {
@@ -231,7 +229,7 @@ public class DiscoveryDataBag {
     }
 
     /**
-     * @param cmpId component ID.
+     * @param cmpId Component ID.
      * @param data Data.
      */
     public void addGridCommonData(Integer cmpId, Serializable data) {
@@ -239,7 +237,7 @@ public class DiscoveryDataBag {
     }
 
     /**
-     * @param cmpId component ID.
+     * @param cmpId Component ID.
      * @param data Data.
      */
     public void addNodeSpecificData(Integer cmpId, Serializable data) {
@@ -250,7 +248,8 @@ public class DiscoveryDataBag {
     }
 
     /**
-     * @param cmpId component ID.
+     * @param cmpId Component ID.
+     * @return {@code True} if common data collected for given component.
      */
     public boolean commonDataCollectedFor(Integer cmpId) {
         assert cmnDataInitializedCmps != null;
@@ -299,5 +298,4 @@ public class DiscoveryDataBag {
     @Nullable public Map<Integer, Serializable> localNodeSpecificData() {
         return nodeSpecificData.get(DEFAULT_KEY);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java
index da34424..21b8d0f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java
@@ -85,6 +85,12 @@ public class IgniteCacheStartTest extends GridCommonAbstractTest {
         checkCache(0, "c1", true);
         checkCache(1, "c1", true);
         checkCache(2, "c1", false);
+
+        ignite(2).destroyCache("c1");
+
+        checkCache(0, "c1", false);
+        checkCache(1, "c1", false);
+        checkCache(2, "c1", false);
     }
 
     /**
@@ -144,6 +150,11 @@ public class IgniteCacheStartTest extends GridCommonAbstractTest {
         checkCache(3, "c1", true);
         checkCache(4, "c1", true);
         checkCache(5, "c1", false);
+
+        ignite(5).destroyCache("c1");
+
+        for (int i = 0; i < 5; i++)
+            checkCache(i, "c1", false);
     }
 
     /**
@@ -151,11 +162,7 @@ public class IgniteCacheStartTest extends GridCommonAbstractTest {
      * @return Cache configuration.
      */
     private CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        ccfg.setName(cacheName);
-
-        return ccfg;
+        return new CacheConfiguration(cacheName);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
index fed388a..bc435e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
@@ -2165,7 +2165,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
         Map<String, List<List<ClusterNode>>> aff = new HashMap<>();
 
         for (Ignite node : nodes) {
-            log.info("Check node: " + node.name());
+            log.info("Check affinity [node=" + node.name() + ", topVer=" + topVer + ", expIdeal=" + expIdeal + ']');
 
             IgniteKernal node0 = (IgniteKernal)node;
 
@@ -2175,7 +2175,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
                 fut.get();
 
             for (GridCacheContext cctx : node0.context().cache().context().cacheContexts()) {
-                if (cctx.startTopologyVersion() != null && cctx.startTopologyVersion().compareTo(topVer) > 0)
+                if (cctx.startTopologyVersion().compareTo(topVer) > 0)
                     continue;
 
                 List<List<ClusterNode>> aff1 = aff.get(cctx.name());

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2360d5/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 0f4aa87..6149586 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -20,6 +20,7 @@ package org.apache.ignite.loadtests.hashmap;
 import java.util.IdentityHashMap;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager;
 import org.apache.ignite.internal.processors.cache.CacheOsConflictResolutionManager;
 import org.apache.ignite.internal.processors.cache.CacheType;
@@ -78,6 +79,8 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
             ),
             defaultCacheConfiguration(),
             CacheType.USER,
+            AffinityTopologyVersion.ZERO,
+            AffinityTopologyVersion.ZERO,
             true,
             true,
             null,


[15/26] ignite git commit: IGNITE-5120: Fixed build of Hibernate module.

Posted by sb...@apache.org.
IGNITE-5120: Fixed build of Hibernate module.


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

Branch: refs/heads/ignite-5075
Commit: cd46d46ca91e1921bff2036c450af071496d1c68
Parents: 777078d
Author: oleg-ostanin <oo...@gridgain.com>
Authored: Sat Apr 29 22:04:19 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Sat Apr 29 22:04:19 2017 +0300

----------------------------------------------------------------------
 modules/hibernate-core/pom.xml | 8 ++++++++
 pom.xml                        | 2 +-
 2 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cd46d46c/modules/hibernate-core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate-core/pom.xml b/modules/hibernate-core/pom.xml
index 91ec68b..9131ad3 100644
--- a/modules/hibernate-core/pom.xml
+++ b/modules/hibernate-core/pom.xml
@@ -71,6 +71,14 @@
                 <groupId>org.apache.felix</groupId>
                 <artifactId>maven-bundle-plugin</artifactId>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <version>2.8.2</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd46d46c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5fd1005..8d388a5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,6 +80,7 @@
         <module>modules/twitter</module>
         <module>modules/mqtt</module>
         <module>modules/zookeeper</module>
+        <module>modules/hibernate-core</module>
         <module>modules/camel</module>
         <module>modules/storm</module>
         <module>modules/osgi-paxlogging</module>
@@ -198,7 +199,6 @@
         <profile>
             <id>lgpl</id>
             <modules>
-                <module>modules/hibernate-core</module>
                 <module>modules/hibernate-4.2</module>
                 <module>modules/hibernate-5.1</module>
                 <module>modules/geospatial</module>


[20/26] ignite git commit: ignite-5075

Posted by sb...@apache.org.
ignite-5075


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

Branch: refs/heads/ignite-5075
Commit: 98b4378d4c216dd8c072bf066973a352e02fc597
Parents: bf2360d
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 13:21:58 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 13:21:58 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheClientReconnectDiscoveryData.java    | 6 ++++++
 .../org/apache/ignite/internal/processors/cache/CacheData.java | 3 +++
 .../internal/processors/cache/CacheJoinNodeDiscoveryData.java  | 6 ++++++
 .../processors/cache/CacheNodeCommonDiscoveryData.java         | 3 +++
 .../ignite/internal/processors/cache/ClusterCachesInfo.java    | 1 -
 5 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/98b4378d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
index 172547d..f970469 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
@@ -29,6 +29,9 @@ import java.util.Map;
  */
 public class CacheClientReconnectDiscoveryData implements Serializable {
     /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
     private final Map<String, CacheInfo> clientCaches;
 
     /**
@@ -50,6 +53,9 @@ public class CacheClientReconnectDiscoveryData implements Serializable {
      */
     static class CacheInfo implements Serializable {
         /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
         private final CacheConfiguration ccfg;
 
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/98b4378d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
index 43422da..82afdc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -30,6 +30,9 @@ import org.apache.ignite.lang.IgniteUuid;
  */
 public class CacheData implements Serializable {
     /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
     private final CacheConfiguration cacheCfg;
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/98b4378d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
index 94e5736..c569818 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
@@ -29,6 +29,9 @@ import org.apache.ignite.lang.IgniteUuid;
  */
 class CacheJoinNodeDiscoveryData implements Serializable {
     /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
     @GridToStringInclude
     private final Map<String, CacheInfo> caches;
 
@@ -80,6 +83,9 @@ class CacheJoinNodeDiscoveryData implements Serializable {
      */
     static class CacheInfo implements Serializable {
         /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
         @GridToStringInclude
         private final CacheConfiguration ccfg;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/98b4378d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
index 0109b3b..84a33dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
@@ -28,6 +28,9 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  */
 class CacheNodeCommonDiscoveryData implements Serializable {
     /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
     @GridToStringInclude
     private final Map<String, CacheData> caches;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/98b4378d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 645881f..e75f93c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -32,7 +32,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 
 import java.io.Serializable;


[26/26] ignite git commit: ignite-5075

Posted by sb...@apache.org.
ignite-5075


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

Branch: refs/heads/ignite-5075
Commit: a1d410412ed39e0a16003b6ce291a7f217a49b24
Parents: 019d2f7
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 18:07:54 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 18:07:54 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       | 69 ++++++++------------
 .../processors/cache/CacheGroupData.java        | 11 ++--
 .../processors/cache/CacheGroupDescriptor.java  | 41 ++++++++++--
 .../processors/cache/ClusterCachesInfo.java     | 63 ++++++++++++++----
 .../processors/cache/ExchangeActions.java       | 41 ++++++++++++
 .../processors/cache/GridCacheProcessor.java    | 37 ++++++-----
 6 files changed, 182 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d41041/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 64c1fd7..507d6e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -324,30 +324,16 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param exchActions Cache change requests to execte on exchange.
      */
     private void updateCachesInfo(ExchangeActions exchActions) {
-        for (ExchangeActions.ActionData action : exchActions.stopRequests()) {
-            DynamicCacheDescriptor desc = registeredCaches.remove(action.descriptor().cacheId());
+        for (CacheGroupDescriptor stopDesc : exchActions.cacheGroupsToStop()) {
+            CacheGroupDescriptor rmvd = registeredCacheGrps.remove(stopDesc.groupId());
 
-            assert desc != null : action.request().cacheName();
+            assert rmvd != null : stopDesc.groupName();
         }
 
-        for (ExchangeActions.ActionData action : exchActions.newAndClientCachesStartRequests()) {
-            DynamicCacheChangeRequest req = action.request();
-
-            if (!req.clientStartOnly()) {
-                Integer cacheId = CU.cacheId(req.cacheName());
+        for (CacheGroupDescriptor startDesc : exchActions.cacheGroupsToStart()) {
+            CacheGroupDescriptor old = registeredCacheGrps.put(startDesc.groupId(), startDesc);
 
-                DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(),
-                    req.startCacheConfiguration(),
-                    req.cacheType(),
-                    req.cacheDescriptor().groupId(),
-                    false,
-                    req.deploymentId(),
-                    req.schema());
-
-                DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc);
-
-                assert old == null : old;
-            }
+            assert old == null : old;
         }
     }
 
@@ -362,7 +348,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      */
     public boolean onCacheChangeRequest(final GridDhtPartitionsExchangeFuture fut,
         boolean crd,
-        ExchangeActions exchActions)
+        final ExchangeActions exchActions)
         throws IgniteCheckedException
     {
         assert exchActions != null && !exchActions.empty() : exchActions;
@@ -372,7 +358,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         // Affinity did not change for existing caches.
         forAllCacheGroups(crd && lateAffAssign, new IgniteInClosureX<GridAffinityAssignmentCache>() {
             @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
-                if (fut.stopping(aff.cacheId()))
+                if (exchActions.cacheGroupStopping(aff.groupId()))
                     return;
 
                 aff.clientEventTopologyChange(fut.discoveryEvent(), fut.topologyVersion());
@@ -402,7 +388,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 cctx.cache().prepareCacheStart(cacheDesc, nearCfg, fut.topologyVersion());
 
             if (fut.isCacheAdded(cacheDesc.cacheId(), fut.topologyVersion())) {
-                if (fut.discoCache().cacheGroupAffinityNodes(desc.groupDescriptor().groupId()).isEmpty())
+                if (fut.discoCache().cacheGroupAffinityNodes(cacheDesc.groupDescriptor().groupId()).isEmpty())
                     U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
             }
 
@@ -430,7 +416,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 }
             }
             else
-                initStartedCacheOnCoordinator(fut, cacheDesc.cacheId());
+                initStartedCacheOnCoordinator(fut, cacheDesc.groupDescriptor());
         }
 
         for (DynamicCacheChangeRequest req : exchActions.closeRequests(cctx.localNodeId())) {
@@ -767,24 +753,26 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
     /**
      * @param fut Exchange future.
-     * @param desc Cache descriptor.
+     * @param grpDesc Cache group descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private void initStartedCacheOnCoordinator(GridDhtPartitionsExchangeFuture fut, final DynamicCacheDescriptor desc)
+    private void initStartedCacheOnCoordinator(GridDhtPartitionsExchangeFuture fut, final CacheGroupDescriptor grpDesc)
         throws IgniteCheckedException {
-        assert desc != null && desc.groupId() != 0 : desc;
+        assert grpDesc != null && grpDesc.groupId() != 0 : grpDesc;
 
-        int grpId = desc.groupId();
+        if (grpDesc.config().getCacheMode() == LOCAL)
+            return;
 
-        CacheGroupHolder grpHolder = cacheGrps.get(desc.groupId());
+        int grpId = grpDesc.groupId();
+
+        CacheGroupHolder grpHolder = cacheGrps.get(grpId);
 
         CacheGroupInfrastructure grp = cctx.kernalContext().cache().cacheGroup(grpId);
 
         if (grpHolder == null) {
-            if (desc.cacheConfiguration().getCacheMode() == LOCAL)
-                return;
-
-            grpHolder = grp != null ? new CacheGroupHolder1(grp, null) : CacheGroupHolder2.create(cctx, desc, fut, null);
+            grpHolder = grp != null ?
+                new CacheGroupHolder1(grp, null) :
+                CacheGroupHolder2.create(cctx, grpDesc, fut, null);
 
             CacheGroupHolder old = cacheGrps.put(grpId, grpHolder);
 
@@ -1725,7 +1713,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         /**
          * @param cctx Context.
-         * @param cacheDesc Cache descriptor.
+         * @param grpDesc Cache group descriptor.
          * @param fut Exchange future.
          * @param initAff Current affinity.
          * @return Cache holder.
@@ -1733,18 +1721,18 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
          */
         static CacheGroupHolder2 create(
             GridCacheSharedContext cctx,
-            DynamicCacheDescriptor cacheDesc,
+            CacheGroupDescriptor grpDesc,
             GridDhtPartitionsExchangeFuture fut,
             @Nullable GridAffinityAssignmentCache initAff) throws IgniteCheckedException {
-            assert cacheDesc != null;
+            assert grpDesc != null;
             assert !cctx.kernalContext().clientNode();
 
-            CacheConfiguration<?, ?> ccfg = cacheDesc.cacheConfiguration();
+            CacheConfiguration<?, ?> ccfg = grpDesc.config();
 
-            assert ccfg != null : cacheDesc;
+            assert ccfg != null : grpDesc;
             assert ccfg.getCacheMode() != LOCAL : ccfg.getName();
 
-            assert !cctx.discovery().cacheAffinityNodes(ccfg.getName(), fut.topologyVersion()).contains(cctx.localNode());
+            assert !cctx.discovery().cacheGroupAffinityNodes(grpDesc.groupId(), fut.topologyVersion()).contains(cctx.localNode());
 
             AffinityFunction affFunc = cctx.cache().clone(ccfg.getAffinity());
 
@@ -1754,7 +1742,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             U.startLifecycleAware(F.asList(affFunc));
 
             GridAffinityAssignmentCache aff = new GridAffinityAssignmentCache(cctx.kernalContext(),
-                ccfg.getName(),
+                grpDesc.groupName(),
+                grpDesc.groupId(),
                 affFunc,
                 ccfg.getNodeFilter(),
                 ccfg.getBackups(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d41041/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
index 4a6edda..0507839 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.io.Serializable;
+import java.util.Map;
 import java.util.Set;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -45,7 +46,7 @@ public class CacheGroupData implements Serializable {
 
     /** */
     @GridToStringInclude
-    private final Set<String> cacheNames;
+    private final Map<String, Integer> caches;
 
     /**
      * @param cacheCfg Cache configuration.
@@ -56,7 +57,7 @@ public class CacheGroupData implements Serializable {
         String grpName,
         int grpId,
         AffinityTopologyVersion startTopVer,
-        Set<String> cacheNames) {
+        Map<String, Integer> caches) {
         assert cacheCfg != null;
         assert grpName != null;
         assert grpId != 0;
@@ -66,7 +67,7 @@ public class CacheGroupData implements Serializable {
         this.grpName = grpName;
         this.grpId = grpId;
         this.startTopVer = startTopVer;
-        this.cacheNames = cacheNames;
+        this.caches = caches;
     }
 
     public String groupName() {
@@ -85,8 +86,8 @@ public class CacheGroupData implements Serializable {
         return startTopVer;
     }
 
-    Set<String> cacheNames() {
-        return cacheNames;
+    Map<String, Integer> caches() {
+        return caches;
     }
 
     @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d41041/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
index 7b0f8fe..c0ad67a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
@@ -17,11 +17,14 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
 
 /**
  *
@@ -41,13 +44,13 @@ public class CacheGroupDescriptor {
 
     /** */
     @GridToStringInclude
-    private final Set<String> cacheNames;
+    private Map<String, Integer> caches;
 
     CacheGroupDescriptor(String grpName,
         int grpId,
         CacheConfiguration cacheCfg,
         AffinityTopologyVersion startTopVer,
-        Set<String> cacheNames) {
+        Map<String, Integer> caches) {
         assert cacheCfg != null;
         assert grpName != null;
         assert grpId != 0;
@@ -57,7 +60,35 @@ public class CacheGroupDescriptor {
         this.grpId = grpId;
         this.cacheCfg = cacheCfg;
         this.startTopVer = startTopVer;
-        this.cacheNames = cacheNames;
+        this.caches = caches;
+    }
+
+    void onCacheAdded(String cacheName, int cacheId) {
+        assert cacheName != null;
+        assert cacheId != 0;
+
+        Map<String, Integer> caches = new HashMap<>(this.caches);
+
+        caches.put(cacheName, cacheId);
+
+        this.caches = caches;
+    }
+
+    void onCacheStopped(String cacheName, int cacheId) {
+        assert cacheName != null;
+        assert cacheId != 0;
+
+        Map<String, Integer> caches = new HashMap<>(this.caches);
+
+        Integer rmvd = caches.remove(cacheName);
+
+        assert rmvd != null && rmvd == cacheId : cacheName;
+
+        this.caches = caches;
+    }
+
+    boolean hasCaches() {
+        return caches != null;
     }
 
     public String groupName() {
@@ -76,8 +107,8 @@ public class CacheGroupDescriptor {
         return startTopVer;
     }
 
-    Set<String> cacheNames() {
-        return cacheNames;
+    Map<String, Integer> caches() {
+        return caches;
     }
 
     @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d41041/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index e539f79..c1cde6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -164,7 +164,12 @@ class ClusterCachesInfo {
                         assert req.cacheType() != null : req;
                         assert F.eq(ccfg.getName(), req.cacheName()) : req;
 
-                        CacheGroupDescriptor grpDesc = registerCacheGroup(ccfg, topVer.nextMinorVersion());
+                        int cacheId = CU.cacheId(req.cacheName());
+
+                        CacheGroupDescriptor grpDesc = registerCacheGroup(exchangeActions,
+                            ccfg,
+                            cacheId,
+                            topVer.nextMinorVersion());
 
                         DynamicCacheDescriptor startDesc = new DynamicCacheDescriptor(ctx,
                             ccfg,
@@ -256,13 +261,25 @@ class ClusterCachesInfo {
                 if (desc != null) {
                     DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
 
-                    assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
+                    assert old != null && old == desc : "Dynamic cache map was concurrently modified [req=" + req + ']';
 
                     ctx.discovery().removeCacheFilter(req.cacheName());
 
                     needExchange = true;
 
                     exchangeActions.addCacheToStop(req, desc);
+
+                    CacheGroupDescriptor grpDesc = registeredCacheGrps.get(desc.groupDescriptor().groupName());
+
+                    assert grpDesc != null && grpDesc.groupId() == desc.groupDescriptor().groupId() : desc;
+
+                    grpDesc.onCacheStopped(desc.cacheName(), desc.cacheId());
+
+                    if (!grpDesc.hasCaches()) {
+                        registeredCacheGrps.remove(grpDesc.groupId());
+
+                        exchangeActions.addCacheGroupToStop(grpDesc);
+                    }
                 }
             }
             else if (req.close()) {
@@ -514,7 +531,8 @@ class ClusterCachesInfo {
             CacheGroupData grpData = new CacheGroupData(grpDesc.config(),
                 grpDesc.groupName(),
                 grpDesc.groupId(),
-                grpDesc.startTopologyVersion());
+                grpDesc.startTopologyVersion(),
+                grpDesc.caches());
 
             cacheGrps.put(grpDesc.groupName(), grpData);
         }
@@ -562,7 +580,8 @@ class ClusterCachesInfo {
             CacheGroupDescriptor grpDesc = new CacheGroupDescriptor(grpData.groupName(),
                 grpData.groupId(),
                 grpData.config(),
-                grpData.startTopologyVersion());
+                grpData.startTopologyVersion(),
+                grpData.caches());
 
             CacheGroupDescriptor old = registeredCacheGrps.put(grpDesc.groupName(), grpDesc);
 
@@ -710,7 +729,9 @@ class ClusterCachesInfo {
             CacheConfiguration cfg = cacheInfo.config();
 
             if (!registeredCaches.containsKey(cfg.getName())) {
-                CacheGroupDescriptor grpDesc = registerCacheGroup(cfg, topVer);
+                int cacheId = CU.cacheId(cfg.getName());
+
+                CacheGroupDescriptor grpDesc = registerCacheGroup(null, cfg, cacheId, topVer);
 
                 DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
                     cfg,
@@ -737,23 +758,39 @@ class ClusterCachesInfo {
         }
     }
 
-    private CacheGroupDescriptor registerCacheGroup(CacheConfiguration cfg, AffinityTopologyVersion topVer) {
-        if (cfg.getGroupName() != null) {
-            CacheGroupDescriptor desc = registeredCacheGrps.get(cfg.getGroupName());
+    private CacheGroupDescriptor registerCacheGroup(
+        ExchangeActions exchActions,
+        CacheConfiguration startedCacheCfg,
+        Integer cacheId,
+        AffinityTopologyVersion topVer) {
+        if (startedCacheCfg.getGroupName() != null) {
+            CacheGroupDescriptor desc = registeredCacheGrps.get(startedCacheCfg.getGroupName());
+
+            if (desc != null) {
+                desc.onCacheAdded(startedCacheCfg.getName(), cacheId);
 
-            if (desc != null)
                 return desc;
+            }
         }
 
         int grpId = cacheGrpIdGen++;
 
+        Map<String, Integer> caches = Collections.singletonMap(startedCacheCfg.getName(), cacheId);
+
+        String grpName = startedCacheCfg.getGroupName() != null ?
+            startedCacheCfg.getGroupName() : startedCacheCfg.getName();
+
         CacheGroupDescriptor grpDesc = new CacheGroupDescriptor(
-            cfg.getGroupName() != null ? cfg.getGroupName() : cfg.getName(),
+            grpName,
             grpId,
-            cfg,
-            topVer);
+            startedCacheCfg,
+            topVer,
+            caches);
+
+        ctx.discovery().addCacheGroup(grpDesc, startedCacheCfg.getNodeFilter(), startedCacheCfg.getCacheMode());
 
-        ctx.discovery().addCacheGroup(grpDesc, cfg.getNodeFilter(), cfg.getCacheMode());
+        if (exchActions != null)
+            exchActions.addCacheGroupToStart(grpDesc);
 
         return grpDesc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d41041/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index bcc77f6..c31b2d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -35,6 +35,12 @@ import org.jetbrains.annotations.Nullable;
  */
 public class ExchangeActions {
     /** */
+    private List<CacheGroupDescriptor> cacheGrpsToStart;
+
+    /** */
+    private List<CacheGroupDescriptor> cacheGrpsToStop;
+
+    /** */
     private Map<String, ActionData> cachesToStart;
 
     /** */
@@ -273,6 +279,39 @@ public class ExchangeActions {
         cachesToResetLostParts = add(cachesToResetLostParts, req, desc);
     }
 
+    void addCacheGroupToStart(CacheGroupDescriptor grpDesc) {
+        if (cacheGrpsToStart == null)
+            cacheGrpsToStart = new ArrayList<>();
+
+        cacheGrpsToStart.add(grpDesc);
+    }
+
+    List<CacheGroupDescriptor> cacheGroupsToStart() {
+        return cacheGrpsToStart != null ? cacheGrpsToStart : Collections.<CacheGroupDescriptor>emptyList();
+    }
+
+    void addCacheGroupToStop(CacheGroupDescriptor grpDesc) {
+        if (cacheGrpsToStop == null)
+            cacheGrpsToStop = new ArrayList<>();
+
+        cacheGrpsToStop.add(grpDesc);
+    }
+
+    List<CacheGroupDescriptor> cacheGroupsToStop() {
+        return cacheGrpsToStop != null ? cacheGrpsToStop : Collections.<CacheGroupDescriptor>emptyList();
+    }
+
+    boolean cacheGroupStopping(int grpId) {
+        if (cacheGrpsToStop != null) {
+            for (CacheGroupDescriptor grpToStop : cacheGrpsToStop) {
+                if (grpToStop.groupId() == grpId)
+                    return true;
+            }
+        }
+
+        return false;
+    }
+
     /**
      * @return {@code True} if there are no cache change actions.
      */
@@ -281,6 +320,8 @@ public class ExchangeActions {
             F.isEmpty(clientCachesToStart) &&
             F.isEmpty(cachesToStop) &&
             F.isEmpty(cachesToClose) &&
+            F.isEmpty(cacheGrpsToStart) &&
+            F.isEmpty(cacheGrpsToStop) &&
             F.isEmpty(cachesToResetLostParts);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d41041/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 93d12ba..d108b2c 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
@@ -1745,6 +1745,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cacheDesc.cacheConfiguration(),
             nearCfg,
             cacheDesc.cacheType(),
+            cacheDesc.groupDescriptor().groupId(),
             cacheDesc.deploymentId(),
             cacheDesc.startTopologyVersion(),
             exchTopVer,
@@ -1767,6 +1768,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     desc.cacheConfiguration(),
                     t.get2(),
                     desc.cacheType(),
+                    desc.groupDescriptor().groupId(),
                     desc.deploymentId(),
                     desc.startTopologyVersion(),
                     exchTopVer,
@@ -1797,6 +1799,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         desc.cacheConfiguration(),
                         null,
                         desc.cacheType(),
+                        desc.groupDescriptor().groupId(),
                         desc.deploymentId(),
                         desc.startTopologyVersion(),
                         exchTopVer,
@@ -1809,20 +1812,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         return started != null ? started : Collections.<DynamicCacheDescriptor>emptyList();
     }
 
-    private CacheGroupInfrastructure startCacheGroup(CacheConfiguration cfg0, int grpId) throws IgniteCheckedException {
-        CacheConfiguration ccfg = new CacheConfiguration(cfg0);
-
-        CacheGroupInfrastructure grp = new CacheGroupInfrastructure(grpId, sharedCtx, ccfg);
-
-        grp.start();
-
-        CacheGroupInfrastructure old = cacheGrps.put(grpId, grp);
-
-        assert old == null;
-
-        return grp;
-    }
-
     /**
      * @param startCfg Start configuration.
      * @param reqNearCfg Near configuration if specified for client cache start request.
@@ -1845,7 +1834,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     ) throws IgniteCheckedException {
         assert !caches.containsKey(startCfg.getName()) : startCfg.getName();
 
-        String grpName = cfg.getGroupName();
+        String grpName = startCfg.getGroupName();
 
         CacheGroupInfrastructure grp = null;
 
@@ -1859,10 +1848,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
 
             if (grp == null)
-                grp = startCacheGroup(cfg, grpId);
+                grp = startCacheGroup(startCfg, grpId);
         }
         else
-            grp = startCacheGroup(cfg, grpId);
+            grp = startCacheGroup(startCfg, grpId);
 
         CacheConfiguration ccfg = new CacheConfiguration(startCfg);
 
@@ -1906,6 +1895,20 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         onKernalStart(cache);
     }
 
+    private CacheGroupInfrastructure startCacheGroup(CacheConfiguration cfg0, int grpId) throws IgniteCheckedException {
+        CacheConfiguration ccfg = new CacheConfiguration(cfg0);
+
+        CacheGroupInfrastructure grp = new CacheGroupInfrastructure(grpId, sharedCtx, ccfg);
+
+        grp.start();
+
+        CacheGroupInfrastructure old = cacheGrps.put(grpId, grp);
+
+        assert old == null;
+
+        return grp;
+    }
+
     /**
      * @param req Stop request.
      */


[13/26] ignite git commit: Release notes for 2.0.

Posted by sb...@apache.org.
Release notes for 2.0.


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

Branch: refs/heads/ignite-5075
Commit: 6ab579c77123d72fee14c5bf9401ef0c4959c839
Parents: e80773b
Author: devozerov <vo...@gridgain.com>
Authored: Fri Apr 28 18:16:24 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Apr 28 18:16:24 2017 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 36 ++++++++++++++++++++++++++++++++++++
 1 file changed, 36 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ab579c7/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index 65cc9d4..b64aced 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -1,6 +1,42 @@
 Apache Ignite Release Notes
 ===========================
 
+Apache Ignite In-Memory Data Fabric 2.0
+---------------------------------------
+Ignite:
+* Introduced new page memory architecture
+* Machine Learning beta: distributed algebra support for dense and sparse data sets
+* Reworked and simplified API for asynchronous operations
+* Custom thread pool executors for compute tasks
+* Removed CLOCK mode in ATOMIC cache
+* Deprecated schema-import utility in favor of Web Console
+* Integration with Spring Data
+* Integration with Hibernate 5
+* Integration with RocketMQ
+* Integration with ZeroMQ
+* SQL: CREATE INDEX and DROP INDEX commands
+* SQL: Ability to execute queries over specific set of partitions
+* SQL: Improved REPLICATED cache support
+* SQL: Updated H2 version to 1.4.195
+* SQL: Improved performance of MIN/MAX aggregate functions
+* ODBC: Added Time data type support
+* Massive performance improvements
+
+Ignite.NET:
+* Custom plugin API
+* Generic cache store
+* Binary types now can be registered dynamically
+* LINQ: join, "contains" and DateTime property support
+
+Ignite CPP:
+* Implemented Cache::Invoke
+* Added remote filters support to continuous queries
+
+Web Console:
+* Multi-cluster support
+* Possibility to configure Kubernetes IP finder
+* EnforceJoinOrder option on Queries screen
+
 Apache Ignite In-Memory Data Fabric 1.9
 ---------------------------------------
 Ignite:


[02/26] ignite git commit: cache discovery data refactoring

Posted by sb...@apache.org.
cache discovery data refactoring


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

Branch: refs/heads/ignite-5075
Commit: c73e1c90ba7fea972e95728b94078a69c35bafbd
Parents: 4be320a
Author: sboikov <sb...@gridgain.com>
Authored: Wed Apr 26 14:01:40 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Apr 26 19:38:44 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/CacheData.java    | 152 ++++
 .../cache/CacheJoinNodeDiscoveryData.java       |  91 ++
 .../cache/CacheNodeCommonDiscoveryData.java     |  56 ++
 .../CacheReconnectClientDiscoveryData.java      |  26 +
 .../processors/cache/ClusterCachesInfo.java     | 620 ++++++++++++++
 .../cache/DynamicCacheDescriptor.java           |   3 -
 .../processors/cache/GridCacheContext.java      |   6 +
 .../processors/cache/GridCacheIoManager.java    |   9 +-
 .../processors/cache/GridCacheProcessor.java    | 834 +++++++------------
 .../GridDhtPartitionsExchangeFuture.java        |   9 +-
 10 files changed, 1270 insertions(+), 536 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
new file mode 100644
index 0000000..4579c27
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -0,0 +1,152 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.UUID;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.QuerySchema;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ *
+ */
+public class CacheData implements Serializable {
+    /** */
+    private final CacheConfiguration cacheCfg;
+
+    /** */
+    private final Integer cacheId;
+
+    /** */
+    private final CacheType cacheType;
+
+    /** */
+    private final AffinityTopologyVersion startTopVer;
+
+    /** */
+    private final IgniteUuid deploymentId;
+
+    /** */
+    private final QuerySchema schema;
+
+    /** */
+    private final UUID rcvdFrom;
+
+    /** */
+    private final boolean staticCfg;
+
+    /** */
+    private final boolean template;
+
+    CacheData(CacheConfiguration cacheCfg,
+        int cacheId,
+        CacheType cacheType,
+        AffinityTopologyVersion startTopVer,
+        IgniteUuid deploymentId,
+        QuerySchema schema,
+        UUID rcvdFrom,
+        boolean staticCfg,
+        boolean template) {
+        assert cacheCfg != null;
+        assert rcvdFrom != null;
+        assert startTopVer != null;
+        assert deploymentId != null;
+        assert template || cacheId != 0;
+
+        this.cacheCfg = cacheCfg;
+        this.cacheId = cacheId;
+        this.cacheType = cacheType;
+        this.startTopVer = startTopVer;
+        this.deploymentId = deploymentId;
+        this.schema = schema;
+        this.rcvdFrom = rcvdFrom;
+        this.staticCfg = staticCfg;
+        this.template = template;
+    }
+
+    /**
+     * @return Cache ID.
+     */
+    public Integer cacheId() {
+        return cacheId;
+    }
+
+    /**
+     * @return Start topology version.
+     */
+    public AffinityTopologyVersion startTopologyVersion() {
+        return startTopVer;
+    }
+
+    /**
+     * @return {@code True} if this is template configuration.
+     */
+    public boolean template() {
+        return template;
+    }
+
+    /**
+     * @return Cache type.
+     */
+    public CacheType cacheType() {
+        return cacheType;
+    }
+
+    /**
+     * @return Start ID.
+     */
+    public IgniteUuid deploymentId() {
+        return deploymentId;
+    }
+
+    /**
+     * @return {@code True} if statically configured.
+     */
+    public boolean staticallyConfigured() {
+        return staticCfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    public CacheConfiguration cacheConfiguration() {
+        return cacheCfg;
+    }
+
+    /**
+     * @return Schema.
+     */
+    public QuerySchema schema() {
+        return schema.copy();
+    }
+
+    /**
+     * @return ID of node provided cache configuration.
+     */
+    public UUID receivedFrom() {
+        return rcvdFrom;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheData.class, this, "cacheName", cacheCfg.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
new file mode 100644
index 0000000..0624217
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
@@ -0,0 +1,91 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Map;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ *
+ */
+class CacheJoinNodeDiscoveryData implements Serializable {
+    /** */
+    private final Map<String, CacheInfo> caches;
+
+    /** */
+    private final Map<String, CacheInfo> templates;
+
+    /** */
+    private final IgniteUuid cacheDeploymentId;
+
+    /**
+     * @param cacheDeploymentId Deployment ID for started caches.
+     * @param caches Caches.
+     * @param templates Templates.
+     */
+    CacheJoinNodeDiscoveryData(
+        IgniteUuid cacheDeploymentId,
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates) {
+        this.cacheDeploymentId = cacheDeploymentId;
+        this.caches = caches;
+        this.templates = templates;
+    }
+
+    IgniteUuid cacheDeploymentId() {
+        return cacheDeploymentId;
+    }
+
+    Map<String, CacheInfo> templates() {
+        return templates;
+    }
+
+    Map<String, CacheInfo> caches() {
+        return caches;
+    }
+
+    /**
+     *
+     */
+    static class CacheInfo implements Serializable {
+        /** */
+        private final CacheConfiguration ccfg;
+
+        /** */
+        private final CacheType cacheType;
+
+        /** */
+        private final byte flags;
+
+        CacheInfo(CacheConfiguration ccfg, CacheType cacheType, byte flags) {
+            this.ccfg = ccfg;
+            this.cacheType = cacheType;
+            this.flags = flags;
+        }
+
+        CacheConfiguration config() {
+            return ccfg;
+        }
+
+        CacheType cacheType() {
+            return cacheType;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
new file mode 100644
index 0000000..10df452
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
@@ -0,0 +1,56 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ *
+ */
+class CacheNodeCommonDiscoveryData implements Serializable {
+    /** */
+    private final Map<String, CacheData> caches;
+
+    /** */
+    private final Map<String, CacheData> templates;
+
+    /** */
+    private final Map<String, Map<UUID, Boolean>> clientNodesMap;
+
+    CacheNodeCommonDiscoveryData(Map<String, CacheData> caches,
+        Map<String, CacheData> templates,
+        Map<String, Map<UUID, Boolean>> clientNodesMap) {
+        this.caches = caches;
+        this.templates = templates;
+        this.clientNodesMap = clientNodesMap;
+    }
+
+    Map<String, CacheData> caches() {
+        return caches;
+    }
+
+    Map<String, CacheData> templates() {
+        return templates;
+    }
+
+    Map<String, Map<UUID, Boolean>> clientNodesMap() {
+        return clientNodesMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java
new file mode 100644
index 0000000..10a8f7e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java
@@ -0,0 +1,26 @@
+/*
+ * 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 java.io.Serializable;
+
+/**
+ *
+ */
+public class CacheReconnectClientDiscoveryData implements Serializable {
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
new file mode 100644
index 0000000..bd4ee1f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -0,0 +1,620 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheExistsException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.QuerySchema;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+
+/**
+ *
+ */
+class ClusterCachesInfo {
+    /** */
+    private final GridKernalContext ctx;
+
+    /** Dynamic caches. */
+    private final ConcurrentMap<String, DynamicCacheDescriptor> registeredCaches = new ConcurrentHashMap<>();
+
+    /** Cache templates. */
+    private final ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates = new ConcurrentHashMap<>();
+
+    /** */
+    private CacheJoinNodeDiscoveryData joinDiscoData;
+
+    /** */
+    private CacheNodeCommonDiscoveryData gridData;
+
+    /** */
+    private List<DynamicCacheDescriptor> locJoinStartCaches;
+
+    /**
+     * @param ctx Context.
+     */
+    ClusterCachesInfo(GridKernalContext ctx) {
+        this.ctx = ctx;
+    }
+
+    void onStart(CacheJoinNodeDiscoveryData joinDiscoData) {
+        this.joinDiscoData = joinDiscoData;
+    }
+
+    void onKernalStart() throws IgniteCheckedException {
+
+    }
+
+    /**
+     * @param batch Cache change request.
+     * @param topVer Topology version.
+     * @return {@code True} if minor topology version should be increased.
+     */
+    boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTopologyVersion topVer) {
+        boolean incMinorTopVer = false;
+
+        List<DynamicCacheDescriptor> added = null;
+
+        for (DynamicCacheChangeRequest req : batch.requests()) {
+            if (req.template()) {
+                CacheConfiguration ccfg = req.startCacheConfiguration();
+
+                assert ccfg != null : req;
+
+                DynamicCacheDescriptor desc = registeredTemplates().get(req.cacheName());
+
+                if (desc == null) {
+                    DynamicCacheDescriptor templateDesc = new DynamicCacheDescriptor(ctx,
+                        ccfg,
+                        req.cacheType(),
+                        true,
+                        req.deploymentId(),
+                        req.schema());
+
+                    templateDesc.receivedFrom(req.initiatingNodeId());
+
+                    DynamicCacheDescriptor old = registeredTemplates().put(ccfg.getName(), templateDesc);
+
+                    assert old == null;
+
+                    if (added == null)
+                        added = new ArrayList<>();
+
+                    added.add(templateDesc);
+                }
+
+                ctx.cache().completeTemplateAddFuture(ccfg.getName(), req.deploymentId());
+
+                continue;
+            }
+
+            DynamicCacheDescriptor desc = registeredCaches.get(req.cacheName());
+
+            boolean needExchange = false;
+
+            if (req.start()) {
+                if (desc == null) {
+                    if (req.clientStartOnly()) {
+                        ctx.cache().completeCacheStartFuture(req, new IgniteCheckedException("Failed to start " +
+                            "client cache (a cache with the given name is not started): " + req.cacheName()));
+                    }
+                    else {
+                        CacheConfiguration ccfg = req.startCacheConfiguration();
+
+                        assert req.cacheType() != null : req;
+                        assert F.eq(ccfg.getName(), req.cacheName()) : req;
+
+                        DynamicCacheDescriptor startDesc = new DynamicCacheDescriptor(ctx,
+                            ccfg,
+                            req.cacheType(),
+                            false,
+                            req.deploymentId(),
+                            req.schema());
+
+                        startDesc.receivedFrom(req.initiatingNodeId());
+
+                        DynamicCacheDescriptor old = registeredCaches.put(ccfg.getName(), startDesc);
+
+                        assert old == null;
+
+                        ctx.discovery().setCacheFilter(
+                            ccfg.getName(),
+                            ccfg.getNodeFilter(),
+                            ccfg.getNearConfiguration() != null,
+                            ccfg.getCacheMode());
+
+                        ctx.discovery().addClientNode(req.cacheName(),
+                            req.initiatingNodeId(),
+                            req.nearCacheConfiguration() != null);
+
+                        added.add(startDesc);
+
+                        needExchange = true;
+                    }
+                }
+                else {
+                    assert req.initiatingNodeId() != null : req;
+
+                    // Cache already exists, exchange is needed only if client cache should be created.
+                    ClusterNode node = ctx.discovery().node(req.initiatingNodeId());
+
+                    boolean clientReq = node != null &&
+                        !ctx.discovery().cacheAffinityNode(node, req.cacheName());
+
+                    if (req.clientStartOnly()) {
+                        needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
+                            req.initiatingNodeId(),
+                            req.nearCacheConfiguration() != null);
+                    }
+                    else {
+                        if (req.failIfExists()) {
+                            ctx.cache().completeCacheStartFuture(req,
+                                new CacheExistsException("Failed to start cache " +
+                                    "(a cache with the same name is already started): " + req.cacheName()));
+                        }
+                        else {
+                            needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
+                                req.initiatingNodeId(),
+                                req.nearCacheConfiguration() != null);
+
+                            if (needExchange)
+                                req.clientStartOnly(true);
+                        }
+                    }
+
+                    if (needExchange) {
+                        if (newTopVer == null) {
+                            newTopVer = new AffinityTopologyVersion(topVer.topologyVersion(),
+                                topVer.minorTopologyVersion() + 1);
+                        }
+
+                        desc.clientCacheStartVersion(newTopVer);
+                    }
+                }
+
+                if (!needExchange && desc != null) {
+                    if (desc.clientCacheStartVersion() != null)
+                        req.cacheFutureTopologyVersion(desc.clientCacheStartVersion());
+                    else
+                        req.cacheFutureTopologyVersion(desc.startTopologyVersion());
+                }
+            }
+            else if (req.globalStateChange() || req.resetLostPartitions())
+                needExchange = true;
+            else {
+                assert req.stop() ^ req.close() : req;
+
+                if (desc != null) {
+                    if (req.stop()) {
+                        DynamicCacheDescriptor old = cachesInfo.registeredCaches().remove(maskNull(req.cacheName()));
+
+                        assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
+
+                        ctx.discovery().removeCacheFilter(req.cacheName());
+
+                        needExchange = true;
+                    }
+                    else {
+                        assert req.close() : req;
+
+                        needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
+                    }
+                }
+            }
+
+            req.exchangeNeeded(needExchange);
+
+            incMinorTopVer |= needExchange;
+        }
+
+        if (added != null) {
+            AffinityTopologyVersion startTopVer = incMinorTopVer ?
+                new AffinityTopologyVersion(topVer.topologyVersion(), topVer.minorTopologyVersion() + 1) : topVer;
+
+            for (DynamicCacheDescriptor desc : added)
+                desc.startTopologyVersion(startTopVer);
+        }
+
+        return incMinorTopVer;
+    }
+
+    CacheJoinNodeDiscoveryData joinDiscoveryData() {
+        if (cachesOnDisconnect != null) {
+//            Collection<DynamicCacheChangeRequest> reqs;
+//
+//            Map<String, Map<UUID, Boolean>> clientNodesMap;
+//
+//            reqs = new ArrayList<>(caches.size() + 1);
+//
+//            clientNodesMap = U.newHashMap(caches.size());
+//
+//            collectDataOnReconnectingNode(reqs, clientNodesMap, joiningNodeId);
+
+            // TODO
+            return null;
+        }
+        else {
+            assert ctx.config().isDaemon() || joinDiscoData != null;
+
+            return joinDiscoData;
+        }
+    }
+
+    /**
+     * @param reqs requests.
+     * @param clientNodesMap Client nodes map.
+     * @param nodeId Node id.
+     */
+    private void collectDataOnReconnectingNode(
+        Collection<GridCacheAdapter> caches,
+        Collection<DynamicCacheChangeRequest> reqs,
+        Map<String, Map<UUID, Boolean>> clientNodesMap,
+        UUID nodeId
+    ) {
+        for (GridCacheAdapter<?, ?> cache : caches) {
+            DynamicCacheDescriptor desc = cachesOnDisconnect.get(cache.name());
+
+            if (desc == null)
+                continue;
+
+            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(null, cache.name(), null);
+
+            req.startCacheConfiguration(desc.cacheConfiguration());
+            req.cacheType(desc.cacheType());
+            req.deploymentId(desc.deploymentId());
+            req.receivedFrom(desc.receivedFrom());
+            req.schema(desc.schema());
+
+            reqs.add(req);
+
+            Boolean nearEnabled = cache.isNear();
+
+            Map<UUID, Boolean> map = U.newHashMap(1);
+
+            map.put(nodeId, nearEnabled);
+
+            clientNodesMap.put(cache.name(), map);
+        }
+    }
+
+    /**
+     * Called from exchange worker.
+     *
+     * @return Caches to be started when this node starts.
+     */
+    List<DynamicCacheDescriptor> cachesToStartOnLocalJoin() {
+        assert locJoinStartCaches != null;
+
+        List<DynamicCacheDescriptor> locJoinStartCaches = this.locJoinStartCaches;
+
+        this.locJoinStartCaches = null;
+
+        return locJoinStartCaches;
+    }
+
+    List<DynamicCacheDescriptor> cachesReceivedFromJoin(UUID joinedNodeId) {
+        assert joinedNodeId != null;
+
+        List<DynamicCacheDescriptor> started = null;
+
+        if (!ctx.clientNode() && !ctx.isDaemon()) {
+            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                if (desc.staticallyConfigured()) {
+                    assert desc.receivedFrom() != null : desc;
+
+                    IgnitePredicate<ClusterNode> filter = desc.cacheConfiguration().getNodeFilter();
+
+                    if (joinedNodeId.equals(desc.receivedFrom()) &&
+                        CU.affinityNode(ctx.discovery().localNode(), filter)) {
+                        if (started == null)
+                            started = new ArrayList<>();
+
+                        started.add(desc);
+                    }
+                }
+            }
+        }
+
+        return started;
+    }
+
+    /**
+     * Discovery event callback, executed from discovery thread.
+     *
+     * @param type Event type.
+     * @param node Event node.
+     * @param topVer Topology version.
+     */
+    void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
+        if (type == EVT_NODE_JOINED) {
+            if (node.id().equals(ctx.discovery().localNode().id())) {
+                if (gridData == null) { // First node starts.
+                    assert registeredCaches.isEmpty();
+                    assert registeredTemplates.isEmpty();
+                    assert joinDiscoData != null;
+
+                    processJoiningNode(joinDiscoData, node.id());
+                }
+
+                assert locJoinStartCaches == null;
+
+                locJoinStartCaches = new ArrayList<>();
+
+                for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                    CacheConfiguration cfg = desc.cacheConfiguration();
+
+                    boolean locCfg = joinDiscoData.caches().containsKey(cfg.getName());
+
+                    if (locCfg || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()))
+                        locJoinStartCaches.add(desc);
+                }
+
+                joinDiscoData = null;
+            }
+
+            initStartVersionOnJoin(registeredCaches.values(), node, topVer);
+
+            initStartVersionOnJoin(registeredTemplates.values(), node, topVer);
+        }
+    }
+
+    private void initStartVersionOnJoin(Collection<DynamicCacheDescriptor> descs,
+        ClusterNode joinedNode,
+        AffinityTopologyVersion topVer) {
+        for (DynamicCacheDescriptor cacheDesc : descs) {
+            if (cacheDesc.staticallyConfigured() && joinedNode.id().equals(cacheDesc.receivedFrom()))
+                cacheDesc.startTopologyVersion(topVer);
+        }
+    }
+
+    CacheNodeCommonDiscoveryData collectCommonDiscoveryData() {
+        Map<String, CacheData> caches = new HashMap<>();
+
+        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+            CacheData cacheData = new CacheData(desc.cacheConfiguration(),
+                desc.cacheId(),
+                desc.cacheType(),
+                desc.startTopologyVersion(),
+                desc.deploymentId(),
+                desc.schema(),
+                desc.receivedFrom(),
+                desc.staticallyConfigured(),
+                false);
+
+            caches.put(desc.cacheConfiguration().getName(), cacheData);
+        }
+
+        Map<String, CacheData> templates = new HashMap<>();
+
+        for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
+            CacheData cacheData = new CacheData(desc.cacheConfiguration(),
+                0,
+                desc.cacheType(),
+                desc.startTopologyVersion(),
+                null,
+                desc.schema(),
+                desc.receivedFrom(),
+                desc.staticallyConfigured(),
+                true);
+
+            templates.put(desc.cacheConfiguration().getName(), cacheData);
+        }
+
+        return new CacheNodeCommonDiscoveryData(caches, templates, ctx.discovery().clientNodesMap());
+    }
+
+    void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) {
+        assert joinDiscoData != null;
+        assert data.commonData() instanceof CacheNodeCommonDiscoveryData : data;
+
+        CacheNodeCommonDiscoveryData cachesData = (CacheNodeCommonDiscoveryData)data.commonData();
+
+        for (CacheData cacheData : cachesData.templates().values()) {
+            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
+                ctx,
+                cacheData.cacheConfiguration(),
+                cacheData.cacheType(),
+                true,
+                cacheData.deploymentId(),
+                cacheData.schema());
+
+            desc.startTopologyVersion(cacheData.startTopologyVersion());
+            desc.receivedFrom(cacheData.receivedFrom());
+            desc.staticallyConfigured(cacheData.staticallyConfigured());
+
+            DynamicCacheDescriptor old = registeredTemplates.put(cacheData.cacheConfiguration().getName(), desc);
+
+            assert old == null;
+        }
+
+        for (CacheData cacheData : cachesData.caches().values()) {
+            CacheConfiguration cfg = cacheData.cacheConfiguration();
+
+            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
+                ctx,
+                cacheData.cacheConfiguration(),
+                cacheData.cacheType(),
+                false,
+                cacheData.deploymentId(),
+                cacheData.schema());
+
+            desc.startTopologyVersion(cacheData.startTopologyVersion());
+            desc.receivedFrom(cacheData.receivedFrom());
+            desc.staticallyConfigured(cacheData.staticallyConfigured());
+
+            DynamicCacheDescriptor old = registeredCaches.put(cacheData.cacheConfiguration().getName(), desc);
+
+            assert old == null;
+
+            ctx.discovery().setCacheFilter(
+                cfg.getName(),
+                cfg.getNodeFilter(),
+                cfg.getNearConfiguration() != null,
+                cfg.getCacheMode());
+        }
+
+        if (!F.isEmpty(cachesData.clientNodesMap())) {
+            for (Map.Entry<String, Map<UUID, Boolean>> entry : cachesData.clientNodesMap().entrySet()) {
+                String cacheName = entry.getKey();
+
+                for (Map.Entry<UUID, Boolean> tup : entry.getValue().entrySet())
+                    ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue());
+            }
+        }
+
+        gridData = cachesData;
+    }
+
+    void onJoiningNodeDataReceived(DiscoveryDataBag.JoiningNodeDiscoveryData data) {
+        if (data.hasJoiningNodeData()) {
+            Serializable joiningNodeData = data.joiningNodeData();
+
+            if (joiningNodeData instanceof CacheReconnectClientDiscoveryData) {
+                // TODO
+            }
+            else if (joiningNodeData instanceof CacheJoinNodeDiscoveryData)
+                processJoiningNode((CacheJoinNodeDiscoveryData)joiningNodeData, data.joiningNodeId());
+        }
+    }
+
+    private void processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId) {
+        for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.templates().values()) {
+            CacheConfiguration cfg = cacheInfo.config();
+
+            if (!registeredTemplates.containsKey(cfg.getName())) {
+                DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
+                    cfg,
+                    cacheInfo.cacheType(),
+                    true,
+                    joinData.cacheDeploymentId(),
+                    new QuerySchema(cfg.getQueryEntities()));
+
+                desc.staticallyConfigured(true);
+                desc.receivedFrom(nodeId);
+
+                DynamicCacheDescriptor old = registeredTemplates.put(cfg.getName(), desc);
+
+                assert old == null : old;
+            }
+        }
+
+        for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.caches().values()) {
+            CacheConfiguration cfg = cacheInfo.config();
+
+            if (!registeredCaches.containsKey(cfg.getName())) {
+                DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
+                    cfg,
+                    cacheInfo.cacheType(),
+                    false,
+                    joinData.cacheDeploymentId(),
+                    new QuerySchema(cfg.getQueryEntities()));
+
+                desc.staticallyConfigured(true);
+                desc.receivedFrom(nodeId);
+
+                DynamicCacheDescriptor old = registeredCaches.put(cfg.getName(), desc);
+
+                assert old == null : old;
+
+                ctx.discovery().setCacheFilter(
+                    cfg.getName(),
+                    cfg.getNodeFilter(),
+                    cfg.getNearConfiguration() != null,
+                    cfg.getCacheMode());
+            }
+
+            ctx.discovery().addClientNode(cfg.getName(),
+                nodeId,
+                cfg.getNearConfiguration() != null);
+        }
+    }
+
+    ConcurrentMap<String, DynamicCacheDescriptor> registeredCaches() {
+        return registeredCaches;
+    }
+
+    ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates() {
+        return registeredTemplates;
+    }
+
+    /** */
+    private Map<String, DynamicCacheDescriptor> cachesOnDisconnect;
+
+    void onDisconnect() {
+        cachesOnDisconnect = new HashMap<>(registeredCaches);
+
+        registeredCaches.clear();
+        registeredTemplates.clear();
+    }
+
+    Set<String> onReconnected() {
+        assert cachesOnDisconnect != null;
+
+        Set<String> stoppedCaches = new HashSet<>();
+
+        for(Map.Entry<String, DynamicCacheDescriptor> e : cachesOnDisconnect.entrySet()) {
+            DynamicCacheDescriptor desc = e.getValue();
+
+            String name = e.getKey();
+
+            boolean stopped;
+
+            boolean sysCache = CU.isUtilityCache(name) || CU.isAtomicsCache(name);
+
+            if (!sysCache) {
+                DynamicCacheDescriptor newDesc = registeredCaches.get(name);
+
+                stopped = newDesc == null || !desc.deploymentId().equals(newDesc.deploymentId());
+            }
+            else
+                stopped = false;
+
+            if (stopped)
+                stoppedCaches.add(name);
+        }
+
+        cachesOnDisconnect = null;
+
+        return stoppedCaches;
+    }
+
+    void clearCaches() {
+        registeredCaches.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/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 92a7af3..a2e91e9 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
@@ -50,9 +50,6 @@ public class DynamicCacheDescriptor {
     /** Statically configured flag. */
     private boolean staticCfg;
 
-    /** Started flag. */
-    private boolean started;
-
     /** Cache type. */
     private CacheType cacheType;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 92c144c..67f25b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -237,6 +237,8 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Start topology version. */
     private AffinityTopologyVersion startTopVer;
 
+    private AffinityTopologyVersion cacheStartTopVer;
+
     /** Dynamic cache deployment ID. */
     private IgniteUuid dynamicDeploymentId;
 
@@ -458,6 +460,10 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.startTopVer = startTopVer;
     }
 
+    public void cacheStartTopologyVersion(AffinityTopologyVersion cacheStartTopVer) {
+        this.cacheStartTopVer = cacheStartTopVer;
+    }
+
     /**
      * @return Cache default {@link ExpiryPolicy}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index fdd29e4..b9c066b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -150,12 +150,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                     DynamicCacheDescriptor cacheDesc = cctx.cache().cacheDescriptor(cacheMsg.cacheId());
 
-                    if (cacheDesc != null) {
-                        if (cacheDesc.startTopologyVersion() != null)
-                            startTopVer = cacheDesc.startTopologyVersion();
-                        else if (cacheDesc.receivedFromStartVersion() != null)
-                            startTopVer = cacheDesc.receivedFromStartVersion();
-                    }
+                    // TODO: should be specified on request since cache desc can be removed,
+                    if (cacheDesc != null)
+                        startTopVer = cacheDesc.startTopologyVersion();
 
                     // Need to wait for exchange to avoid race between cache start and affinity request.
                     fut = cctx.exchange().affinityReadyFuture(startTopVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c73e1c90/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 4b79361..ecbf475 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
@@ -152,7 +152,6 @@ import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
 import static org.apache.ignite.configuration.DeploymentMode.ISOLATED;
 import static org.apache.ignite.configuration.DeploymentMode.PRIVATE;
 import static org.apache.ignite.configuration.DeploymentMode.SHARED;
-import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC;
 import static org.apache.ignite.internal.IgniteComponentType.JTA;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED;
@@ -191,11 +190,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Template configuration add futures. */
     private ConcurrentMap<String, IgniteInternalFuture> pendingTemplateFuts = new ConcurrentHashMap<>();
 
-    /** Dynamic caches. */
-    private ConcurrentMap<String, DynamicCacheDescriptor> registeredCaches = new ConcurrentHashMap<>();
-
-    /** Cache templates. */
-    private ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates = new ConcurrentHashMap<>();
+    /** */
+    private ClusterCachesInfo cachesInfo;
 
     /** */
     private IdentityHashMap<CacheStore, ThreadLocal> sesHolders = new IdentityHashMap<>();
@@ -207,9 +203,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private final CountDownLatch cacheStartedLatch = new CountDownLatch(1);
 
     /** */
-    private Map<String, DynamicCacheDescriptor> cachesOnDisconnect;
-
-    /** */
     private Map<UUID, DynamicCacheChangeBatch> clientReconnectReqs;
 
     /** Internal cache names. */
@@ -389,16 +382,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param cc Cache Configuration.
      * @return {@code true} if cache is starting on client node and this node is affinity node for the cache.
      */
-    private boolean storesLocallyOnClient(IgniteConfiguration c,
-                                          CacheConfiguration cc) {
+    private boolean storesLocallyOnClient(IgniteConfiguration c, CacheConfiguration cc) {
         if (c.isClientMode() && c.getMemoryConfiguration() == null) {
             if (cc.getCacheMode() == LOCAL)
                 return true;
 
-            if (ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cc.getName()))
-                return true;
+            return ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cc.getName());
 
-            return false;
         }
         else
             return false;
@@ -623,6 +613,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
     @Override public void start(boolean activeOnStart) throws IgniteCheckedException {
+        cachesInfo = new ClusterCachesInfo(ctx);
+
         DeploymentMode depMode = ctx.config().getDeploymentMode();
 
         if (!F.isEmpty(ctx.config().getCacheConfiguration())) {
@@ -643,72 +635,31 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheSharedManager mgr : sharedCtx.managers())
             mgr.start(sharedCtx);
 
-        //if inActivate on start then skip registrate caches
-        if (!activeOnStart)
-            return;
+        if (activeOnStart && !ctx.config().isDaemon()) {
+            Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches = new HashMap<>();
 
-        CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration();
+            Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates = new HashMap<>();
 
-        registerCacheFromConfig(cfgs);
+            registerCacheFromConfig(caches, templates);
 
-        registerCacheFromPersistentStore(cfgs);
+            registerCacheFromPersistentStore(caches, templates);
 
-        if (log.isDebugEnabled())
-            log.debug("Started cache processor.");
-    }
-
-    /**
-     * @param cfgs Cache configurations.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void registerCacheFromConfig(CacheConfiguration[] cfgs) throws IgniteCheckedException {
-        for (int i = 0; i < cfgs.length; i++) {
-            if (ctx.config().isDaemon())
-                continue;
-
-            CacheConfiguration<?, ?> cfg = new CacheConfiguration(cfgs[i]);
-
-            cfgs[i] = cfg; // Replace original configuration value.
-
-            registerCache(cfg);
+            cachesInfo.onStart(new CacheJoinNodeDiscoveryData(IgniteUuid.randomUuid(), caches, templates));
         }
-    }
-
-    /**
-     * @param cfgs Cache configurations.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void registerCacheFromPersistentStore(CacheConfiguration[] cfgs) throws IgniteCheckedException {
-        if (sharedCtx.pageStore() != null &&
-            sharedCtx.database().persistenceEnabled() &&
-            !ctx.config().isDaemon()) {
-
-            Set<String> savedCacheNames = sharedCtx.pageStore().savedCacheNames();
-
-            for (CacheConfiguration cfg : cfgs)
-                savedCacheNames.remove(cfg.getName());
-
-            for (String name : internalCaches)
-                savedCacheNames.remove(name);
-
-            if (!F.isEmpty(savedCacheNames)) {
-                log.info("Registrate persistent caches: " + savedCacheNames);
-
-                for (String name : savedCacheNames) {
-                    CacheConfiguration cfg = sharedCtx.pageStore().readConfiguration(name);
 
-                    if (cfg != null)
-                        registerCache(cfg);
-                }
-            }
-        }
+        if (log.isDebugEnabled())
+            log.debug("Started cache processor.");
     }
 
     /**
      * @param cfg Cache configuration.
+     * @param caches Caches map.
+     * @param templates Templates map.
      * @throws IgniteCheckedException If failed.
      */
-    private void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
+    private void registerCache(CacheConfiguration cfg,
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates) throws IgniteCheckedException {
         cloneCheckSerializable(cfg);
 
         CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
@@ -716,20 +667,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         // Initialize defaults.
         initialize(cfg, cacheObjCtx);
 
-        String masked = maskNull(cfg.getName());
-
-        if (cacheDescriptor(cfg.getName()) != null) {
-            String cacheName = cfg.getName();
+        boolean template = cfg.getName() != null && cfg.getName().endsWith("*");
 
-            if (cacheName != null)
+        if (!template) {
+            if (caches.containsKey(cfg.getName())) {
                 throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
-                    "assign unique name to each cache): " + U.maskName(cacheName));
-            else
-                throw new IgniteCheckedException("Default cache has already been configured (check configuration and " +
-                    "assign unique name to each cache).");
-        }
+                    "assign unique name to each cache): " + cfg.getName());
+            }
 
-        CacheType cacheType;
+            CacheType cacheType;
 
             if (CU.isUtilityCache(cfg.getName()))
                 cacheType = CacheType.UTILITY;
@@ -738,63 +684,163 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             else
                 cacheType = CacheType.USER;
 
-        if (cacheType != CacheType.USER && cfg.getMemoryPolicyName() == null)
-            cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
-
-        boolean template = cfg.getName() != null && cfg.getName().endsWith("*");
-
-        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
-            cfg,
-            cacheType,
-            template,
-            IgniteUuid.randomUuid(),
-            new QuerySchema(cfg.getQueryEntities()));
-
-        desc.locallyConfigured(true);
-        desc.staticallyConfigured(true);
-        desc.receivedFrom(ctx.localNodeId());
-
-        if (!template) {
-            cacheDescriptor(cfg.getName(), desc);
-
-            ctx.discovery().setCacheFilter(
-                cfg.getName(),
-                cfg.getNodeFilter(),
-                cfg.getNearConfiguration() != null && cfg.getCacheMode() == PARTITIONED,
-                cfg.getCacheMode());
-
-            ctx.discovery().addClientNode(cfg.getName(),
-                ctx.localNodeId(),
-                cfg.getNearConfiguration() != null);
+            if (cacheType != CacheType.USER && cfg.getMemoryPolicyName() == null)
+                cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
 
             if (!cacheType.userCache())
                 stopSeq.addLast(cfg.getName());
             else
                 stopSeq.addFirst(cfg.getName());
+
+            caches.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, cacheType, (byte)0));
         }
-        else {
-            if (log.isDebugEnabled())
-                log.debug("Use cache configuration as template: " + cfg);
+        else
+            templates.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, CacheType.USER, (byte)0));
+    }
+
+    /**
+     * @param caches Caches map.
+     * @param templates Templates map.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void registerCacheFromConfig(
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates
+    ) throws IgniteCheckedException {
+        assert !ctx.config().isDaemon();
+
+        CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration();
+
+        for (int i = 0; i < cfgs.length; i++) {
+            CacheConfiguration<?, ?> cfg = new CacheConfiguration(cfgs[i]);
 
-            registeredTemplates.put(masked, desc);
+            registerCache(cfg, caches, templates);
         }
+    }
 
-        if (cfg.getName() == null) { // Use cache configuration with null name as template.
-            DynamicCacheDescriptor desc0 = new DynamicCacheDescriptor(ctx,
-                cfg,
-                cacheType,
-                true,
-                IgniteUuid.randomUuid(),
-                new QuerySchema(cfg.getQueryEntities()));
+    /**
+     * @param caches Caches map.
+     * @param templates Templates map.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void registerCacheFromPersistentStore(
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
+        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates
+    ) throws IgniteCheckedException {
+        assert !ctx.config().isDaemon();
 
-            desc0.locallyConfigured(true);
-            desc0.staticallyConfigured(true);
+        if (sharedCtx.pageStore() != null && sharedCtx.database().persistenceEnabled()) {
+            Set<String> savedCacheNames = sharedCtx.pageStore().savedCacheNames();
 
-            registeredTemplates.put(masked, desc0);
+            savedCacheNames.removeAll(caches.keySet());
+
+            savedCacheNames.removeAll(internalCaches);
+
+            if (!F.isEmpty(savedCacheNames)) {
+                if (log.isInfoEnabled())
+                    log.info("Register persistent caches: " + savedCacheNames);
+
+                for (String name : savedCacheNames) {
+                    CacheConfiguration cfg = sharedCtx.pageStore().readConfiguration(name);
+
+                    if (cfg != null)
+                        registerCache(cfg, caches, templates);
+                }
+            }
         }
     }
 
     /**
+     * @param cfg Cache configuration.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
+//        cloneCheckSerializable(cfg);
+//
+//        CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
+//
+//        // Initialize defaults.
+//        initialize(cfg, cacheObjCtx);
+//
+//        String masked = maskNull(cfg.getName());
+//
+//        if (cacheDescriptor(cfg.getName()) != null) {
+//            String cacheName = cfg.getName();
+//
+//            if (cacheName != null)
+//                throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
+//                    "assign unique name to each cache): " + U.maskName(cacheName));
+//            else
+//                throw new IgniteCheckedException("Default cache has already been configured (check configuration and " +
+//                    "assign unique name to each cache).");
+//        }
+//
+//        CacheType cacheType;
+//
+//        if (CU.isUtilityCache(cfg.getName()))
+//            cacheType = CacheType.UTILITY;
+//        else if (internalCaches.contains(maskNull(cfg.getName())))
+//            cacheType = CacheType.INTERNAL;
+//        else
+//            cacheType = CacheType.USER;
+//
+//        if (cacheType != CacheType.USER && cfg.getMemoryPolicyName() == null)
+//            cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
+//
+//        boolean template = cfg.getName() != null && cfg.getName().endsWith("*");
+//
+//        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
+//            cfg,
+//            cacheType,
+//            template,
+//            IgniteUuid.randomUuid(),
+//            new QuerySchema(cfg.getQueryEntities()));
+//
+//        desc.locallyConfigured(true);
+//        desc.staticallyConfigured(true);
+//        desc.receivedFrom(ctx.localNodeId());
+//
+//        if (!template) {
+//            cacheDescriptor(cfg.getName(), desc);
+//
+//            ctx.discovery().setCacheFilter(
+//                cfg.getName(),
+//                cfg.getNodeFilter(),
+//                cfg.getNearConfiguration() != null && cfg.getCacheMode() == PARTITIONED,
+//                cfg.getCacheMode());
+//
+//            ctx.discovery().addClientNode(cfg.getName(),
+//                ctx.localNodeId(),
+//                cfg.getNearConfiguration() != null);
+//
+//            if (!cacheType.userCache())
+//                stopSeq.addLast(cfg.getName());
+//            else
+//                stopSeq.addFirst(cfg.getName());
+//        }
+//        else {
+//            if (log.isDebugEnabled())
+//                log.debug("Use cache configuration as template: " + cfg);
+//
+//            registeredTemplates.put(masked, desc);
+//        }
+//
+//        if (cfg.getName() == null) { // Use cache configuration with null name as template.
+//            DynamicCacheDescriptor desc0 = new DynamicCacheDescriptor(ctx,
+//                cfg,
+//                cacheType,
+//                true,
+//                IgniteUuid.randomUuid(),
+//                new QuerySchema(cfg.getQueryEntities()));
+//
+//            desc0.locallyConfigured(true);
+//            desc0.staticallyConfigured(true);
+//
+//            registeredTemplates.put(masked, desc0);
+//        }
+    }
+
+    /**
      * Initialize internal cache names
      */
     private void initializeInternalCacheNames() {
@@ -864,54 +910,55 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             ctx.query().onCacheKernalStart();
 
             // Start dynamic caches received from collect discovery data.
-            for (DynamicCacheDescriptor desc : cacheDescriptors()) {
-                if (ctx.config().isDaemon())
-                    continue;
-
-                desc.clearRemoteConfigurations();
-
-                CacheConfiguration ccfg = desc.cacheConfiguration();
-
-                IgnitePredicate filter = ccfg.getNodeFilter();
-
-                boolean loc = desc.locallyConfigured();
-
-                if (loc || (desc.receivedOnDiscovery() && CU.affinityNode(locNode, filter))) {
-                    boolean started = desc.onStart();
-
-                    assert started : "Failed to change started flag for locally configured cache: " + desc;
-
-                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
-
-                    CachePluginManager pluginMgr = desc.pluginManager();
-
-                    GridCacheContext ctx = createCache(
-                        ccfg, pluginMgr, desc.cacheType(), cacheObjCtx, desc.updatesAllowed());
-
-                    ctx.dynamicDeploymentId(desc.deploymentId());
-
-                    sharedCtx.addCacheContext(ctx);
-
-                    GridCacheAdapter cache = ctx.cache();
-
-                    String name = ccfg.getName();
-
-                    caches.put(maskNull(name), cache);
-
-                    startCache(cache, desc.schema());
-
-                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
-                }
-            }
+//            for (DynamicCacheDescriptor desc : cacheDescriptors()) {
+//                if (ctx.config().isDaemon())
+//                    continue;
+//
+//                desc.clearRemoteConfigurations();
+//
+//                CacheConfiguration ccfg = desc.cacheConfiguration();
+//
+//                IgnitePredicate filter = ccfg.getNodeFilter();
+//
+//                boolean loc = desc.locallyConfigured();
+//
+//                if (loc || (desc.receivedOnDiscovery() && CU.affinityNode(locNode, filter))) {
+//                    boolean started = desc.onStart();
+//
+//                    assert started : "Failed to change started flag for locally configured cache: " + desc;
+//
+//                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+//
+//                    CachePluginManager pluginMgr = desc.pluginManager();
+//
+//                    GridCacheContext ctx = createCache(
+//                        ccfg, pluginMgr, desc.cacheType(), cacheObjCtx, desc.updatesAllowed());
+//
+//                    ctx.dynamicDeploymentId(desc.deploymentId());
+//
+//                    sharedCtx.addCacheContext(ctx);
+//
+//                    GridCacheAdapter cache = ctx.cache();
+//
+//                    String name = ccfg.getName();
+//
+//                    caches.put(maskNull(name), cache);
+//
+//                    startCache(cache, desc.schema());
+//
+//                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
+//                }
+//            }
         }
         finally {
             cacheStartedLatch.countDown();
         }
 
         // Must call onKernalStart on shared managers after creation of fetched caches.
-        for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
+        for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers()) {
             if (sharedCtx.database() != mgr)
                 mgr.onKernalStart(false);
+        }
 
         // Escape if start active on start false
         if (!activeOnStart)
@@ -925,23 +972,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         ctx.service().onUtilityCacheStarted();
 
-        // Wait for caches in SYNC preload mode.
-        for (DynamicCacheDescriptor desc : cacheDescriptors()) {
-            CacheConfiguration cfg = desc.cacheConfiguration();
+        AffinityTopologyVersion startTopVer = new AffinityTopologyVersion(locNode.order(), 0);
 
-            IgnitePredicate filter = cfg.getNodeFilter();
-
-            if (desc.locallyConfigured() || (desc.receivedOnDiscovery() && CU.affinityNode(locNode, filter))) {
-                GridCacheAdapter cache = caches.get(maskNull(cfg.getName()));
+        for (GridCacheAdapter cache : caches.values()) {
+            CacheConfiguration cfg = cache.configuration();
 
-                if (cache != null) {
-                    if (cfg.getRebalanceMode() == SYNC) {
-                        CacheMode cacheMode = cfg.getCacheMode();
+            if (cache.context().affinityNode() &&
+                cfg.getRebalanceMode() == SYNC &&
+                startTopVer.equals(cache.context().startTopologyVersion())) {
+                CacheMode cacheMode = cfg.getCacheMode();
 
-                        if (cacheMode == REPLICATED || (cacheMode == PARTITIONED && cfg.getRebalanceDelay() >= 0))
-                            cache.preloader().syncFuture().get();
-                    }
-                }
+                if (cacheMode == REPLICATED || (cacheMode == PARTITIONED && cfg.getRebalanceDelay() >= 0))
+                    cache.preloader().syncFuture().get();
             }
         }
 
@@ -1031,7 +1073,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 stopCache(cache, cancel, false);
         }
 
-        registeredCaches.clear();
+        cachesInfo.clearCaches();
     }
 
     /**
@@ -1102,8 +1144,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
-        cachesOnDisconnect = new HashMap<>(registeredCaches);
-
         IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(
             ctx.cluster().clientReconnectFuture(),
             "Failed to execute dynamic cache change request, client node disconnected.");
@@ -1130,9 +1170,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         sharedCtx.onDisconnected(reconnectFut);
 
-        registeredCaches.clear();
-
-        registeredTemplates.clear();
+        cachesInfo.onDisconnect();
     }
 
     /** {@inheritDoc} */
@@ -1141,24 +1179,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         GridCompoundFuture<?, ?> stopFut = null;
 
-        for (final GridCacheAdapter cache : caches.values()) {
-            String name = cache.name();
-
-            boolean stopped;
+        Set<String> stoppedCaches = cachesInfo.onReconnected();
 
-            boolean sysCache = CU.isUtilityCache(name) || CU.isAtomicsCache(name);
-
-            if (!sysCache) {
-                DynamicCacheDescriptor oldDesc = cachesOnDisconnect.get(maskNull(name));
-
-                assert oldDesc != null : "No descriptor for cache: " + name;
-
-                DynamicCacheDescriptor newDesc = cacheDescriptor(name);
-
-                stopped = newDesc == null || !oldDesc.deploymentId().equals(newDesc.deploymentId());
-            }
-            else
-                stopped = false;
+        for (final GridCacheAdapter cache : caches.values()) {
+            boolean stopped = stoppedCaches.contains(cache.name());
 
             if (stopped) {
                 cache.context().gate().reconnected(true);
@@ -1185,11 +1209,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 reconnected.add(cache);
 
-                if (!sysCache) {
+                if (cache.context().userCache()) {
                     // Re-create cache structures inside indexing in order to apply recent schema changes.
                     GridCacheContext cctx = cache.context();
 
-                    DynamicCacheDescriptor desc = cacheDescriptor(name);
+                    DynamicCacheDescriptor desc = cacheDescriptor(cctx.name());
 
                     assert desc != null;
 
@@ -1211,8 +1235,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheAdapter cache : reconnected)
             cache.context().gate().reconnected(false);
 
-        cachesOnDisconnect = null;
-
         if (stopFut != null)
             stopFut.markInitialized();
 
@@ -1735,17 +1757,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Collection of started cache names.
      */
     public Collection<String> cacheNames() {
-        return F.viewReadOnly(cacheDescriptors(),
-            new IgniteClosure<DynamicCacheDescriptor, String>() {
-                @Override public String apply(DynamicCacheDescriptor desc) {
-                    return desc.cacheConfiguration().getName();
-                }
-            },
-            new IgnitePredicate<DynamicCacheDescriptor>() {
-                @Override public boolean apply(DynamicCacheDescriptor desc) {
-                    return desc.started();
-                }
-            });
+        return F.viewReadOnly(cacheDescriptors(), new IgniteClosure<DynamicCacheDescriptor, String>() {
+            @Override public String apply(DynamicCacheDescriptor desc) {
+                return desc.cacheConfiguration().getName();
+            }
+        });
     }
 
     /**
@@ -1768,7 +1784,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         if (start) {
-            for (Map.Entry<String, DynamicCacheDescriptor> e : registeredCaches.entrySet()) {
+            for (Map.Entry<String, DynamicCacheDescriptor> e : cachesInfo.registeredCaches().entrySet()) {
                 DynamicCacheDescriptor desc = e.getValue();
 
                 CacheConfiguration ccfg = desc.cacheConfiguration();
@@ -1828,9 +1844,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         DynamicCacheDescriptor desc = cacheDescriptor(req.cacheName());
 
-        if (desc != null)
-            desc.onStart();
-
         prepareCacheStart(
             req.startCacheConfiguration(),
             req.nearCacheConfiguration(),
@@ -1838,48 +1851,55 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             req.clientStartOnly(),
             req.initiatingNodeId(),
             req.deploymentId(),
+            desc.startTopologyVersion(),
             topVer,
             desc != null ? desc.schema() : null
         );
     }
 
+    public void startCachesOnLocalJoin(AffinityTopologyVersion exchTopVer) throws IgniteCheckedException {
+        List<DynamicCacheDescriptor> caches = cachesInfo.cachesToStartOnLocalJoin();
+
+        for (DynamicCacheDescriptor desc : caches) {
+            prepareCacheStart(
+                desc.cacheConfiguration(),
+                null,
+                desc.cacheType(),
+                false,
+                null,
+                desc.deploymentId(),
+                desc.startTopologyVersion(),
+                exchTopVer,
+                desc.schema()
+            );
+        }
+    }
+
     /**
      * Starts statically configured caches received from remote nodes during exchange.
      *
-     * @param topVer Topology version.
+     * @param nodeId Joining node ID.
+     * @param exchTopVer Current exchange version.
      * @return Started caches descriptors.
      * @throws IgniteCheckedException If failed.
      */
-    public Collection<DynamicCacheDescriptor> startReceivedCaches(AffinityTopologyVersion topVer)
+    public Collection<DynamicCacheDescriptor> startReceivedCaches(UUID nodeId, AffinityTopologyVersion exchTopVer)
         throws IgniteCheckedException {
-        List<DynamicCacheDescriptor> started = null;
+        List<DynamicCacheDescriptor> started = cachesInfo.cachesReceivedFromJoin(nodeId);
 
-        for (DynamicCacheDescriptor desc : cacheDescriptors()) {
-            if (!desc.started() && desc.staticallyConfigured() && !desc.locallyConfigured()) {
-                if (desc.receivedFrom() != null) {
-                    AffinityTopologyVersion startVer = desc.receivedFromStartVersion();
-
-                    if (startVer == null || startVer.compareTo(topVer) > 0)
-                        continue;
-                }
-
-                if (desc.onStart()) {
-                    if (started == null)
-                        started = new ArrayList<>();
-
-                    started.add(desc);
-
-                    prepareCacheStart(
-                        desc.cacheConfiguration(),
-                        null,
-                        desc.cacheType(),
-                        false,
-                        null,
-                        desc.deploymentId(),
-                        topVer,
-                        desc.schema()
-                    );
-                }
+        if (started != null) {
+            for (DynamicCacheDescriptor desc : started) {
+                prepareCacheStart(
+                    desc.cacheConfiguration(),
+                    null,
+                    desc.cacheType(),
+                    false,
+                    null,
+                    desc.deploymentId(),
+                    desc.startTopologyVersion(),
+                    exchTopVer,
+                    desc.schema()
+                );
             }
         }
 
@@ -1893,7 +1913,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param clientStartOnly Client only start request.
      * @param initiatingNodeId Initiating node ID.
      * @param deploymentId Deployment ID.
-     * @param topVer Topology version.
+     * @param cacheStartTopVer Cache start topology version.
+     * @param exchTopVer Current exchange version.
      * @param schema Query schema.
      * @throws IgniteCheckedException If failed.
      */
@@ -1904,7 +1925,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         boolean clientStartOnly,
         UUID initiatingNodeId,
         IgniteUuid deploymentId,
-        AffinityTopologyVersion topVer,
+        AffinityTopologyVersion cacheStartTopVer,
+        AffinityTopologyVersion exchTopVer,
         @Nullable QuerySchema schema
     ) throws IgniteCheckedException {
         CacheConfiguration ccfg = new CacheConfiguration(cfg);
@@ -1916,8 +1938,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         boolean affNodeStart = !clientStartOnly && CU.affinityNode(locNode, nodeFilter);
         boolean clientNodeStart = locNode.id().equals(initiatingNodeId);
 
-        if (sharedCtx.cacheContext(CU.cacheId(cfg.getName())) != null)
-            return;
+        assert !caches.containsKey(ccfg.getName()) : ccfg.getName();
 
         if (affNodeStart || clientNodeStart || CU.isSystemCache(cfg.getName())) {
             if (clientNodeStart && !affNodeStart) {
@@ -1931,7 +1952,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx, true);
 
-            cacheCtx.startTopologyVersion(topVer);
+            cacheCtx.startTopologyVersion(exchTopVer);
+            cacheCtx.cacheStartTopologyVersion(cacheStartTopVer);
 
             cacheCtx.dynamicDeploymentId(deploymentId);
 
@@ -1950,7 +1972,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * @param req Stop request.
      */
-    public void blockGateway(DynamicCacheChangeRequest req) {
+    void blockGateway(DynamicCacheChangeRequest req) {
         assert req.stop() || req.close();
 
         if (req.stop() || (req.close() && req.initiatingNodeId().equals(ctx.localNodeId()))) {
@@ -2127,263 +2149,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
-        dataBag.addJoiningNodeData(CACHE_PROC.ordinal(), getDiscoveryData(dataBag.joiningNodeId()));
+        dataBag.addJoiningNodeData(CACHE_PROC.ordinal(), cachesInfo.joinDiscoveryData());
     }
 
     /** {@inheritDoc} */
     @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
-        dataBag.addNodeSpecificData(CACHE_PROC.ordinal(), getDiscoveryData(dataBag.joiningNodeId()));
-    }
-
-    /**
-     * @param joiningNodeId Joining node id.
-     */
-    private Serializable getDiscoveryData(UUID joiningNodeId) {
-        boolean reconnect = ctx.localNodeId().equals(joiningNodeId) && cachesOnDisconnect != null;
-
-        // Collect dynamically started caches to a single object.
-        Collection<DynamicCacheChangeRequest> reqs;
-
-        Map<String, Map<UUID, Boolean>> clientNodesMap;
-
-        if (reconnect) {
-            reqs = new ArrayList<>(caches.size() + 1);
-
-            clientNodesMap = U.newHashMap(caches.size());
-
-            collectDataOnReconnectingNode(reqs, clientNodesMap, joiningNodeId);
-        }
-        else {
-            reqs = new ArrayList<>(registeredCaches.size() + registeredTemplates.size() + 1);
-
-            clientNodesMap = ctx.discovery().clientNodesMap();
-
-            collectDataOnGridNode(reqs);
-        }
-
-        DynamicCacheChangeBatch batch = new DynamicCacheChangeBatch(reqs);
-
-        batch.clientNodes(clientNodesMap);
-
-        batch.clientReconnect(reconnect);
-
-        // Reset random batch ID so that serialized batches with the same descriptors will be exactly the same.
-        batch.id(null);
-
-        return batch;
-    }
-
-    /**
-     * @param reqs requests.
-     */
-    private void collectDataOnGridNode(Collection<DynamicCacheChangeRequest> reqs) {
-        for (DynamicCacheDescriptor desc : cacheDescriptors()) {
-            // RequestId must be null because on different node will be different byte [] and
-            // we get duplicate discovery data, for more details see
-            // TcpDiscoveryNodeAddedMessage#addDiscoveryData.
-            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(null, desc.cacheConfiguration().getName(),
-                null);
-
-            req.startCacheConfiguration(desc.cacheConfiguration());
-            req.cacheType(desc.cacheType());
-            req.deploymentId(desc.deploymentId());
-            req.receivedFrom(desc.receivedFrom());
-            req.schema(desc.schema());
-
-            reqs.add(req);
-        }
-
-        for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
-            // RequestId must be null because on different node will be different byte [] and
-            // we get duplicate discovery data, for more details see
-            // TcpDiscoveryNodeAddedMessage#addDiscoveryData.
-            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(null, desc.cacheConfiguration().getName(),
-                null);
-
-            req.startCacheConfiguration(desc.cacheConfiguration());
-            req.schema(desc.schema());
-
-            req.template(true);
-
-            reqs.add(req);
-        }
-    }
-
-    /**
-     * @param reqs requests.
-     * @param clientNodesMap Client nodes map.
-     * @param nodeId Node id.
-     */
-    private void collectDataOnReconnectingNode(
-            Collection<DynamicCacheChangeRequest> reqs,
-            Map<String, Map<UUID, Boolean>> clientNodesMap,
-            UUID nodeId
-    ) {
-        for (GridCacheAdapter<?, ?> cache : caches.values()) {
-            DynamicCacheDescriptor desc = cachesOnDisconnect.get(maskNull(cache.name()));
-
-            if (desc == null)
-                continue;
-
-            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(null, cache.name(), null);
-
-            req.startCacheConfiguration(desc.cacheConfiguration());
-            req.cacheType(desc.cacheType());
-            req.deploymentId(desc.deploymentId());
-            req.receivedFrom(desc.receivedFrom());
-            req.schema(desc.schema());
-
-            reqs.add(req);
-
-            Boolean nearEnabled = cache.isNear();
-
-            Map<UUID, Boolean> map = U.newHashMap(1);
-
-            map.put(nodeId, nearEnabled);
-
-            clientNodesMap.put(cache.name(), map);
-        }
+        if (!dataBag.commonDataCollectedFor(CACHE_PROC.ordinal()))
+            dataBag.addGridCommonData(CACHE_PROC.ordinal(), cachesInfo.collectCommonDiscoveryData());
     }
 
     /** {@inheritDoc} */
     @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
-        if (data.hasJoiningNodeData()) {
-            Serializable joiningNodeData = data.joiningNodeData();
-            if (joiningNodeData instanceof DynamicCacheChangeBatch)
-                onDiscoDataReceived(
-                        data.joiningNodeId(),
-                        data.joiningNodeId(),
-                        (DynamicCacheChangeBatch) joiningNodeData, true);
-        }
+        cachesInfo.onJoiningNodeDataReceived(data);
     }
 
     /** {@inheritDoc} */
     @Override public void onGridDataReceived(GridDiscoveryData data) {
-        Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
-
-        if (nodeSpecData != null) {
-            for (Map.Entry<UUID, Serializable> e : nodeSpecData.entrySet()) {
-                if (e.getValue() != null && e.getValue() instanceof DynamicCacheChangeBatch) {
-                    DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch) e.getValue();
-
-                    onDiscoDataReceived(data.joiningNodeId(), e.getKey(), batch, false);
-                }
-            }
-        }
-    }
-
-    /**
-     * @param joiningNodeId Joining node id.
-     * @param rmtNodeId Rmt node id.
-     * @param batch Batch.
-     * @param join Whether this is data from joining node.
-     */
-    private void onDiscoDataReceived(UUID joiningNodeId, UUID rmtNodeId, DynamicCacheChangeBatch batch, boolean join) {
-        if (batch.clientReconnect()) {
-            if (ctx.clientDisconnected()) {
-                if (clientReconnectReqs == null)
-                    clientReconnectReqs = new LinkedHashMap<>();
-
-                clientReconnectReqs.put(joiningNodeId, batch);
-
-                return;
-            }
-
-            processClientReconnectData(joiningNodeId, batch);
-        }
-        else {
-            for (DynamicCacheChangeRequest req : batch.requests()) {
-                initReceivedCacheConfiguration(req);
-
-                if (req.template()) {
-                    CacheConfiguration ccfg = req.startCacheConfiguration();
-
-                    assert ccfg != null : req;
-
-                    DynamicCacheDescriptor existing = registeredTemplates.get(maskNull(req.cacheName()));
-
-                    if (existing == null) {
-                        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
-                                ctx,
-                                ccfg,
-                                req.cacheType(),
-                                true,
-                                req.deploymentId(),
-                                req.schema());
-
-                        registeredTemplates.put(maskNull(req.cacheName()), desc);
-                    }
-
-                    continue;
-                }
-
-                DynamicCacheDescriptor existing = cacheDescriptor(req.cacheName());
-
-                if (req.start() && !req.clientStartOnly()) {
-                    CacheConfiguration ccfg = req.startCacheConfiguration();
-
-                    if (existing != null) {
-                        if (joiningNodeId.equals(ctx.localNodeId())) {
-                            existing.receivedFrom(req.receivedFrom());
-                            existing.deploymentId(req.deploymentId());
-                        }
-
-                        if (existing.locallyConfigured()) {
-                            existing.addRemoteConfiguration(rmtNodeId, req.startCacheConfiguration());
-
-                            if (!join)
-                                // Overwrite existing with remote.
-                                existing.schema(req.schema());
-
-                            ctx.discovery().setCacheFilter(
-                                    req.cacheName(),
-                                    ccfg.getNodeFilter(),
-                                    ccfg.getNearConfiguration() != null,
-                                    ccfg.getCacheMode());
-                        }
-                    }
-                    else {
-                        assert req.cacheType() != null : req;
-
-                        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
-                                ctx,
-                                ccfg,
-                                req.cacheType(),
-                                false,
-                                req.deploymentId(),
-                                req.schema());
-
-                        // Received statically configured cache.
-                        if (req.initiatingNodeId() == null)
-                            desc.staticallyConfigured(true);
-
-                        if (joiningNodeId.equals(ctx.localNodeId()))
-                            desc.receivedOnDiscovery(true);
-
-                        desc.receivedFrom(req.receivedFrom());
-
-                        DynamicCacheDescriptor old = cacheDescriptor(req.cacheName(), desc);
-
-                        assert old == null : old;
-
-                        ctx.discovery().setCacheFilter(
-                                req.cacheName(),
-                                ccfg.getNodeFilter(),
-                                ccfg.getNearConfiguration() != null,
-                                ccfg.getCacheMode());
-                    }
-                }
-            }
-
-            if (!F.isEmpty(batch.clientNodes())) {
-                for (Map.Entry<String, Map<UUID, Boolean>> entry : batch.clientNodes().entrySet()) {
-                    String cacheName = entry.getKey();
-
-                    for (Map.Entry<UUID, Boolean> tup : entry.getValue().entrySet())
-                        ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue());
-                }
-            }
-        }
+        cachesInfo.onGridDataReceived(data);
     }
 
     /**
@@ -2469,7 +2251,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         List<CacheConfiguration> wildcardNameCfgs = null;
 
-        for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
+        for (DynamicCacheDescriptor desc : cachesInfo.registeredTemplates().values()) {
             assert desc.template();
 
             CacheConfiguration cfg = desc.cacheConfiguration();
@@ -2744,7 +2526,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         checkEmptyTransactions();
 
         if (F.isEmpty(cacheNames))
-            cacheNames = registeredCaches.keySet();
+            cacheNames = cachesInfo.registeredCaches().keySet();
 
         Collection<DynamicCacheChangeRequest> reqs = new ArrayList<>(cacheNames.size());
 
@@ -2965,12 +2747,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param topVer Topology version.
      */
     public void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
-        if (type == EVT_NODE_JOINED) {
-            for (DynamicCacheDescriptor cacheDesc : cacheDescriptors()) {
-                if (node.id().equals(cacheDesc.receivedFrom()))
-                    cacheDesc.receivedFromStartVersion(topVer);
-            }
-        }
+        cachesInfo.onDiscoveryEvent(type, node, topVer);
 
         sharedCtx.affinity().onDiscoveryEvent(type, node, topVer);
     }
@@ -2997,11 +2774,28 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             return true;
 
         if (msg instanceof DynamicCacheChangeBatch)
-            return onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer);
+            return cachesInfo.onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer);
 
         return false;
     }
 
+    void completeTemplateAddFuture(String name, IgniteUuid deploymentId) {
+        GridCacheProcessor.TemplateConfigurationFuture fut =
+            (GridCacheProcessor.TemplateConfigurationFuture)pendingTemplateFuts.get(name);
+
+        if (fut != null && fut.deploymentId().equals(deploymentId))
+            fut.onDone();
+    }
+
+    void completeCacheStartFuture(DynamicCacheChangeRequest req, Exception err) {
+        if (ctx.localNodeId().equals(req.initiatingNodeId())) {
+            DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
+
+            if (fut != null && F.eq(req.deploymentId(), fut.deploymentId()))
+                fut.onDone(err);
+        }
+    }
+
     /**
      * @param batch Change request batch.
      * @param topVer Current topology version.
@@ -3023,13 +2817,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 assert ccfg != null : req;
 
-                DynamicCacheDescriptor desc = registeredTemplates.get(maskNull(req.cacheName()));
+                DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(maskNull(req.cacheName()));
 
                 if (desc == null) {
                     DynamicCacheDescriptor templateDesc = new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), true,
                         req.deploymentId(), req.schema());
 
-                    DynamicCacheDescriptor old = registeredTemplates.put(maskNull(ccfg.getName()), templateDesc);
+                    DynamicCacheDescriptor old = cachesInfo.registeredTemplates().put(maskNull(ccfg.getName()), templateDesc);
 
                     assert old == null :
                         "Dynamic cache map was concurrently modified [new=" + templateDesc + ", old=" + old + ']';
@@ -3080,7 +2874,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                         startDesc.startTopologyVersion(newTopVer);
 
-                        DynamicCacheDescriptor old = cacheDescriptor(ccfg.getName(), startDesc);
+                        // TODO
+                        DynamicCacheDescriptor old = null;//cacheDescriptor(ccfg.getName(), startDesc);
 
                         assert old == null :
                             "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
@@ -3152,7 +2947,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 if (desc != null) {
                     if (req.stop()) {
-                        DynamicCacheDescriptor old = registeredCaches.remove(maskNull(req.cacheName()));
+                        DynamicCacheDescriptor old = cachesInfo.registeredCaches().remove(maskNull(req.cacheName()));
 
                         assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
 
@@ -3610,25 +3405,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Descriptor.
      */
     public DynamicCacheDescriptor cacheDescriptor(String name) {
-        return registeredCaches.get(maskNull(name));
-    }
-
-    /**
-     * Put registered cache descriptor.
-     *
-     * @param name Name.
-     * @param desc Descriptor.
-     * @return Old descriptor (if any).
-     */
-    private DynamicCacheDescriptor cacheDescriptor(String name, DynamicCacheDescriptor desc) {
-        return registeredCaches.put(maskNull(name), desc);
+        return cachesInfo.registeredCaches().get(name);
     }
 
     /**
      * @return Cache descriptors.
      */
     public Collection<DynamicCacheDescriptor> cacheDescriptors() {
-        return registeredCaches.values();
+        return cachesInfo.registeredCaches().values();
     }
 
     /**
@@ -3655,7 +3439,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public void addCacheConfiguration(CacheConfiguration cacheCfg) throws IgniteCheckedException {
         String masked = maskNull(cacheCfg.getName());
 
-        DynamicCacheDescriptor desc = registeredTemplates.get(masked);
+        DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(masked);
 
         if (desc != null)
             return;
@@ -3833,7 +3617,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException In case of error.
      */
     public void createMissingQueryCaches() throws IgniteCheckedException {
-        for (Map.Entry<String, DynamicCacheDescriptor> e : registeredCaches.entrySet()) {
+        for (Map.Entry<String, DynamicCacheDescriptor> e : cachesInfo.registeredCaches().entrySet()) {
             DynamicCacheDescriptor desc = e.getValue();
 
             if (isMissingQueryCache(desc))


[16/26] ignite git commit: IGNITE-5122: Fixed http-rest-example.php.

Posted by sb...@apache.org.
IGNITE-5122: Fixed http-rest-example.php.


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

Branch: refs/heads/ignite-5075
Commit: a04aa10a38a5e8c89223f567b4ce4822490509b2
Parents: cd46d46
Author: devozerov <pp...@gmail.com>
Authored: Sun Apr 30 11:23:14 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Sun Apr 30 11:23:14 2017 +0300

----------------------------------------------------------------------
 examples/rest/http-rest-example.php | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a04aa10a/examples/rest/http-rest-example.php
----------------------------------------------------------------------
diff --git a/examples/rest/http-rest-example.php b/examples/rest/http-rest-example.php
index fe20f4e..df565b2 100644
--- a/examples/rest/http-rest-example.php
+++ b/examples/rest/http-rest-example.php
@@ -43,8 +43,8 @@ if (!function_exists('json_decode'))
 
 $URL = 'http://localhost:8080/ignite?';
 
-// Cache name to use (null or empty string for default cache).
-$CACHE_NAME = null;
+// Cache name to use.
+$CACHE_NAME = 'default';
 
 /**
  * Creates URL parameters.


[03/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java


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

Branch: refs/heads/ignite-5075
Commit: 07aa9bbe2fe598b97988c28ea533f0bb3c2d09a9
Parents: c73e1c9 4d26061
Author: sboikov <sb...@gridgain.com>
Authored: Wed Apr 26 20:37:38 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Apr 26 20:37:38 2017 +0300

----------------------------------------------------------------------
 assembly/dependencies-fabric.xml                |    5 +-
 examples/config/example-cache.xml               |    1 +
 examples/pom-standalone-lgpl.xml                |    2 +-
 examples/pom-standalone.xml                     |    2 +-
 examples/pom.xml                                |   38 +-
 .../hibernate/HibernateL2CacheExample.java      |   12 +-
 .../apache/ignite/gridify/AbstractAopTest.java  |    4 +-
 .../gridify/ExternalNonSpringAopSelfTest.java   |    6 +-
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |    7 +-
 .../stream/camel/IgniteCamelStreamerTest.java   |   10 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |   37 +-
 .../client/ClientDefaultCacheSelfTest.java      |    2 +-
 .../ClientAbstractMultiNodeSelfTest.java        |   25 +-
 .../integration/ClientAbstractSelfTest.java     |    9 +-
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |   16 +-
 .../jdbc2/JdbcComplexQuerySelfTest.java         |    5 +-
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |    5 +-
 .../jdbc2/JdbcDistributedJoinsQueryTest.java    |    6 +-
 .../jdbc2/JdbcDynamicIndexAbstractSelfTest.java |   14 +-
 .../jdbc2/JdbcInsertStatementSelfTest.java      |    4 +-
 .../jdbc2/JdbcMergeStatementSelfTest.java       |    4 +-
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |   31 +-
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  |   45 +-
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |    4 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |    4 +-
 .../internal/jdbc2/JdbcStatementSelfTest.java   |    4 +-
 .../internal/jdbc2/JdbcStreamingSelfTest.java   |   14 +-
 .../rest/AbstractRestProcessorSelfTest.java     |    2 +-
 .../JettyRestProcessorAbstractSelfTest.java     |  184 +-
 .../rest/JettyRestProcessorSignedSelfTest.java  |    4 +-
 .../rest/RestBinaryProtocolSelfTest.java        |  101 +-
 .../rest/RestMemcacheProtocolSelfTest.java      |   47 +-
 .../processors/rest/RestProcessorTest.java      |    2 +-
 .../rest/TaskCommandHandlerSelfTest.java        |    7 +-
 .../processors/rest/TestBinaryClient.java       |   23 +-
 .../tcp/redis/RedisProtocolSelfTest.java        |    8 +-
 .../jdbc/AbstractJdbcPojoQuerySelfTest.java     |    2 +-
 .../ignite/jdbc/JdbcConnectionSelfTest.java     |    5 +-
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |    8 +-
 .../ignite/jdbc/JdbcNoDefaultCacheTest.java     |    3 +-
 .../ignite/jdbc/JdbcPojoQuerySelfTest.java      |    2 +-
 .../jdbc/JdbcPreparedStatementSelfTest.java     |    2 +-
 .../ignite/jdbc/JdbcResultSetSelfTest.java      |    2 +-
 .../ignite/jdbc/JdbcStatementSelfTest.java      |    2 +-
 .../clients/src/test/resources/spring-cache.xml |    1 -
 .../src/main/java/org/apache/ignite/Ignite.java |   10 +-
 .../java/org/apache/ignite/IgniteCompute.java   |    8 +-
 .../org/apache/ignite/cache/QueryEntity.java    |   81 +
 .../apache/ignite/cluster/ClusterMetrics.java   |    4 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   11 +-
 .../configuration/CacheConfiguration.java       |   18 +-
 .../configuration/IgniteConfiguration.java      |   52 +-
 .../org/apache/ignite/events/EventType.java     |    2 +-
 .../ignite/internal/IgniteComputeImpl.java      |   37 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   12 +-
 .../apache/ignite/internal/IgniteKernal.java    |   76 +-
 .../internal/cluster/ClusterGroupAdapter.java   |   23 +-
 .../ignite/internal/cluster/ClusterGroupEx.java |    5 +-
 .../affinity/GridAffinityProcessor.java         |   67 +-
 .../cache/DynamicCacheDescriptor.java           |   24 +
 .../processors/cache/GridCacheProcessor.java    |  170 +-
 .../cache/GridCacheSharedContext.java           |   18 +-
 .../processors/cache/GridCacheUtils.java        |   30 +-
 .../cache/query/GridCacheQueryManager.java      |   14 +-
 .../cache/transactions/IgniteTxManager.java     |   20 +-
 .../cache/transactions/TxDeadlock.java          |   19 +-
 .../cache/transactions/TxLocksResponse.java     |   37 +-
 .../datastructures/GridCacheLockImpl.java       |    6 +
 .../processors/job/GridJobProcessor.java        |   18 +-
 .../utils/PlatformConfigurationUtils.java       |   37 +-
 .../processors/query/GridQueryIndexing.java     |   23 +-
 .../query/GridQueryTypeDescriptor.java          |   12 +
 .../query/QueryTypeDescriptorImpl.java          |   34 +-
 .../internal/processors/query/QueryUtils.java   |  134 +-
 .../handlers/cache/GridCacheCommandHandler.java |    4 +-
 .../redis/GridRedisRestCommandHandler.java      |    3 +
 .../redis/key/GridRedisDelCommandHandler.java   |    1 +
 .../key/GridRedisExistsCommandHandler.java      |    1 +
 .../server/GridRedisDbSizeCommandHandler.java   |    1 +
 .../string/GridRedisAppendCommandHandler.java   |    3 +
 .../string/GridRedisGetCommandHandler.java      |    1 +
 .../string/GridRedisGetRangeCommandHandler.java |    1 +
 .../string/GridRedisGetSetCommandHandler.java   |    1 +
 .../string/GridRedisIncrDecrCommandHandler.java |    2 +
 .../string/GridRedisMGetCommandHandler.java     |    1 +
 .../string/GridRedisMSetCommandHandler.java     |    1 +
 .../string/GridRedisSetCommandHandler.java      |    1 +
 .../string/GridRedisSetRangeCommandHandler.java |    2 +
 .../string/GridRedisStrlenCommandHandler.java   |    1 +
 .../tcp/GridTcpMemcachedNioListener.java        |    5 +-
 .../visor/node/VisorBasicConfiguration.java     |   13 +
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   32 +
 .../spi/IgniteSpiOperationTimeoutHelper.java    |    8 +-
 .../jobstealing/JobStealingCollisionSpi.java    |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    6 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   31 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  142 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  143 +-
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |   26 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   33 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   72 -
 .../TcpDiscoveryClientMetricsUpdateMessage.java |   72 +
 .../messages/TcpDiscoveryHeartbeatMessage.java  |  338 ---
 .../TcpDiscoveryMetricsUpdateMessage.java       |  338 +++
 .../spi/indexing/IndexingQueryFilter.java       |    2 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |   12 +-
 .../resources/META-INF/classnames.properties    |    8 +-
 .../core/src/test/config/discovery-stress.xml   |    2 +-
 .../core/src/test/config/load/dsi-load-base.xml |    3 +-
 .../src/test/config/load/merge-sort-base.xml    |    7 +-
 .../core/src/test/config/spring-cache-load.xml  |    1 +
 .../core/src/test/config/spring-cache-swap.xml  |    2 +
 .../src/test/config/spring-cache-teststore.xml  |    2 +
 .../test/config/store/jdbc/ignite-jdbc-type.xml |    6 +
 .../config/streamer/spring-streamer-base.xml    |    5 +-
 .../GridCacheAffinityBackupsSelfTest.java       |    2 +-
 .../java/org/apache/ignite/GridTestJob.java     |   19 +
 .../apache/ignite/GridTestStoreNodeStartup.java |    2 +-
 .../java/org/apache/ignite/GridTestTask.java    |   18 +-
 .../ignite/IgniteCacheAffinitySelfTest.java     |    2 +-
 .../cache/IgniteWarmupClosureSelfTest.java      |    2 +-
 .../ignite/cache/LargeEntryUpdateTest.java      |    2 +-
 .../affinity/AffinityClientNodeSelfTest.java    |   14 +-
 ...ityFunctionBackupFilterAbstractSelfTest.java |    8 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    4 +-
 .../affinity/AffinityHistoryCleanupTest.java    |    2 +-
 .../local/LocalAffinityFunctionTest.java        |    2 +-
 ...cheStoreSessionListenerAbstractSelfTest.java |    6 +-
 .../store/GridCacheBalancingStoreSelfTest.java  |    8 +-
 .../IgniteCacheExpiryStoreLoadSelfTest.java     |    2 +-
 .../store/StoreResourceInjectionSelfTest.java   |    2 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |    8 +-
 .../internal/ClusterNodeMetricsSelfTest.java    |   10 +-
 .../ignite/internal/GridAffinityMappedTest.java |   11 +-
 .../internal/GridAffinityP2PSelfTest.java       |    9 +-
 .../ignite/internal/GridAffinitySelfTest.java   |    9 +-
 .../GridCancelledJobsMetricsSelfTest.java       |    4 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |    2 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    4 +-
 .../GridJobCollisionCancelSelfTest.java         |    2 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |    8 +-
 .../GridProjectionForCachesSelfTest.java        |   13 +-
 ...ectionLocalJobMultipleArgumentsSelfTest.java |    4 +-
 .../ignite/internal/GridStartStopSelfTest.java  |    8 +-
 .../GridTaskFailoverAffinityRunTest.java        |    4 +-
 .../IgniteClientReconnectApiExceptionTest.java  |   18 +-
 .../IgniteClientReconnectCacheTest.java         |   70 +-
 ...eClientReconnectContinuousProcessorTest.java |   10 +-
 .../IgniteClientReconnectFailoverTest.java      |    4 +-
 .../internal/IgniteClientReconnectStopTest.java |    2 +-
 .../IgniteComputeEmptyClusterGroupTest.java     |    8 +-
 ...eConcurrentEntryProcessorAccessStopTest.java |    2 +-
 .../internal/binary/BinaryEnumsSelfTest.java    |    2 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |    2 +-
 .../internal/binary/BinaryTreeSelfTest.java     |    4 +-
 .../binary/GridBinaryAffinityKeySelfTest.java   |   30 +-
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |    4 +-
 .../IgniteVariousConnectionNumberTest.java      |    4 +-
 .../GridDeploymentMessageCountSelfTest.java     |    6 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    6 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    6 +-
 .../GridCacheTxLoadFromStoreOnLockSelfTest.java |    2 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    2 +-
 .../CacheAtomicSingleMessageCountSelfTest.java  |    4 +-
 .../cache/CacheClientStoreSelfTest.java         |    2 +-
 .../cache/CacheConcurrentReadThroughTest.java   |    2 +-
 .../cache/CacheConfigurationLeakTest.java       |    2 +-
 .../cache/CacheDeferredDeleteQueueTest.java     |    4 +-
 .../CacheDeferredDeleteSanitySelfTest.java      |    4 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |    6 +-
 .../cache/CacheEnumOperationsAbstractTest.java  |    2 +-
 ...CacheExchangeMessageDuplicatedStateTest.java |   10 +-
 .../cache/CacheFutureExceptionSelfTest.java     |    2 +-
 .../cache/CacheGetEntryAbstractTest.java        |   16 +-
 .../processors/cache/CacheGetFromJobTest.java   |    2 +-
 ...erceptorPartitionCounterLocalSanityTest.java |    2 +-
 ...torPartitionCounterRandomOperationsTest.java |    2 +-
 .../CacheMemoryPolicyConfigurationTest.java     |   10 +-
 .../processors/cache/CacheNamesSelfTest.java    |    8 +-
 .../CacheNamesWithSpecialCharactersTest.java    |    4 +-
 .../cache/CacheNearReaderUpdateTest.java        |    2 +-
 ...cheNearUpdateTopologyChangeAbstractTest.java |    8 +-
 .../cache/CacheOffheapMapEntrySelfTest.java     |    2 +-
 .../processors/cache/CachePutIfAbsentTest.java  |    2 +-
 .../cache/CacheReadThroughRestartSelfTest.java  |    8 +-
 .../cache/CacheRebalancingSelfTest.java         |    4 +-
 .../cache/CacheRemoveAllSelfTest.java           |    4 +-
 .../CacheSerializableTransactionsTest.java      |    6 +-
 .../CacheStartupInDeploymentModesTest.java      |    4 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   |    8 +-
 ...eUsageMultinodeDynamicStartAbstractTest.java |    4 +-
 .../processors/cache/CacheTxFastFinishTest.java |    4 +-
 .../processors/cache/CrossCacheLockTest.java    |    4 +-
 .../cache/CrossCacheTxRandomOperationsTest.java |    2 +-
 .../EntryVersionConsistencyReadThroughTest.java |   10 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    8 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  114 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |   26 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |  102 +-
 .../GridCacheAbstractRemoveFailureTest.java     |    6 +-
 .../cache/GridCacheAbstractSelfTest.java        |    8 +-
 ...acheAbstractUsersAffinityMapperSelfTest.java |    7 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |   26 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |   10 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    6 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |   30 +-
 .../cache/GridCacheClearLocallySelfTest.java    |    8 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |   10 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |   10 +-
 .../GridCacheConditionalDeploymentSelfTest.java |    4 +-
 ...idCacheConfigurationConsistencySelfTest.java |    2 +-
 .../GridCacheDaemonNodeAbstractSelfTest.java    |    8 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   46 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |    4 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |   16 +-
 .../GridCacheEvictionEventAbstractTest.java     |    2 +-
 .../GridCacheFinishPartitionsSelfTest.java      |   16 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |    2 +-
 .../cache/GridCacheIncrementTransformTest.java  |    4 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |    8 +-
 .../cache/GridCacheIteratorPerformanceTest.java |    6 +-
 .../cache/GridCacheKeyCheckSelfTest.java        |    6 +-
 .../GridCacheMarshallerTxAbstractTest.java      |   10 +-
 .../GridCacheMarshallingNodeJoinSelfTest.java   |    4 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |    2 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    4 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |    6 +-
 ...ridCacheMultinodeUpdateAbstractSelfTest.java |    6 +-
 .../cache/GridCacheMvccFlagsTest.java           |    4 +-
 .../cache/GridCacheMvccManagerSelfTest.java     |    9 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |   34 +-
 .../processors/cache/GridCacheMvccSelfTest.java |   58 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |   12 +-
 .../cache/GridCacheObjectToStringSelfTest.java  |    4 +-
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |   20 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |   14 +-
 .../cache/GridCacheOffheapUpdateSelfTest.java   |   14 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |   16 +-
 ...hePartitionedProjectionAffinitySelfTest.java |    8 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |   14 +-
 .../GridCacheQueryIndexingDisabledSelfTest.java |    2 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |    6 +-
 .../GridCacheReferenceCleanupSelfTest.java      |   10 +-
 ...ridCacheReplicatedSynchronousCommitTest.java |    6 +-
 .../GridCacheReturnValueTransferSelfTest.java   |    4 +-
 .../processors/cache/GridCacheStopSelfTest.java |    8 +-
 ...ridCacheStoreManagerDeserializationTest.java |    3 +
 .../cache/GridCacheStorePutxSelfTest.java       |    2 +-
 .../cache/GridCacheStoreValueBytesSelfTest.java |    4 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |   10 +-
 ...acheTcpClientDiscoveryMultiThreadedTest.java |    4 +-
 ...cheTransactionalAbstractMetricsSelfTest.java |    8 +-
 .../GridCacheTtlManagerEvictionSelfTest.java    |    6 +-
 .../cache/GridCacheTtlManagerLoadTest.java      |    4 +-
 .../GridCacheTtlManagerNotificationTest.java    |    8 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |   12 +-
 .../GridCacheValueBytesPreloadingSelfTest.java  |   12 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |   14 +-
 .../GridCacheVariableTopologySelfTest.java      |    4 +-
 .../cache/GridCacheVersionMultinodeTest.java    |    2 +-
 .../GridCacheVersionTopologyChangeTest.java     |    2 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java |   16 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    3 +-
 .../cache/IgniteCacheAbstractTest.java          |   12 +-
 ...IgniteCacheBinaryEntryProcessorSelfTest.java |    6 +-
 ...teCacheConfigurationDefaultTemplateTest.java |    6 +-
 .../IgniteCacheConfigurationTemplateTest.java   |   29 +-
 ...niteCacheCopyOnReadDisabledAbstractTest.java |    2 +-
 .../cache/IgniteCacheDynamicStopSelfTest.java   |   10 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |    6 +-
 ...niteCacheEntryListenerExpiredEventsTest.java |    2 +-
 .../IgniteCacheEntryProcessorCallTest.java      |    8 +-
 .../IgniteCacheEntryProcessorNodeJoinTest.java  |   16 +-
 ...niteCacheExpireAndUpdateConsistencyTest.java |    2 +-
 ...IgniteCacheGetCustomCollectionsSelfTest.java |    2 +-
 .../cache/IgniteCacheIncrementTxTest.java       |   10 +-
 .../cache/IgniteCacheInvokeAbstractTest.java    |    2 +-
 ...gniteCacheInvokeReadThroughAbstractTest.java |    2 +-
 ...gniteCacheLoadRebalanceEvictionSelfTest.java |    6 +-
 .../IgniteCacheManyAsyncOperationsTest.java     |    2 +-
 .../cache/IgniteCacheObjectPutSelfTest.java     |    2 +-
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |    4 +-
 .../IgniteCachePartitionMapUpdateTest.java      |    4 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |   22 +-
 .../IgniteCacheReadThroughStoreCallTest.java    |    2 +-
 ...iteCacheScanPredicateDeploymentSelfTest.java |    2 +-
 .../cache/IgniteCacheSerializationSelfTest.java |    4 +-
 .../cache/IgniteCacheStartStopLoadTest.java     |    2 +-
 .../cache/IgniteCacheStoreCollectionTest.java   |    4 +-
 .../IgniteCacheStoreValueAbstractTest.java      |   30 +-
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |   12 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |   22 +-
 .../IgniteDaemonNodeMarshallerCacheTest.java    |    2 +-
 .../cache/IgniteDynamicCacheAndNodeStop.java    |    4 +-
 .../cache/IgniteDynamicCacheFilterTest.java     |   10 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java |   34 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   36 +-
 ...niteDynamicCacheStartStopConcurrentTest.java |    6 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |   44 +-
 .../cache/IgniteExchangeFutureHistoryTest.java  |    2 +-
 ...iteMarshallerCacheClassNameConflictTest.java |    8 +-
 ...lerCacheClientRequestsMappingOnMissTest.java |   22 +-
 ...eMarshallerCacheConcurrentReadWriteTest.java |    8 +-
 .../cache/IgniteOnePhaseCommitNearSelfTest.java |    8 +-
 .../cache/IgnitePutAllLargeBatchSelfTest.java   |   12 +-
 ...tAllUpdateNonPreloadedPartitionSelfTest.java |    6 +-
 .../IgniteStartCacheInTransactionSelfTest.java  |   18 +-
 .../cache/IgniteStaticCacheStartSelfTest.java   |    2 +-
 ...gniteTopologyValidatorAbstractCacheTest.java |   37 +-
 ...iteTopologyValidatorAbstractTxCacheTest.java |   20 +-
 ...niteTopologyValidatorGridSplitCacheTest.java |    2 +-
 .../processors/cache/IgniteTxAbstractTest.java  |    6 +-
 .../IgniteTxConcurrentGetAbstractTest.java      |    6 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |    2 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   36 +-
 .../cache/IgniteTxMultiNodeAbstractTest.java    |   58 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |    4 +-
 .../cache/IgniteTxReentryAbstractSelfTest.java  |    2 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |   34 +-
 .../binary/BinaryMetadataUpdatesFlowTest.java   |   14 +-
 .../CacheKeepBinaryWithInterceptorTest.java     |    6 +-
 ...yAtomicEntryProcessorDeploymentSelfTest.java |    6 +-
 ...naryObjectMetadataExchangeMultinodeTest.java |   16 +-
 ...acheBinaryObjectUserClassloaderSelfTest.java |    4 +-
 ...naryObjectsAbstractDataStreamerSelfTest.java |    4 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |    2 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |    8 +-
 .../GridCacheBinaryStoreAbstractSelfTest.java   |    2 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |    8 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |    4 +-
 .../GridDataStreamerImplSelfTest.java           |   20 +-
 ...IgniteCacheAbstractExecutionContextTest.java |    4 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |   66 +-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |    2 +-
 ...idCacheAtomicStampedApiSelfAbstractTest.java |    2 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |    2 +-
 .../GridCacheQueueCleanupSelfTest.java          |   10 +-
 .../GridCacheSequenceApiSelfAbstractTest.java   |    2 +-
 .../GridCacheSetAbstractSelfTest.java           |    2 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |    2 +-
 .../IgniteAtomicLongApiAbstractSelfTest.java    |    2 +-
 .../IgniteCountDownLatchAbstractSelfTest.java   |    2 +-
 .../IgniteLockAbstractSelfTest.java             |    2 +-
 .../IgniteSemaphoreAbstractSelfTest.java        |    2 +-
 ...achePartitionedAtomicSequenceTxSelfTest.java |    2 +-
 ...idCachePartitionedNodeRestartTxSelfTest.java |   24 +-
 ...PartitionedQueueCreateMultiNodeSelfTest.java |    4 +-
 ...acheAsyncOperationsFailoverAbstractTest.java |    4 +-
 .../distributed/CacheAsyncOperationsTest.java   |    2 +-
 .../CacheGetFutureHangsSelfTest.java            |    4 +-
 .../CacheGetInsideLockChangingTopologyTest.java |    2 +-
 .../CacheLateAffinityAssignmentTest.java        |   11 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |   20 +-
 .../CacheLockReleaseNodeLeaveTest.java          |   26 +-
 .../CachePutAllFailoverAbstractTest.java        |    2 +-
 .../CacheTryLockMultithreadedTest.java          |    4 +-
 .../GridCacheAbstractJobExecutionTest.java      |   14 +-
 ...tractPartitionedByteArrayValuesSelfTest.java |    2 +-
 .../GridCacheAbstractPrimarySyncSelfTest.java   |    6 +-
 .../GridCacheBasicOpAbstractTest.java           |   26 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   20 +-
 .../GridCacheEntrySetAbstractSelfTest.java      |    2 +-
 .../distributed/GridCacheLockAbstractTest.java  |    6 +-
 .../distributed/GridCacheMixedModeSelfTest.java |    4 +-
 .../GridCacheMultiNodeAbstractTest.java         |   10 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |   27 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |    2 +-
 .../GridCacheNodeFailureAbstractTest.java       |    9 +-
 ...ridCachePartitionNotLoadedEventSelfTest.java |    4 +-
 ...chePartitionedReloadAllAbstractSelfTest.java |    8 +-
 .../GridCachePreloadEventsAbstractSelfTest.java |    4 +-
 .../GridCacheTransformEventSelfTest.java        |    2 +-
 ...niteBinaryMetadataUpdateNodeRestartTest.java |    2 +-
 .../distributed/IgniteCache150ClientsTest.java  |    4 +-
 ...niteCacheClientNodeChangingTopologyTest.java |  130 +-
 .../IgniteCacheClientNodeConcurrentStart.java   |    2 +-
 ...teCacheClientNodePartitionsExchangeTest.java |   16 +-
 .../IgniteCacheClientReconnectTest.java         |    2 +-
 .../IgniteCacheConnectionRecoveryTest.java      |    2 +-
 .../distributed/IgniteCacheCreatePutTest.java   |    6 +-
 .../distributed/IgniteCacheGetRestartTest.java  |    2 +-
 .../distributed/IgniteCacheManyClientsTest.java |    6 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |    4 +-
 ...eCacheMessageRecoveryIdleConnectionTest.java |    2 +-
 .../IgniteCacheNearRestartRollbackSelfTest.java |    6 +-
 .../distributed/IgniteCachePrimarySyncTest.java |    4 +-
 .../IgniteCacheReadFromBackupTest.java          |    2 +-
 .../IgniteCacheServerNodeConcurrentStart.java   |    6 +-
 .../IgniteCacheSingleGetMessageTest.java        |    2 +-
 .../IgniteCacheSizeFailoverTest.java            |    6 +-
 .../IgniteCacheSystemTransactionsSelfTest.java  |    6 +-
 .../IgniteNoClassOnServerAbstractTest.java      |    2 +-
 .../IgniteTxCachePrimarySyncTest.java           |   45 +-
 ...teSynchronizationModesMultithreadedTest.java |   15 +-
 ...iteTxConsistencyRestartAbstractSelfTest.java |   12 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   10 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   22 +-
 .../IgniteTxTimeoutAbstractTest.java            |    2 +-
 ...heAbstractTransformWriteThroughSelfTest.java |    2 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |   50 +-
 .../dht/GridCacheColocatedDebugTest.java        |  112 +-
 ...eColocatedOptimisticTransactionSelfTest.java |    2 +-
 ...dCacheColocatedTxSingleThreadedSelfTest.java |    2 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |    6 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    8 +-
 .../dht/GridCacheDhtMappingSelfTest.java        |    4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   10 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   26 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   10 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |    8 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |    4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   18 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |    4 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   24 +-
 ...ePartitionedNearDisabledMetricsSelfTest.java |   12 +-
 ...idCachePartitionedPreloadEventsSelfTest.java |    6 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   18 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   10 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   10 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   12 +-
 .../IgniteCacheCommitDelayTxRecoveryTest.java   |   22 +-
 .../dht/IgniteCacheConcurrentPutGetRemove.java  |    2 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |    5 +-
 .../dht/IgniteCacheLockFailoverSelfTest.java    |    8 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    2 +-
 ...artitionedBackupNodeFailureRecoveryTest.java |    6 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |   12 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |    8 +-
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |    4 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |    6 +-
 .../dht/IgniteCacheTxRecoveryRollbackTest.java  |   20 +-
 .../dht/IgniteTxReentryColocatedSelfTest.java   |    2 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   12 +-
 .../atomic/GridCacheAtomicPreloadSelfTest.java  |   10 +-
 .../atomic/IgniteCacheAtomicProtocolTest.java   |    4 +-
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |    6 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |    2 +-
 ...AtomicPartitionedTckMetricsSelfTestImpl.java |    8 +-
 .../near/GridCacheGetStoreErrorSelfTest.java    |    4 +-
 .../near/GridCacheNearEvictionSelfTest.java     |    6 +-
 .../near/GridCacheNearMetricsSelfTest.java      |   32 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |    6 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |   14 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   10 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |   16 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |    2 +-
 .../GridCacheNearReaderPreloadSelfTest.java     |    2 +-
 .../near/GridCacheNearReadersSelfTest.java      |   36 +-
 .../near/GridCacheNearTxForceKeyTest.java       |    8 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |   18 +-
 ...AffinityExcludeNeighborsPerformanceTest.java |    2 +-
 .../GridCachePartitionedAffinitySelfTest.java   |    8 +-
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedEvictionSelfTest.java   |    2 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |    4 +-
 ...GridCachePartitionedFilteredPutSelfTest.java |    2 +-
 .../GridCachePartitionedFullApiSelfTest.java    |    8 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |    4 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |    2 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |   24 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   34 +-
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    8 +-
 .../GridCachePartitionedStorePutSelfTest.java   |    6 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |    2 +-
 ...achePartitionedTxSingleThreadedSelfTest.java |    2 +-
 .../near/GridCachePutArrayValueSelfTest.java    |    2 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    2 +-
 .../near/GridPartitionedBackupLoadSelfTest.java |    4 +-
 .../near/IgniteCacheNearOnlyTxTest.java         |   24 +-
 .../near/IgniteCacheNearReadCommittedTest.java  |    4 +-
 .../near/IgniteTxReentryNearSelfTest.java       |    2 +-
 .../near/NearCacheMultithreadedUpdateTest.java  |    8 +-
 .../near/NearCachePutAllMultinodeTest.java      |    4 +-
 .../near/NearCacheSyncUpdateTest.java           |    4 +-
 .../near/NoneRebalanceModeSelfTest.java         |    4 +-
 ...cingDelayedPartitionMapExchangeSelfTest.java |    8 +-
 .../GridCacheRebalancingOrderingTest.java       |    2 +-
 .../GridCacheRebalancingSyncCheckDataTest.java  |    6 +-
 .../GridCacheRebalancingSyncSelfTest.java       |   10 +-
 ...eRebalancingUnmarshallingFailedSelfTest.java |    2 +-
 ...stractReplicatedByteArrayValuesSelfTest.java |    2 +-
 .../GridCacheSyncReplicatedPreloadSelfTest.java |   14 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |   50 +-
 ...eplicatedPreloadStartStopEventsSelfTest.java |    2 +-
 .../cache/eviction/EvictionAbstractTest.java    |   12 +-
 ...heConcurrentEvictionConsistencySelfTest.java |    2 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |    2 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |    2 +-
 .../GridCacheEvictionFilterSelfTest.java        |    4 +-
 .../GridCacheEvictionTouchSelfTest.java         |    8 +-
 .../lru/LruNearEvictionPolicySelfTest.java      |    6 +-
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |    8 +-
 .../paged/PageEvictionAbstractTest.java         |    5 +-
 .../SortedEvictionPolicyPerformanceTest.java    |    2 +-
 .../IgniteCacheClientNearCacheExpiryTest.java   |    4 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |   10 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |    6 +-
 .../expiry/IgniteCacheLargeValueExpireTest.java |    4 +-
 ...eCacheOnlyOneTtlCleanupThreadExistsTest.java |    5 +-
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |    8 +-
 .../IgniteCacheLoadAllAbstractTest.java         |    2 +-
 .../IgniteCacheStoreSessionAbstractTest.java    |    5 +-
 ...acheStoreSessionWriteBehindAbstractTest.java |    2 +-
 .../IgniteCacheTxStoreSessionTest.java          |   22 +-
 ...dCacheAtomicLocalTckMetricsSelfTestImpl.java |    8 +-
 .../GridCacheLocalByteArrayValuesSelfTest.java  |    2 +-
 .../local/GridCacheLocalFullApiSelfTest.java    |    4 +-
 .../cache/local/GridCacheLocalLockSelfTest.java |    6 +-
 .../GridCacheLocalMultithreadedSelfTest.java    |    2 +-
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |    2 +-
 .../BinaryTxCacheLocalEntriesSelfTest.java      |    2 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |    4 +-
 .../continuous/CacheContinuousBatchAckTest.java |    2 +-
 ...eContinuousQueryAsyncFilterListenerTest.java |    2 +-
 ...acheContinuousQueryExecuteInPrimaryTest.java |    2 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |   85 +-
 ...ontinuousQueryOperationFromCallbackTest.java |    2 +-
 .../CacheContinuousQueryOperationP2PTest.java   |    2 +-
 .../CacheContinuousQueryOrderingEventTest.java  |    2 +-
 ...acheContinuousQueryRandomOperationsTest.java |    2 +-
 .../CacheKeepBinaryIterationTest.java           |    2 +-
 .../ClientReconnectContinuousQueryTest.java     |    4 +-
 ...yRemoteFilterMissingInClassPathSelfTest.java |    2 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   68 +-
 ...dCacheContinuousQueryNodesFilteringTest.java |    2 +-
 ...dCacheContinuousQueryReplicatedSelfTest.java |    8 +-
 ...eContinuousQueryReplicatedTxOneNodeTest.java |    4 +-
 ...CacheContinuousQueryClientReconnectTest.java |    6 +-
 .../IgniteCacheContinuousQueryClientTest.java   |   18 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |   10 +-
 ...teCacheContinuousQueryNoUnsubscribeTest.java |   14 +-
 ...IgniteCacheContinuousQueryReconnectTest.java |    8 +-
 ...BehindStorePartitionedMultiNodeSelfTest.java |    6 +-
 .../IgniteCacheWriteBehindNoUpdateSelfTest.java |    2 +-
 ...CacheClientWriteBehindStoreAbstractTest.java |    2 +-
 ...ClientWriteBehindStoreNonCoalescingTest.java |    2 +-
 ...DeadlockDetectionMessageMarshallingTest.java |  116 ++
 .../TxDeadlockDetectionUnmasrhalErrorsTest.java |  225 ++
 ...simisticDeadlockDetectionCrossCacheTest.java |    3 +-
 .../CacheVersionedEntryAbstractTest.java        |   10 +-
 .../database/IgniteDbAbstractTest.java          |    2 +-
 .../database/IgniteDbDynamicCacheSelfTest.java  |    4 +-
 .../database/IgniteDbPutGetAbstractTest.java    |   64 +-
 .../DataStreamProcessorSelfTest.java            |   58 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |   10 +-
 .../DataStreamerMultiThreadedSelfTest.java      |    2 +-
 .../DataStreamerUpdateAfterLoadTest.java        |    5 +-
 .../IgniteDataStreamerPerformanceTest.java      |    2 +-
 ...lockMessageSystemPoolStarvationSelfTest.java |    4 +-
 .../processors/igfs/IgfsCacheSelfTest.java      |    3 +-
 .../igfs/IgfsDataManagerSelfTest.java           |    3 +-
 .../processors/igfs/IgfsIgniteMock.java         |    7 -
 .../igfs/IgfsMetaManagerSelfTest.java           |    3 +-
 .../processors/igfs/IgfsOneClientNodeTest.java  |    7 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |    3 +-
 .../processors/igfs/IgfsStartCacheTest.java     |    4 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |    3 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |    4 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |    4 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |    4 +
 .../query/GridQueryCommandHandlerTest.java      |    2 +-
 .../service/GridServiceClientNodeTest.java      |    7 +-
 .../GridServiceProcessorAbstractSelfTest.java   |    2 +-
 .../ServicePredicateAccessCacheTest.java        |    2 +-
 .../IgniteOffheapReadWriteLockSelfTest.java     |   18 +-
 .../cache/GridCacheAbstractLoadTest.java        |    5 +-
 .../cache/GridCacheDataStructuresLoadTest.java  |    2 +-
 .../loadtests/cache/GridCacheLoadTest.java      |    4 +-
 .../capacity/GridCapacityLoadTest.java          |    2 +-
 .../capacity/spring-capacity-cache.xml          |    2 +
 .../GridContinuousOperationsLoadTest.java       |    2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    2 +-
 ...ridSingleSplitsNewNodesAbstractLoadTest.java |   11 +-
 ...idSingleSplitsNewNodesMulticastLoadTest.java |    9 +-
 .../loadtests/discovery/GridGcTimeoutTest.java  |    2 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   10 +-
 .../p2p/GridP2PSameClassLoaderSelfTest.java     |    2 +-
 .../platform/PlatformComputeEchoTask.java       |   11 +-
 .../ignite/platform/PlatformSqlQueryTask.java   |    2 +-
 .../CacheCheckpointSpiSecondCacheSelfTest.java  |    7 +-
 .../communication/GridCacheMessageSelfTest.java |    2 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |    6 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   19 +-
 ...gniteClientReconnectMassiveShutdownTest.java |    4 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |  245 ++-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |   79 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   12 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   34 +-
 .../tcp/TcpDiscoverySpiConfigSelfTest.java      |    4 +-
 .../TcpDiscoverySpiFailureTimeoutSelfTest.java  |   51 +-
 .../stream/socket/SocketStreamerSelfTest.java   |    6 +-
 .../ignite/testframework/GridTestUtils.java     |    2 +-
 .../testframework/junits/GridAbstractTest.java  |   15 +-
 .../junits/common/GridCommonAbstractTest.java   |   23 +-
 .../multijvm/IgniteClusterProcessProxy.java     |    9 +-
 .../junits/multijvm/IgniteProcessProxy.java     |    5 -
 .../testframework/test/ParametersTest.java      |    7 +-
 .../TxDeadlockDetectionTestSuite.java           |    4 +
 .../webapp/META-INF/ignite-webapp-config.xml    |    1 -
 .../tests/p2p/CacheDeploymentTestTask1.java     |    2 +-
 .../tests/p2p/CacheDeploymentTestTask3.java     |    2 +-
 .../p2p/GridP2PContinuousDeploymentTask1.java   |    2 +-
 .../CacheNoValueClassOnServerTestClient.java    |    2 +-
 .../sink/flink/FlinkIgniteSinkSelfTest.java     |    4 +-
 .../query/h2/H2IndexingAbstractGeoSelfTest.java |    3 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |    4 +-
 modules/hibernate-4.2/README.txt                |   48 +
 modules/hibernate-4.2/licenses/apache-2.0.txt   |  202 ++
 modules/hibernate-4.2/pom.xml                   |  159 ++
 .../HibernateAbstractRegionAccessStrategy.java  |  102 +
 .../hibernate/HibernateCollectionRegion.java    |  100 +
 .../cache/hibernate/HibernateEntityRegion.java  |  112 +
 .../hibernate/HibernateGeneralDataRegion.java   |   76 +
 .../cache/hibernate/HibernateKeyWrapper.java    |   73 +
 .../hibernate/HibernateNaturalIdRegion.java     |  103 +
 .../hibernate/HibernateQueryResultsRegion.java  |   70 +
 .../ignite/cache/hibernate/HibernateRegion.java |   99 +
 .../cache/hibernate/HibernateRegionFactory.java |  179 ++
 .../hibernate/HibernateTimestampsRegion.java    |   39 +
 .../HibernateTransactionalDataRegion.java       |   84 +
 .../ignite/cache/hibernate/package-info.java    |   24 +
 .../hibernate/CacheHibernateBlobStore.java      |  542 +++++
 .../CacheHibernateBlobStoreEntry.hbm.xml        |   31 +
 .../hibernate/CacheHibernateBlobStoreEntry.java |   89 +
 .../CacheHibernateBlobStoreFactory.java         |  235 +++
 .../CacheHibernateStoreSessionListener.java     |  222 ++
 .../cache/store/hibernate/package-info.java     |   22 +
 .../src/test/config/factory-cache.xml           |   59 +
 .../src/test/config/factory-cache1.xml          |   61 +
 .../config/factory-incorrect-store-cache.xml    |   56 +
 .../HibernateL2CacheConfigurationSelfTest.java  |  409 ++++
 .../hibernate/HibernateL2CacheMultiJvmTest.java |  440 ++++
 .../hibernate/HibernateL2CacheSelfTest.java     | 1954 +++++++++++++++++
 .../HibernateL2CacheTransactionalSelfTest.java  |  154 ++
 ...nateL2CacheTransactionalUseSyncSelfTest.java |   31 +
 .../CacheHibernateBlobStoreNodeRestartTest.java |   46 +
 .../CacheHibernateBlobStoreSelfTest.java        |  113 +
 .../CacheHibernateStoreFactorySelfTest.java     |  288 +++
 ...heHibernateStoreSessionListenerSelfTest.java |  238 +++
 .../cache/store/hibernate/hibernate.cfg.xml     |   42 +
 .../cache/store/hibernate/package-info.java     |   22 +
 .../IgniteBinaryHibernateTestSuite.java         |   37 +
 .../testsuites/IgniteHibernateTestSuite.java    |   57 +
 modules/hibernate-5.1/README.txt                |   48 +
 modules/hibernate-5.1/licenses/apache-2.0.txt   |  202 ++
 modules/hibernate-5.1/pom.xml                   |  159 ++
 .../HibernateAbstractRegionAccessStrategy.java  |  103 +
 .../hibernate/HibernateCollectionRegion.java    |  114 +
 .../cache/hibernate/HibernateEntityRegion.java  |  128 ++
 .../hibernate/HibernateGeneralDataRegion.java   |   79 +
 .../cache/hibernate/HibernateKeyWrapper.java    |  109 +
 .../hibernate/HibernateNaturalIdRegion.java     |  113 +
 .../hibernate/HibernateQueryResultsRegion.java  |   70 +
 .../ignite/cache/hibernate/HibernateRegion.java |   99 +
 .../cache/hibernate/HibernateRegionFactory.java |  168 ++
 .../hibernate/HibernateTimestampsRegion.java    |   39 +
 .../HibernateTransactionalDataRegion.java       |   84 +
 .../ignite/cache/hibernate/package-info.java    |   24 +
 .../hibernate/CacheHibernateBlobStore.java      |  543 +++++
 .../CacheHibernateBlobStoreEntry.hbm.xml        |   31 +
 .../hibernate/CacheHibernateBlobStoreEntry.java |   89 +
 .../CacheHibernateBlobStoreFactory.java         |  235 +++
 .../CacheHibernateStoreSessionListener.java     |  224 ++
 .../cache/store/hibernate/package-info.java     |   22 +
 .../src/test/config/factory-cache.xml           |   59 +
 .../src/test/config/factory-cache1.xml          |   61 +
 .../config/factory-incorrect-store-cache.xml    |   56 +
 .../HibernateL2CacheConfigurationSelfTest.java  |  407 ++++
 .../hibernate/HibernateL2CacheMultiJvmTest.java |  429 ++++
 .../hibernate/HibernateL2CacheSelfTest.java     | 1960 ++++++++++++++++++
 .../HibernateL2CacheTransactionalSelfTest.java  |  154 ++
 ...nateL2CacheTransactionalUseSyncSelfTest.java |   31 +
 .../CacheHibernateBlobStoreNodeRestartTest.java |   46 +
 .../CacheHibernateBlobStoreSelfTest.java        |  114 +
 .../CacheHibernateStoreFactorySelfTest.java     |  256 +++
 ...heHibernateStoreSessionListenerSelfTest.java |  242 +++
 .../cache/store/hibernate/hibernate.cfg.xml     |   42 +
 .../cache/store/hibernate/package-info.java     |   22 +
 .../IgniteBinaryHibernate5TestSuite.java        |   37 +
 .../testsuites/IgniteHibernate5TestSuite.java   |   57 +
 modules/hibernate-core/pom.xml                  |   76 +
 .../HibernateAccessStrategyAdapter.java         |  340 +++
 .../HibernateAccessStrategyFactory.java         |  235 +++
 .../cache/hibernate/HibernateCacheProxy.java    |  801 +++++++
 .../hibernate/HibernateExceptionConverter.java  |   29 +
 .../hibernate/HibernateKeyTransformer.java      |   29 +
 .../HibernateNonStrictAccessStrategy.java       |  230 ++
 .../HibernateReadOnlyAccessStrategy.java        |  105 +
 .../HibernateReadWriteAccessStrategy.java       |  326 +++
 .../HibernateTransactionalAccessStrategy.java   |  141 ++
 .../ignite/cache/hibernate/package-info.java    |   24 +
 modules/hibernate/README.txt                    |   48 -
 modules/hibernate/licenses/apache-2.0.txt       |  202 --
 modules/hibernate/pom.xml                       |  146 --
 .../HibernateAbstractRegionAccessStrategy.java  |   98 -
 .../HibernateAccessStrategyAdapter.java         |  379 ----
 .../cache/hibernate/HibernateCacheProxy.java    |  801 -------
 .../hibernate/HibernateCollectionRegion.java    |  100 -
 .../cache/hibernate/HibernateEntityRegion.java  |  112 -
 .../hibernate/HibernateGeneralDataRegion.java   |   71 -
 .../hibernate/HibernateKeyTransformer.java      |   28 -
 .../cache/hibernate/HibernateKeyWrapper.java    |   72 -
 .../hibernate/HibernateNaturalIdRegion.java     |  100 -
 .../HibernateNonStrictAccessStrategy.java       |  222 --
 .../hibernate/HibernateQueryResultsRegion.java  |   70 -
 .../HibernateReadOnlyAccessStrategy.java        |  107 -
 .../HibernateReadWriteAccessStrategy.java       |  328 ---
 .../ignite/cache/hibernate/HibernateRegion.java |   99 -
 .../cache/hibernate/HibernateRegionFactory.java |  266 ---
 .../hibernate/HibernateTimestampsRegion.java    |   39 -
 .../HibernateTransactionalAccessStrategy.java   |  141 --
 .../HibernateTransactionalDataRegion.java       |  107 -
 .../ignite/cache/hibernate/package-info.java    |   24 -
 .../hibernate/CacheHibernateBlobStore.java      |  542 -----
 .../CacheHibernateBlobStoreEntry.hbm.xml        |   31 -
 .../hibernate/CacheHibernateBlobStoreEntry.java |   89 -
 .../CacheHibernateBlobStoreFactory.java         |  235 ---
 .../CacheHibernateStoreSessionListener.java     |  222 --
 .../cache/store/hibernate/package-info.java     |   22 -
 .../hibernate/src/test/config/factory-cache.xml |   59 -
 .../src/test/config/factory-cache1.xml          |   61 -
 .../config/factory-incorrect-store-cache.xml    |   56 -
 .../HibernateL2CacheConfigurationSelfTest.java  |  408 ----
 .../hibernate/HibernateL2CacheSelfTest.java     | 1949 -----------------
 .../HibernateL2CacheTransactionalSelfTest.java  |  154 --
 ...nateL2CacheTransactionalUseSyncSelfTest.java |   31 -
 .../CacheHibernateBlobStoreNodeRestartTest.java |   46 -
 .../CacheHibernateBlobStoreSelfTest.java        |  113 -
 .../CacheHibernateStoreFactorySelfTest.java     |  285 ---
 ...heHibernateStoreSessionListenerSelfTest.java |  238 ---
 .../cache/store/hibernate/hibernate.cfg.xml     |   42 -
 .../cache/store/hibernate/package-info.java     |   22 -
 .../IgniteBinaryHibernateTestSuite.java         |   37 -
 .../testsuites/IgniteHibernateTestSuite.java    |   57 -
 modules/hibernate5/README.txt                   |   48 -
 modules/hibernate5/licenses/apache-2.0.txt      |  202 --
 modules/hibernate5/pom.xml                      |  146 --
 .../HibernateAbstractRegionAccessStrategy.java  |   99 -
 .../HibernateAccessStrategyAdapter.java         |  379 ----
 .../cache/hibernate/HibernateCacheProxy.java    |  801 -------
 .../hibernate/HibernateCollectionRegion.java    |  114 -
 .../cache/hibernate/HibernateEntityRegion.java  |  129 --
 .../hibernate/HibernateGeneralDataRegion.java   |   72 -
 .../hibernate/HibernateKeyTransformer.java      |   28 -
 .../cache/hibernate/HibernateKeyWrapper.java    |  108 -
 .../hibernate/HibernateNaturalIdRegion.java     |  113 -
 .../HibernateNonStrictAccessStrategy.java       |  222 --
 .../hibernate/HibernateQueryResultsRegion.java  |   70 -
 .../HibernateReadOnlyAccessStrategy.java        |  107 -
 .../HibernateReadWriteAccessStrategy.java       |  328 ---
 .../ignite/cache/hibernate/HibernateRegion.java |   99 -
 .../cache/hibernate/HibernateRegionFactory.java |  255 ---
 .../hibernate/HibernateTimestampsRegion.java    |   39 -
 .../HibernateTransactionalAccessStrategy.java   |  141 --
 .../HibernateTransactionalDataRegion.java       |  107 -
 .../ignite/cache/hibernate/package-info.java    |   24 -
 .../hibernate/CacheHibernateBlobStore.java      |  542 -----
 .../CacheHibernateBlobStoreEntry.hbm.xml        |   31 -
 .../hibernate/CacheHibernateBlobStoreEntry.java |   89 -
 .../CacheHibernateBlobStoreFactory.java         |  235 ---
 .../CacheHibernateStoreSessionListener.java     |  223 --
 .../cache/store/hibernate/package-info.java     |   22 -
 .../src/test/config/factory-cache.xml           |   59 -
 .../src/test/config/factory-cache1.xml          |   61 -
 .../config/factory-incorrect-store-cache.xml    |   56 -
 .../HibernateL2CacheConfigurationSelfTest.java  |  409 ----
 .../hibernate/HibernateL2CacheSelfTest.java     | 1948 -----------------
 .../HibernateL2CacheTransactionalSelfTest.java  |  154 --
 ...nateL2CacheTransactionalUseSyncSelfTest.java |   31 -
 .../CacheHibernateBlobStoreNodeRestartTest.java |   46 -
 .../CacheHibernateBlobStoreSelfTest.java        |  113 -
 .../CacheHibernateStoreFactorySelfTest.java     |  326 ---
 ...heHibernateStoreSessionListenerSelfTest.java |  241 ---
 .../cache/store/hibernate/hibernate.cfg.xml     |   42 -
 .../cache/store/hibernate/package-info.java     |   22 -
 .../IgniteBinaryHibernate5TestSuite.java        |   37 -
 .../testsuites/IgniteHibernate5TestSuite.java   |   57 -
 .../query/h2/DmlStatementsProcessor.java        |   17 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  223 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |   32 +-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |   78 +-
 .../query/h2/opt/GridH2CollocationModel.java    |    4 +-
 .../query/h2/opt/GridH2IndexBase.java           |   15 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |    6 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |    6 +-
 .../query/h2/opt/GridH2ProxyIndex.java          |  204 ++
 .../query/h2/opt/GridH2ProxySpatialIndex.java   |   70 +
 .../query/h2/opt/GridH2RowDescriptor.java       |   67 +
 .../processors/query/h2/opt/GridH2Table.java    |  114 +-
 .../query/h2/opt/GridLuceneIndex.java           |    4 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |   36 +-
 .../cache/BinarySerializationQuerySelfTest.java |    5 +-
 .../CacheBinaryKeyConcurrentQueryTest.java      |    2 +-
 .../cache/CacheIndexStreamerTest.java           |    4 +-
 .../CacheOffheapBatchIndexingBaseTest.java      |    2 +-
 .../CacheOperationsWithExpirationTest.java      |    2 +-
 .../cache/CacheQueryBuildValueTest.java         |    4 +-
 .../cache/CacheQueryEvictDataLostTest.java      |    2 +-
 .../cache/CacheQueryFilterExpiredTest.java      |    2 +-
 .../CacheRandomOperationsMultithreadedTest.java |    2 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java |   12 +-
 .../cache/CacheSqlQueryValueCopySelfTest.java   |   18 +-
 .../cache/GridCacheOffHeapSelfTest.java         |    4 +-
 .../GridCacheOffheapIndexEntryEvictTest.java    |    4 +-
 .../cache/GridCacheQuerySimpleBenchmark.java    |    2 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    2 +-
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  |    9 +-
 .../IgniteBinaryObjectQueryArgumentsTest.java   |    8 +-
 ...eBinaryWrappedObjectFieldsQuerySelfTest.java |    3 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   17 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |    2 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |    2 +-
 .../IgniteCacheCollocatedQuerySelfTest.java     |    6 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |   14 +-
 .../IgniteCacheCrossCacheJoinRandomTest.java    |    2 +-
 .../IgniteCacheDeleteSqlQuerySelfTest.java      |    6 +-
 ...acheDistributedJoinCollocatedAndNotTest.java |    2 +-
 ...acheDistributedJoinCustomAffinityMapper.java |    2 +-
 .../IgniteCacheDistributedJoinNoIndexTest.java  |    2 +-
 ...ributedJoinPartitionedAndReplicatedTest.java |    2 +-
 ...CacheDistributedJoinQueryConditionsTest.java |    2 +-
 .../cache/IgniteCacheDistributedJoinTest.java   |    6 +-
 .../IgniteCacheFieldsQueryNoDataSelfTest.java   |    2 +-
 ...teCacheFullTextQueryNodeJoiningSelfTest.java |    6 +-
 ...PartitionedAndReplicatedCollocationTest.java |    2 +-
 ...teCacheJoinPartitionedAndReplicatedTest.java |    2 +-
 ...IgniteCacheJoinQueryWithAffinityKeyTest.java |    2 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |    4 +-
 ...eLockPartitionOnAffinityRunAbstractTest.java |    4 +
 .../IgniteCacheMultipleIndexedTypesTest.java    |    2 +-
 .../IgniteCacheObjectKeyIndexingSelfTest.java   |    6 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |    2 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  |    4 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |    4 +-
 .../cache/IgniteCacheQueriesLoadTest1.java      |    2 +-
 .../IgniteCacheQueryH2IndexingLeakTest.java     |    4 +-
 .../cache/IgniteCacheQueryIndexSelfTest.java    |    4 +-
 .../cache/IgniteCacheQueryLoadSelfTest.java     |   14 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   15 +-
 ...gniteCacheSqlQueryMultiThreadedSelfTest.java |    6 +-
 .../IgniteCacheStarvationOnRebalanceTest.java   |    2 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |    8 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |   10 +-
 .../cache/IgniteCrossCachesJoinsQueryTest.java  |    2 +-
 .../cache/QueryEntityCaseMismatchTest.java      |    2 +-
 .../cache/SqlFieldsQuerySelfTest.java           |    2 +-
 ...niteCacheDistributedQueryCancelSelfTest.java |    6 +-
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |    6 +-
 .../IgniteCachePartitionedQuerySelfTest.java    |    2 +-
 .../IgniteCacheQueryNoRebalanceSelfTest.java    |    4 +-
 .../near/IgniteCacheQueryNodeFailTest.java      |    4 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |    2 +-
 .../cache/index/AbstractSchemaSelfTest.java     |    3 +
 .../DynamicIndexAbstractConcurrentSelfTest.java |    5 +-
 .../index/DynamicIndexAbstractSelfTest.java     |    3 +
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java |    6 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    6 +-
 .../query/IgniteQueryDedicatedPoolTest.java     |    2 +-
 .../query/IgniteSqlDistributedJoinSelfTest.java |    2 +-
 .../query/IgniteSqlKeyValueFieldsTest.java      |  392 ++++
 .../query/IgniteSqlSchemaIndexingTest.java      |    2 +-
 .../query/IgniteSqlSplitterSelfTest.java        |   10 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   14 +-
 .../query/h2/IgniteSqlQueryMinMaxTest.java      |    4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |    9 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    2 +
 .../stream/jms11/IgniteJmsStreamerTest.java     |   28 +-
 ...CacheJtaConfigurationValidationSelfTest.java |    2 +-
 ...CacheJtaFactoryConfigValidationSelfTest.java |    2 +-
 ...titionedCacheJtaLookupClassNameSelfTest.java |    2 +-
 .../kafka/KafkaIgniteStreamerSelfTest.java      |    8 +-
 .../stream/mqtt/IgniteMqttStreamerTest.java     |   12 +-
 .../osgi-karaf/src/main/resources/features.xml  |    2 +-
 modules/osgi/README.txt                         |   30 -
 .../core-test/config/cache-query-default.xml    |    6 +
 .../cpp/core-test/src/cache_query_test.cpp      |   82 +
 .../cpp/odbc-test/config/queries-default.xml    |    5 +
 .../cpp/odbc-test/include/complex_type.h        |   25 +
 .../cpp/odbc-test/src/queries_test.cpp          |  148 ++
 .../Binary/BinaryCompactFooterInteropTest.cs    |    2 +-
 .../Binary/BinaryDynamicRegistrationTest.cs     |   18 +-
 .../Binary/JavaBinaryInteropTest.cs             |    6 +-
 .../BinaryConfigurationTest.cs                  |    2 +-
 .../Cache/Affinity/AffinityFieldTest.cs         |    4 +-
 .../Cache/Affinity/AffinityTest.cs              |    4 +-
 .../Cache/CacheConfigurationTest.cs             |   11 +-
 .../Cache/CacheForkedTest.cs                    |    2 +-
 .../Cache/CacheMetricsTest.cs                   |    3 +-
 .../Cache/CacheNearTest.cs                      |   34 +-
 .../Cache/Query/CacheLinqTest.cs                |   30 +-
 .../Cache/Query/CacheQueriesTest.cs             |   52 +
 .../Compute/CancellationTest.cs                 |    4 +-
 .../Compute/ComputeApiTest.cs                   |   30 +-
 .../Config/Compute/compute-grid1.xml            |    1 +
 .../Config/Compute/compute-standalone.xml       |    4 +-
 .../Config/cache-binarizables.xml               |    4 +-
 .../native-client-test-cache-affinity.xml       |    1 +
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   13 +-
 .../IgniteConfigurationSerializerTest.cs        |    8 +-
 .../IgniteConfigurationTest.cs                  |    9 +-
 .../IgniteStartStopTest.cs                      |    2 +-
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   24 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |    4 +-
 .../Cache/Configuration/QueryEntity.cs          |   20 +
 .../Discovery/Tcp/TcpDiscoverySpi.cs            |   43 -
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |    3 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |   21 +
 .../IgniteConfigurationSection.xsd              |   36 +-
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |   12 +
 .../Impl/Compute/ComputeImpl.cs                 |    2 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   14 +
 .../Impl/CacheQueryExpressionVisitor.cs         |   14 +-
 .../Impl/CacheQueryModelVisitor.cs              |   12 +-
 .../Datagrid/MultiTieredCacheExample.cs         |    2 +-
 modules/rocketmq/README.txt                     |   25 +
 modules/rocketmq/pom.xml                        |   81 +
 .../stream/rocketmq/RocketMQStreamer.java       |  151 ++
 .../ignite/stream/rocketmq/package-info.java    |   21 +
 .../stream/rocketmq/RocketMQStreamerTest.java   |  214 ++
 .../rocketmq/RocketMQStreamerTestSuite.java     |   37 +
 .../stream/rocketmq/TestRocketMQServer.java     |  148 ++
 .../ignite/stream/rocketmq/package-info.java    |   21 +
 .../scala/org/apache/ignite/scalar/scalar.scala |   19 +-
 .../scalar/src/test/resources/spring-cache.xml  |    1 +
 .../scalar/tests/ScalarCacheQueriesSpec.scala   |    2 +-
 .../spark/JavaEmbeddedIgniteRDDSelfTest.java    |    2 +-
 .../spark/JavaStandaloneIgniteRDDSelfTest.java  |    2 +-
 .../spring/GridSpringCacheManagerSelfTest.java  |    6 +-
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java |    2 +-
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |    4 +-
 .../internal/IgniteDynamicCacheConfigTest.java  |    2 +-
 .../GridTransformSpringInjectionSelfTest.java   |    2 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |    8 +-
 .../IgniteStartFromStreamConfigurationTest.java |    4 +-
 .../GridSpringTransactionManagerSelfTest.java   |    2 +-
 .../twitter/IgniteTwitterStreamerTest.java      |    8 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |    4 +-
 .../cache/VisorCacheClearCommandSpec.scala      |   24 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |    2 +-
 .../cache/VisorCacheResetCommandSpec.scala      |    8 +-
 .../list-of-registered-users.column-defs.js     |   26 +-
 .../list-of-registered-users.controller.js      |  163 +-
 .../list-of-registered-users.tpl.pug            |   25 +-
 .../frontend/app/primitives/badge/index.scss    |    1 +
 .../frontend/app/primitives/btn/index.scss      |   24 +-
 .../frontend/app/primitives/dropdown/index.pug  |    2 +-
 .../frontend/app/primitives/dropdown/index.scss |   26 +-
 .../frontend/app/primitives/panel/index.scss    |    2 +-
 .../app/primitives/ui-grid-header/index.scss    |   10 +-
 .../app/primitives/ui-grid-header/index.tpl.pug |   10 +-
 .../app/primitives/ui-grid-settings/index.scss  |   58 +-
 .../frontend/app/primitives/ui-grid/index.scss  |  149 +-
 .../frontend/gulpfile.babel.js/paths.js         |    1 +
 .../frontend/gulpfile.babel.js/tasks/bundle.js  |    2 +-
 .../webpack/environments/development.js         |    4 +-
 .../frontend/public/images/icons/cross.svg      |    1 +
 .../frontend/public/images/icons/export.svg     |    1 +
 .../frontend/public/images/icons/gear.svg       |    1 +
 .../stylesheets/_bootstrap-variables.scss       |    4 +-
 modules/web-console/licenses/cc-by-3.0.txt      |  319 +++
 .../webapp2/META-INF/ignite-webapp-config.xml   |    1 -
 .../stream/zeromq/IgniteZeroMqStreamerTest.java |   12 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |    5 +-
 parent/pom.xml                                  |    5 +
 pom.xml                                         |   10 +-
 967 files changed, 23690 insertions(+), 21068 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/ignite/blob/07aa9bbe/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 ecbf475,a01bac6..ea9dfa5
--- 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
@@@ -653,13 -705,11 +652,14 @@@ public class GridCacheProcessor extend
  
      /**
       * @param cfg Cache configuration.
 +     * @param caches Caches map.
 +     * @param templates Templates map.
       * @throws IgniteCheckedException If failed.
       */
 -    private void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
 +    private void registerCache(CacheConfiguration cfg,
 +        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
 +        Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates) throws IgniteCheckedException {
+         assert cfg.getName() != null;
 -
          cloneCheckSerializable(cfg);
  
          CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
@@@ -667,19 -717,18 +667,19 @@@
          // Initialize defaults.
          initialize(cfg, cacheObjCtx);
  
-         boolean template = cfg.getName() != null && cfg.getName().endsWith("*");
 -        String cacheName = cfg.getName();
++        boolean template = cfg.getName().endsWith("*");
  
 -        if (cacheDescriptor(cfg.getName()) != null) {
 -            throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
 -                "assign unique name to each cache): " + cacheName);
 -        }
 +        if (!template) {
 +            if (caches.containsKey(cfg.getName())) {
 +                throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
 +                    "assign unique name to each cache): " + cfg.getName());
 +            }
  
 -        CacheType cacheType;
 +            CacheType cacheType;
  
-             if (CU.isUtilityCache(cfg.getName()))
+         if (CU.isUtilityCache(cfg.getName()))
                  cacheType = CacheType.UTILITY;
-             else if (internalCaches.contains(maskNull(cfg.getName())))
+             else if (internalCaches.contains(cfg.getName()))
                  cacheType = CacheType.INTERNAL;
              else
                  cacheType = CacheType.USER;
@@@ -910,45 -861,45 +912,45 @@@
              ctx.query().onCacheKernalStart();
  
              // Start dynamic caches received from collect discovery data.
 -            for (DynamicCacheDescriptor desc : cacheDescriptors()) {
 -                if (ctx.config().isDaemon())
 -                    continue;
 -
 -                desc.clearRemoteConfigurations();
 -
 -                CacheConfiguration ccfg = desc.cacheConfiguration();
 -
 -                IgnitePredicate filter = ccfg.getNodeFilter();
 -
 -                boolean loc = desc.locallyConfigured();
 -
 -                if (loc || (desc.receivedOnDiscovery() && CU.affinityNode(locNode, filter))) {
 -                    boolean started = desc.onStart();
 -
 -                    assert started : "Failed to change started flag for locally configured cache: " + desc;
 -
 -                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 -
 -                    CachePluginManager pluginMgr = desc.pluginManager();
 -
 -                    GridCacheContext ctx = createCache(
 -                        ccfg, pluginMgr, desc.cacheType(), cacheObjCtx, desc.updatesAllowed());
 -
 -                    ctx.dynamicDeploymentId(desc.deploymentId());
 -
 -                    sharedCtx.addCacheContext(ctx);
 -
 -                    GridCacheAdapter cache = ctx.cache();
 -
 -                    String name = ccfg.getName();
 -
 -                    caches.put(name, cache);
 -
 -                    startCache(cache, desc.schema());
 -
 -                    jCacheProxies.put(name, new IgniteCacheProxy(ctx, cache, null, false));
 -                }
 -            }
 +//            for (DynamicCacheDescriptor desc : cacheDescriptors()) {
 +//                if (ctx.config().isDaemon())
 +//                    continue;
 +//
 +//                desc.clearRemoteConfigurations();
 +//
 +//                CacheConfiguration ccfg = desc.cacheConfiguration();
 +//
 +//                IgnitePredicate filter = ccfg.getNodeFilter();
 +//
 +//                boolean loc = desc.locallyConfigured();
 +//
 +//                if (loc || (desc.receivedOnDiscovery() && CU.affinityNode(locNode, filter))) {
 +//                    boolean started = desc.onStart();
 +//
 +//                    assert started : "Failed to change started flag for locally configured cache: " + desc;
 +//
 +//                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 +//
 +//                    CachePluginManager pluginMgr = desc.pluginManager();
 +//
 +//                    GridCacheContext ctx = createCache(
 +//                        ccfg, pluginMgr, desc.cacheType(), cacheObjCtx, desc.updatesAllowed());
 +//
 +//                    ctx.dynamicDeploymentId(desc.deploymentId());
 +//
 +//                    sharedCtx.addCacheContext(ctx);
 +//
 +//                    GridCacheAdapter cache = ctx.cache();
 +//
 +//                    String name = ccfg.getName();
 +//
- //                    caches.put(maskNull(name), cache);
++//                    caches.put(name, cache);
 +//
 +//                    startCache(cache, desc.schema());
 +//
- //                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
++//                    jCacheProxies.put(name, new IgniteCacheProxy(ctx, cache, null, false));
 +//                }
 +//            }
          }
          finally {
              cacheStartedLatch.countDown();
@@@ -2817,13 -3030,13 +2829,13 @@@
  
                  assert ccfg != null : req;
  
-                 DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(maskNull(req.cacheName()));
 -                DynamicCacheDescriptor desc = registeredTemplates.get(req.cacheName());
++                DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(req.cacheName());
  
                  if (desc == null) {
                      DynamicCacheDescriptor templateDesc = new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), true,
                          req.deploymentId(), req.schema());
  
-                     DynamicCacheDescriptor old = cachesInfo.registeredTemplates().put(maskNull(ccfg.getName()), templateDesc);
 -                    DynamicCacheDescriptor old = registeredTemplates.put(ccfg.getName(), templateDesc);
++                    DynamicCacheDescriptor old = cachesInfo.registeredTemplates().put(ccfg.getName(), templateDesc);
  
                      assert old == null :
                          "Dynamic cache map was concurrently modified [new=" + templateDesc + ", old=" + old + ']';
@@@ -2947,7 -3159,7 +2959,7 @@@
  
                  if (desc != null) {
                      if (req.stop()) {
-                         DynamicCacheDescriptor old = cachesInfo.registeredCaches().remove(maskNull(req.cacheName()));
 -                        DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
++                        DynamicCacheDescriptor old = cachesInfo.registeredCaches().remove(req.cacheName());
  
                          assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
  
@@@ -3437,9 -3661,11 +3446,11 @@@
       * @throws IgniteCheckedException If failed.
       */
      public void addCacheConfiguration(CacheConfiguration cacheCfg) throws IgniteCheckedException {
-         String masked = maskNull(cacheCfg.getName());
+         assert cacheCfg.getName() != null;
+ 
+         String masked = cacheCfg.getName();
  
 -        DynamicCacheDescriptor desc = registeredTemplates.get(masked);
 +        DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(masked);
  
          if (desc != null)
              return;


[22/26] ignite git commit: ignite-5075

Posted by sb...@apache.org.
ignite-5075


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

Branch: refs/heads/ignite-5075
Commit: da62febb4613c9aedac0dd91a9d8e9df06b203d3
Parents: c055276
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 13:35:34 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 15:21:11 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  |   6 +-
 .../internal/managers/discovery/DiscoCache.java |  30 ++--
 .../discovery/GridDiscoveryManager.java         | 140 ++++++++++++-------
 .../affinity/GridAffinityAssignmentCache.java   |  12 +-
 .../cache/CacheAffinitySharedManager.java       |  14 +-
 .../internal/processors/cache/CacheData.java    |  17 ++-
 .../processors/cache/CacheGroupData.java        |  95 +++++++++++++
 .../processors/cache/CacheGroupDescriptor.java  |  86 ++++++++++++
 .../cache/CacheGroupInfrastructure.java         |   9 +-
 .../cache/CacheNodeCommonDiscoveryData.java     |  25 ++++
 .../processors/cache/ClusterCachesInfo.java     | 114 +++++++++++++--
 .../cache/DynamicCacheChangeRequest.java        |   8 --
 .../cache/DynamicCacheDescriptor.java           |  16 ++-
 .../processors/cache/GridCacheContext.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    |  33 ++---
 .../dht/GridDhtAssignmentFetchFuture.java       |  11 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   4 +-
 .../GridDhtPartitionsExchangeFuture.java        |   9 +-
 .../datastructures/DataStructuresProcessor.java |   4 -
 .../service/GridServiceProcessor.java           |   4 -
 20 files changed, 482 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 65570ad..4b34891 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -2196,14 +2196,12 @@ public class IgnitionEx {
         public void initializeDefaultCacheConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException {
             List<CacheConfiguration> cacheCfgs = new ArrayList<>();
 
-            // TODO IGNITE-5075.
-            //cacheCfgs.add(utilitySystemCache());
+            cacheCfgs.add(utilitySystemCache());
 
             if (IgniteComponentType.HADOOP.inClassPath())
                 cacheCfgs.add(CU.hadoopSystemCache());
 
-            // TODO IGNITE-5075.
-            //cacheCfgs.add(atomicsSystemCache(cfg.getAtomicConfiguration()));
+            cacheCfgs.add(atomicsSystemCache(cfg.getAtomicConfiguration()));
 
             CacheConfiguration[] userCaches = cfg.getCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
index 5247ac1..b9c7614 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
@@ -69,7 +69,7 @@ public class DiscoCache {
 
     /** Affinity cache nodes by cache name. */
     @GridToStringInclude
-    private final Map<Integer, List<ClusterNode>> affCacheNodes;
+    private final Map<Integer, List<ClusterNode>> cacheGrpAffNodes;
 
     /** Node map. */
     private final Map<UUID, ClusterNode> nodeMap;
@@ -91,7 +91,7 @@ public class DiscoCache {
      * @param allNodesWithCaches All nodes with at least one cache configured.
      * @param rmtNodesWithCaches Remote nodes with at least one cache configured.
      * @param allCacheNodes Cache nodes by cache name.
-     * @param affCacheNodes Affinity cache nodes by cache name.
+     * @param cacheGrpAffNodes Affinity cache nodes by cache group ID.
      * @param nodeMap Node map.
      * @param nearEnabledCaches Caches where at least one node has near cache enabled.
      * @param alives Alive nodes.
@@ -105,7 +105,7 @@ public class DiscoCache {
         List<ClusterNode> allNodesWithCaches,
         List<ClusterNode> rmtNodesWithCaches,
         Map<Integer, List<ClusterNode>> allCacheNodes,
-        Map<Integer, List<ClusterNode>> affCacheNodes,
+        Map<Integer, List<ClusterNode>> cacheGrpAffNodes,
         Map<UUID, ClusterNode> nodeMap,
         Set<Integer> nearEnabledCaches,
         Set<UUID> alives) {
@@ -118,7 +118,7 @@ public class DiscoCache {
         this.allNodesWithCaches = allNodesWithCaches;
         this.rmtNodesWithCaches = rmtNodesWithCaches;
         this.allCacheNodes = allCacheNodes;
-        this.affCacheNodes = affCacheNodes;
+        this.cacheGrpAffNodes = cacheGrpAffNodes;
         this.nodeMap = nodeMap;
         this.nearEnabledCaches = nearEnabledCaches;
         this.alives.addAll(alives);
@@ -235,25 +235,11 @@ public class DiscoCache {
     }
 
     /**
-     * Gets all nodes that have cache with given ID and should participate in affinity calculation. With
-     * partitioned cache nodes with near-only cache do not participate in affinity node calculation.
-     *
-     * @param cacheName Cache name.
-     * @return Collection of nodes.
-     */
-    public List<ClusterNode> cacheAffinityNodes(@Nullable String cacheName) {
-        return cacheAffinityNodes(CU.cacheId(cacheName));
-    }
-
-    /**
-     * Gets all nodes that have cache with given ID and should participate in affinity calculation. With
-     * partitioned cache nodes with near-only cache do not participate in affinity node calculation.
-     *
-     * @param cacheId Cache ID.
-     * @return Collection of nodes.
+     * @param grpId Cache group ID.
+     * @return All nodes that participate in affinity calculation.
      */
-    public List<ClusterNode> cacheAffinityNodes(int cacheId) {
-        return emptyIfNull(affCacheNodes.get(cacheId));
+    public List<ClusterNode> cacheGroupAffinityNodes(int grpId) {
+        return emptyIfNull(cacheGrpAffNodes.get(grpId));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/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 24c7283..aef01f4 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
@@ -69,11 +69,11 @@ import org.apache.ignite.internal.managers.GridManagerAdapter;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetrics;
 import org.apache.ignite.internal.processors.security.SecurityContext;
-import org.apache.ignite.internal.processors.service.GridServiceProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
@@ -248,12 +248,15 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         new ConcurrentHashMap8<>();
 
     /** Local node initialization event listeners. */
-    private final Collection<IgniteInClosure<ClusterNode>> localNodeInitLsnrs = new ArrayList<>();
+    private final Collection<IgniteInClosure<ClusterNode>> locNodeInitLsnrs = new ArrayList<>();
 
     /** Map of dynamic cache filters. */
     private Map<String, CachePredicate> registeredCaches = new HashMap<>();
 
     /** */
+    private Map<Integer, CacheGroupAffinity> registeredCacheGrps = new HashMap<>();
+
+    /** */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
     /** Received custom messages history. */
@@ -302,24 +305,38 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * @param grpDesc Cache group descriptor.
+     * @param filter Node filter.
+     * @param cacheMode Cache mode.
+     */
+    public void addCacheGroup(CacheGroupDescriptor grpDesc, IgnitePredicate<ClusterNode> filter, CacheMode cacheMode) {
+        CacheGroupAffinity old = registeredCacheGrps.put(grpDesc.groupId(),
+            new CacheGroupAffinity(grpDesc.groupName(), filter, cacheMode));
+
+        assert old == null : old;
+    }
+
+    /**
      * Adds dynamic cache filter.
      *
+     * @param grpId Cache group ID.
      * @param cacheName Cache name.
-     * @param filter Cache filter.
      * @param nearEnabled Near enabled flag.
-     * @param cacheMode Cache mode.
      */
     public void setCacheFilter(
+        int grpId,
         String cacheName,
-        IgnitePredicate<ClusterNode> filter,
-        boolean nearEnabled,
-        CacheMode cacheMode
+        boolean nearEnabled
     ) {
         if (!registeredCaches.containsKey(cacheName)) {
-            if (cacheMode == CacheMode.REPLICATED)
+            CacheGroupAffinity grp = registeredCacheGrps.get(grpId);
+
+            assert grp != null : "Failed to find cache group [grpId=" + grpId + ", cache=" + cacheName + ']';
+
+            if (grp.cacheMode == CacheMode.REPLICATED)
                 nearEnabled = false;
             
-            registeredCaches.put(cacheName, new CachePredicate(filter, nearEnabled, cacheMode));
+            registeredCaches.put(cacheName, new CachePredicate(grp, nearEnabled));
         }
     }
 
@@ -471,7 +488,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             /** {@inheritDoc} */
             @Override public void onLocalNodeInitialized(ClusterNode locNode) {
-                for (IgniteInClosure<ClusterNode> lsnr : localNodeInitLsnrs)
+                for (IgniteInClosure<ClusterNode> lsnr : locNodeInitLsnrs)
                     lsnr.apply(locNode);
             }
 
@@ -779,7 +796,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param lsnr Listener to add.
      */
     public void addLocalNodeInitializedEventListener(IgniteInClosure<ClusterNode> lsnr) {
-        localNodeInitLsnrs.add(lsnr);
+        locNodeInitLsnrs.add(lsnr);
     }
 
     /**
@@ -1688,27 +1705,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
-     * Gets cache nodes for cache with given name that participate in affinity calculation.
-     *
-     * @param cacheName Cache name.
-     * @param topVer Topology version.
-     * @return Collection of cache affinity nodes.
-     */
-    public Collection<ClusterNode> cacheAffinityNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
-        int cacheId = CU.cacheId(cacheName);
-
-        return resolveDiscoCache(cacheId, topVer).cacheAffinityNodes(cacheId);
-    }
-
-    /**
      * Gets cache nodes for cache with given ID that participate in affinity calculation.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param topVer Topology version.
      * @return Collection of cache affinity nodes.
      */
-    public Collection<ClusterNode> cacheAffinityNodes(int cacheId, AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(cacheId, topVer).cacheAffinityNodes(cacheId);
+    public Collection<ClusterNode> cacheGroupAffinityNodes(int grpId, AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(grpId, topVer).cacheGroupAffinityNodes(grpId);
     }
 
     /**
@@ -1771,7 +1775,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             if (!CU.isSystemCache(cacheName) && !CU.isIgfsCache(ctx.config(), cacheName) &&
                 pred != null && pred.cacheNode(node))
-                caches.put(cacheName, pred.cacheMode);
+                caches.put(cacheName, pred.aff.cacheMode);
         }
 
         return caches;
@@ -1791,21 +1795,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      * Gets discovery cache for given topology version.
      *
-     * @param cacheId Cache ID (participates in exception message).
+     * @param grpId Cache group ID (participates in exception message).
      * @param topVer Topology version.
      * @return Discovery cache.
      */
-    private DiscoCache resolveDiscoCache(int cacheId, AffinityTopologyVersion topVer) {
+    private DiscoCache resolveDiscoCache(int grpId, AffinityTopologyVersion topVer) {
         Snapshot snap = topSnap.get();
 
         DiscoCache cache = AffinityTopologyVersion.NONE.equals(topVer) || topVer.equals(snap.topVer) ?
             snap.discoCache : discoCacheHist.get(topVer);
 
         if (cache == null) {
-            DynamicCacheDescriptor desc = ctx.cache().cacheDescriptor(cacheId);
+            CacheGroupAffinity grpAff = registeredCacheGrps.get(grpId);
 
             throw new IgniteException("Failed to resolve nodes topology [" +
-                "cacheName=" + (desc != null ? desc.cacheConfiguration().getName() : "N/A") +
+                "cacheGrp=" + (grpAff != null ? grpAff.grpName : "N/A") +
                 ", topVer=" + topVer +
                 ", history=" + discoCacheHist.keySet() +
                 ", snap=" + snap +
@@ -2019,7 +2023,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             " [rmtNodes=" + rmtNodes + ", loc=" + loc + ']';
 
         Map<Integer, List<ClusterNode>> allCacheNodes = U.newHashMap(allNodes.size());
-        Map<Integer, List<ClusterNode>> affCacheNodes = U.newHashMap(allNodes.size());
+        Map<Integer, List<ClusterNode>> cacheGrpAffNodes = U.newHashMap(allNodes.size());
 
         Set<ClusterNode> allNodesWithCaches = new TreeSet<>(GridNodeOrderComparator.INSTANCE);
         Set<ClusterNode> rmtNodesWithCaches = new TreeSet<>(GridNodeOrderComparator.INSTANCE);
@@ -2031,6 +2035,20 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             assert node.order() != 0 : "Invalid node order [locNode=" + loc + ", node=" + node + ']';
             assert !node.isDaemon();
 
+            for (Map.Entry<Integer, CacheGroupAffinity> e : registeredCacheGrps.entrySet()) {
+                CacheGroupAffinity grpAff = e.getValue();
+                Integer grpId = e.getKey();
+
+                if (grpAff.cacheFilter.apply(node)) {
+                    List<ClusterNode> nodes = cacheGrpAffNodes.get(grpId);
+
+                    if (nodes == null)
+                        cacheGrpAffNodes.put(grpId, nodes = new ArrayList<>());
+
+                    nodes.add(node);
+                }
+            }
+
             for (Map.Entry<String, CachePredicate> entry : registeredCaches.entrySet()) {
                 String cacheName = entry.getKey();
                 CachePredicate filter = entry.getValue();
@@ -2046,9 +2064,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     addToMap(allCacheNodes, cacheName, node);
 
-                    if (filter.dataNode(node))
-                        addToMap(affCacheNodes, cacheName, node);
-
                     if (filter.nearNode(node))
                         nearEnabledCaches.add(CU.cacheId(cacheName));
                 }
@@ -2065,7 +2080,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             U.sealList(allNodesWithCaches),
             U.sealList(rmtNodesWithCaches),
             Collections.unmodifiableMap(allCacheNodes),
-            Collections.unmodifiableMap(affCacheNodes),
+            Collections.unmodifiableMap(cacheGrpAffNodes),
             Collections.unmodifiableMap(nodeMap),
             Collections.unmodifiableSet(nearEnabledCaches),
             alives);
@@ -2604,9 +2619,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** Discovery topology future. */
     private static class DiscoTopologyFuture extends GridFutureAdapter<Long> implements GridLocalEventListener {
         /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
         private GridKernalContext ctx;
 
         /** Topology await version. */
@@ -2681,32 +2693,54 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * TODO IGNTIE-5075: also store list of started caches.
+     */
+    private static class CacheGroupAffinity {
+        /** */
+        private final String grpName;
+
+        /** Cache filter. */
+        private final IgnitePredicate<ClusterNode> cacheFilter;
+
+        /** Cache mode. */
+        private final CacheMode cacheMode;
+
+        /**
+         * @param grpName Group name.
+         * @param cacheFilter Node filter.
+         * @param cacheMode Cache mode.
+         */
+        CacheGroupAffinity(String grpName,
+            IgnitePredicate<ClusterNode> cacheFilter,
+            CacheMode cacheMode) {
+            this.grpName = grpName;
+            this.cacheFilter = cacheFilter;
+            this.cacheMode = cacheMode;
+        }
+    }
+
+    /**
      * Cache predicate.
      */
     private static class CachePredicate {
         /** Cache filter. */
-        private final IgnitePredicate<ClusterNode> cacheFilter;
+        private final CacheGroupAffinity aff;
 
         /** If near cache is enabled on data nodes. */
         private final boolean nearEnabled;
 
-        /** Cache mode. */
-        private final CacheMode cacheMode;
-
         /** Collection of client near nodes. */
         private final ConcurrentHashMap<UUID, Boolean> clientNodes;
 
         /**
-         * @param cacheFilter Cache filter.
+         * @param aff Cache group affinity.
          * @param nearEnabled Near enabled flag.
-         * @param cacheMode Cache mode.
          */
-        private CachePredicate(IgnitePredicate<ClusterNode> cacheFilter, boolean nearEnabled, CacheMode cacheMode) {
-            assert cacheFilter != null;
+        private CachePredicate(CacheGroupAffinity aff, boolean nearEnabled) {
+            assert aff != null;
 
-            this.cacheFilter = cacheFilter;
+            this.aff = aff;
             this.nearEnabled = nearEnabled;
-            this.cacheMode = cacheMode;
 
             clientNodes = new ConcurrentHashMap<>();
         }
@@ -2741,7 +2775,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if this node is a data node for given cache.
          */
         public boolean dataNode(ClusterNode node) {
-            return CU.affinityNode(node, cacheFilter);
+            return CU.affinityNode(node, aff.cacheFilter);
         }
 
         /**
@@ -2749,7 +2783,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if cache is accessible on the given node.
          */
         public boolean cacheNode(ClusterNode node) {
-            return !node.isDaemon() && (CU.affinityNode(node, cacheFilter) || clientNodes.containsKey(node.id()));
+            return !node.isDaemon() && (CU.affinityNode(node, aff.cacheFilter) || clientNodes.containsKey(node.id()));
         }
 
         /**
@@ -2757,7 +2791,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if near cache is present on the given nodes.
          */
         public boolean nearNode(ClusterNode node) {
-            if (CU.affinityNode(node, cacheFilter))
+            if (CU.affinityNode(node, aff.cacheFilter))
                 return nearEnabled;
 
             Boolean near = clientNodes.get(node.id());

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 7867e52..2adab4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -63,7 +63,7 @@ public class GridAffinityAssignmentCache {
     /** Group name. */
     private final String grpName;
 
-    /** */
+    /** Group ID. */
     private final int grpId;
 
     /** Number of backups. */
@@ -162,12 +162,18 @@ public class GridAffinityAssignmentCache {
         return similarAffKey;
     }
 
+    /**
+     * @return Cache group name.
+     */
     public String groupName() {
         return grpName;
     }
 
+    /**
+     * @return Cache group ID.
+     */
     public int groupId() {
-        return 0;
+        return grpId;
     }
 
     /**
@@ -265,7 +271,7 @@ public class GridAffinityAssignmentCache {
         List<ClusterNode> sorted;
 
         if (!locCache) {
-            sorted = new ArrayList<>(discoCache.cacheAffinityNodes(cacheId()));
+            sorted = new ArrayList<>(discoCache.cacheGroupAffinityNodes(groupId()));
 
             Collections.sort(sorted, GridNodeOrderComparator.INSTANCE);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 532e3ea..8cc3a5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -82,7 +82,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     private AffinityTopologyVersion lastAffVer;
 
     /** Registered caches (updated from exchange thread). */
-    private final Map<Integer, DynamicCacheDescriptor> registeredCaches = new HashMap<>();
+    private final Map<Integer, CacheGroupDescriptor> registeredCacheGrps = new HashMap<>();
 
     /** */
     private WaitRebalanceInfo waitInfo;
@@ -127,14 +127,14 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
         if (type == EVT_NODE_JOINED && node.isLocal()) {
             // Clean-up in case of client reconnect.
-            registeredCaches.clear();
+            registeredCacheGrps.clear();
 
             affCalcVer = null;
 
             lastAffVer = null;
 
-            for (DynamicCacheDescriptor desc : cctx.cache().cacheDescriptors())
-                registeredCaches.put(desc.cacheId(), desc);
+            for (CacheGroupDescriptor desc : cctx.cache().cacheGroupDescriptors())
+                registeredCacheGrps.put(desc.groupId(), desc);
         }
 
         if (!CU.clientNode(node) && (type == EVT_NODE_FAILED || type == EVT_NODE_JOINED || type == EVT_NODE_LEFT)) {
@@ -382,6 +382,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         });
 
         for (ExchangeActions.ActionData action : exchActions.newAndClientCachesStartRequests()) {
+            DynamicCacheDescriptor desc = action.descriptor();
+
             DynamicCacheChangeRequest req = action.request();
 
             Integer cacheId = CU.cacheId(req.cacheName());
@@ -404,7 +406,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 cctx.cache().prepareCacheStart(req, nearCfg, action.descriptor(), fut.topologyVersion());
 
             if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
-                if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
+                if (fut.discoCache().cacheGroupAffinityNodes(desc.groupDescriptor().groupId()).isEmpty())
                     U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
             }
 
@@ -862,7 +864,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         }
         else {
             GridDhtAssignmentFetchFuture fetchFut = new GridDhtAssignmentFetchFuture(cctx,
-                aff.cacheName(),
+                aff.groupId(),
                 fut.topologyVersion(),
                 fut.discoCache());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
index 82afdc7..fcad88a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -36,7 +36,10 @@ public class CacheData implements Serializable {
     private final CacheConfiguration cacheCfg;
 
     /** */
-    private final Integer cacheId;
+    private final int cacheId;
+
+    /** */
+    private final int grpId;
 
     /** */
     private final CacheType cacheType;
@@ -76,6 +79,7 @@ public class CacheData implements Serializable {
      */
     CacheData(CacheConfiguration cacheCfg,
         int cacheId,
+        int grpId,
         CacheType cacheType,
         AffinityTopologyVersion startTopVer,
         IgniteUuid deploymentId,
@@ -89,9 +93,11 @@ public class CacheData implements Serializable {
         assert startTopVer != null : cacheCfg.getName();
         assert deploymentId != null : cacheCfg.getName();
         assert template || cacheId != 0 : cacheCfg.getName();
+        assert template || grpId != 0 : cacheCfg.getName();
 
         this.cacheCfg = cacheCfg;
         this.cacheId = cacheId;
+        this.grpId = grpId;
         this.cacheType = cacheType;
         this.startTopVer = startTopVer;
         this.deploymentId = deploymentId;
@@ -103,9 +109,16 @@ public class CacheData implements Serializable {
     }
 
     /**
+     * @return Cache group ID.
+     */
+    public int groupId() {
+        return grpId;
+    }
+
+    /**
      * @return Cache ID.
      */
-    public Integer cacheId() {
+    public int cacheId() {
         return cacheId;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
new file mode 100644
index 0000000..4a6edda
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.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 java.io.Serializable;
+import java.util.Set;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class CacheGroupData implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final String grpName;
+
+    /** */
+    private final int grpId;
+
+    /** */
+    private final CacheConfiguration cacheCfg;
+
+    /** */
+    private final AffinityTopologyVersion startTopVer;
+
+    /** */
+    @GridToStringInclude
+    private final Set<String> cacheNames;
+
+    /**
+     * @param cacheCfg Cache configuration.
+     * @param grpId
+     * @param startTopVer
+     */
+    public CacheGroupData(CacheConfiguration cacheCfg,
+        String grpName,
+        int grpId,
+        AffinityTopologyVersion startTopVer,
+        Set<String> cacheNames) {
+        assert cacheCfg != null;
+        assert grpName != null;
+        assert grpId != 0;
+        assert startTopVer != null;
+
+        this.cacheCfg = cacheCfg;
+        this.grpName = grpName;
+        this.grpId = grpId;
+        this.startTopVer = startTopVer;
+        this.cacheNames = cacheNames;
+    }
+
+    public String groupName() {
+        return grpName;
+    }
+
+    public int groupId() {
+        return grpId;
+    }
+
+    public CacheConfiguration config() {
+        return cacheCfg;
+    }
+
+    public AffinityTopologyVersion startTopologyVersion() {
+        return startTopVer;
+    }
+
+    Set<String> cacheNames() {
+        return cacheNames;
+    }
+
+    @Override public String toString() {
+        return S.toString(CacheGroupData.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
new file mode 100644
index 0000000..7b0f8fe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupDescriptor.java
@@ -0,0 +1,86 @@
+/*
+ * 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 java.util.Set;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class CacheGroupDescriptor {
+    /** */
+    private final String grpName;
+
+    /** */
+    private final int grpId;
+
+    /** */
+    private final CacheConfiguration cacheCfg;
+
+    /** */
+    private final AffinityTopologyVersion startTopVer;
+
+    /** */
+    @GridToStringInclude
+    private final Set<String> cacheNames;
+
+    CacheGroupDescriptor(String grpName,
+        int grpId,
+        CacheConfiguration cacheCfg,
+        AffinityTopologyVersion startTopVer,
+        Set<String> cacheNames) {
+        assert cacheCfg != null;
+        assert grpName != null;
+        assert grpId != 0;
+        assert startTopVer != null;
+
+        this.grpName = grpName;
+        this.grpId = grpId;
+        this.cacheCfg = cacheCfg;
+        this.startTopVer = startTopVer;
+        this.cacheNames = cacheNames;
+    }
+
+    public String groupName() {
+        return grpName;
+    }
+
+    public int groupId() {
+        return grpId;
+    }
+
+    public CacheConfiguration config() {
+        return cacheCfg;
+    }
+
+    public AffinityTopologyVersion startTopologyVersion() {
+        return startTopVer;
+    }
+
+    Set<String> cacheNames() {
+        return cacheNames;
+    }
+
+    @Override public String toString() {
+        return S.toString(CacheGroupDescriptor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupInfrastructure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupInfrastructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupInfrastructure.java
index 4a0b3de..0769884 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupInfrastructure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupInfrastructure.java
@@ -17,10 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.HashSet;
-import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
@@ -28,7 +25,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
 
 /**
@@ -47,13 +44,15 @@ public class CacheGroupInfrastructure {
     /** */
     private final GridCacheSharedContext ctx;
 
+    /** */
     private GridDhtPartitionTopology top;
 
     /**
+     * @param id Group ID.
      * @param ctx Context.
      * @param ccfg Cache configuration.
      */
-    public CacheGroupInfrastructure(int id, GridCacheSharedContext ctx, CacheConfiguration ccfg) {
+    CacheGroupInfrastructure(int id, GridCacheSharedContext ctx, CacheConfiguration ccfg) {
         assert id != 0 : "Invalid group ID [cache=" + ccfg.getName() + ", grpName=" + ccfg.getGroupName() + ']';
         assert ccfg != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
index 84a33dc..55fb087 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
@@ -31,6 +31,9 @@ class CacheNodeCommonDiscoveryData implements Serializable {
     private static final long serialVersionUID = 0L;
 
     /** */
+    private final int cacheGrpIdGen;
+
+    /** */
     @GridToStringInclude
     private final Map<String, CacheData> caches;
 
@@ -39,6 +42,10 @@ class CacheNodeCommonDiscoveryData implements Serializable {
     private final Map<String, CacheData> templates;
 
     /** */
+    @GridToStringInclude
+    private final Map<String, CacheGroupData> cacheGrps;
+
+    /** */
     private final Map<String, Map<UUID, Boolean>> clientNodesMap;
 
     /**
@@ -48,12 +55,30 @@ class CacheNodeCommonDiscoveryData implements Serializable {
      */
     CacheNodeCommonDiscoveryData(Map<String, CacheData> caches,
         Map<String, CacheData> templates,
+        Map<String, CacheGroupData> cacheGrps,
+        int cacheGrpIdGen,
         Map<String, Map<UUID, Boolean>> clientNodesMap) {
+        assert caches != null;
+        assert templates != null;
+        assert cacheGrps != null;
+        assert cacheGrpIdGen > 0 : cacheGrpIdGen;
+        assert clientNodesMap != null;
+
         this.caches = caches;
         this.templates = templates;
+        this.cacheGrpIdGen = cacheGrpIdGen;
+        this.cacheGrps = cacheGrps;
         this.clientNodesMap = clientNodesMap;
     }
 
+    int currentCacheGroupId() {
+        return cacheGrpIdGen;
+    }
+
+    Map<String, CacheGroupData> cacheGroups() {
+        return cacheGrps;
+    }
+
     /**
      * @return Started caches.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index e75f93c..88e15ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -59,6 +59,12 @@ class ClusterCachesInfo {
     /** Dynamic caches. */
     private final ConcurrentMap<String, DynamicCacheDescriptor> registeredCaches = new ConcurrentHashMap<>();
 
+    /** */
+    private final ConcurrentMap<String, CacheGroupDescriptor> registeredCacheGrps = new ConcurrentHashMap<>();
+
+    /** */
+    private int cacheGrpIdGen = 1;
+
     /** Cache templates. */
     private final ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates = new ConcurrentHashMap<>();
 
@@ -118,6 +124,7 @@ class ClusterCachesInfo {
                     DynamicCacheDescriptor templateDesc = new DynamicCacheDescriptor(ctx,
                         ccfg,
                         req.cacheType(),
+                        null,
                         true,
                         req.deploymentId(),
                         req.schema());
@@ -154,9 +161,12 @@ class ClusterCachesInfo {
                         assert req.cacheType() != null : req;
                         assert F.eq(ccfg.getName(), req.cacheName()) : req;
 
+                        CacheGroupDescriptor grpDesc = registerCacheGroup(ccfg, topVer.nextMinorVersion());
+
                         DynamicCacheDescriptor startDesc = new DynamicCacheDescriptor(ctx,
                             ccfg,
                             req.cacheType(),
+                            grpDesc,
                             false,
                             req.deploymentId(),
                             req.schema());
@@ -168,10 +178,9 @@ class ClusterCachesInfo {
                         assert old == null;
 
                         ctx.discovery().setCacheFilter(
+                            grpDesc.groupId(),
                             ccfg.getName(),
-                            ccfg.getNodeFilter(),
-                            ccfg.getNearConfiguration() != null,
-                            ccfg.getCacheMode());
+                            ccfg.getNearConfiguration() != null);
 
                         ctx.discovery().addClientNode(req.cacheName(),
                             req.initiatingNodeId(),
@@ -411,7 +420,7 @@ class ClusterCachesInfo {
                     assert joinDiscoData != null;
                 }
 
-                processJoiningNode(joinDiscoData, node.id());
+                processJoiningNode(joinDiscoData, node.id(), topVer);
 
                 assert locJoinStartCaches == null;
 
@@ -436,7 +445,7 @@ class ClusterCachesInfo {
                 CacheJoinNodeDiscoveryData discoData = joiningNodesDiscoData.remove(node.id());
 
                 if (discoData != null)
-                    processJoiningNode(discoData, node.id());
+                    processJoiningNode(discoData, node.id(), topVer);
             }
 
             initStartVersionOnJoin(registeredCaches.values(), node, topVer);
@@ -476,6 +485,7 @@ class ClusterCachesInfo {
         for (DynamicCacheDescriptor desc : registeredCaches.values()) {
             CacheData cacheData = new CacheData(desc.cacheConfiguration(),
                 desc.cacheId(),
+                desc.groupDescriptor().groupId(),
                 desc.cacheType(),
                 desc.startTopologyVersion(),
                 desc.deploymentId(),
@@ -488,11 +498,23 @@ class ClusterCachesInfo {
             caches.put(desc.cacheName(), cacheData);
         }
 
+        Map<String, CacheGroupData> cacheGrps = new HashMap<>();
+
+        for (CacheGroupDescriptor grpDesc : registeredCacheGrps.values()) {
+            CacheGroupData grpData = new CacheGroupData(grpDesc.config(),
+                grpDesc.groupName(),
+                grpDesc.groupId(),
+                grpDesc.startTopologyVersion());
+
+            cacheGrps.put(grpDesc.groupName(), grpData);
+        }
+
         Map<String, CacheData> templates = new HashMap<>();
 
         for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
             CacheData cacheData = new CacheData(desc.cacheConfiguration(),
                 0,
+                0,
                 desc.cacheType(),
                 desc.startTopologyVersion(),
                 null,
@@ -505,7 +527,11 @@ class ClusterCachesInfo {
             templates.put(desc.cacheName(), cacheData);
         }
 
-        return new CacheNodeCommonDiscoveryData(caches, templates, ctx.discovery().clientNodesMap());
+        return new CacheNodeCommonDiscoveryData(caches,
+            templates,
+            cacheGrps,
+            cacheGrpIdGen,
+            ctx.discovery().clientNodesMap());
     }
 
     /**
@@ -520,11 +546,29 @@ class ClusterCachesInfo {
 
         CacheNodeCommonDiscoveryData cachesData = (CacheNodeCommonDiscoveryData)data.commonData();
 
+        cacheGrpIdGen = cachesData.currentCacheGroupId();
+
+        for (CacheGroupData grpData : cachesData.cacheGroups().values()) {
+            CacheGroupDescriptor grpDesc = new CacheGroupDescriptor(grpData.groupName(),
+                grpData.groupId(),
+                grpData.config(),
+                grpData.startTopologyVersion());
+
+            CacheGroupDescriptor old = registeredCacheGrps.put(grpDesc.groupName(), grpDesc);
+
+            assert old == null : old;
+
+            ctx.discovery().addCacheGroup(grpDesc,
+                grpData.config().getNodeFilter(),
+                grpData.config().getCacheMode());
+        }
+
         for (CacheData cacheData : cachesData.templates().values()) {
             DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
                 ctx,
                 cacheData.cacheConfiguration(),
                 cacheData.cacheType(),
+                null,
                 true,
                 cacheData.deploymentId(),
                 cacheData.schema());
@@ -539,12 +583,17 @@ class ClusterCachesInfo {
         }
 
         for (CacheData cacheData : cachesData.caches().values()) {
+            CacheGroupDescriptor grpDesc = groupDescriptor(cacheData.groupId());
+
+            assert grpDesc != null : cacheData.cacheConfiguration().getName();
+
             CacheConfiguration cfg = cacheData.cacheConfiguration();
 
             DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
                 ctx,
                 cacheData.cacheConfiguration(),
                 cacheData.cacheType(),
+                grpDesc,
                 false,
                 cacheData.deploymentId(),
                 cacheData.schema());
@@ -558,10 +607,9 @@ class ClusterCachesInfo {
             assert old == null;
 
             ctx.discovery().setCacheFilter(
+                grpDesc.groupId(),
                 cfg.getName(),
-                cfg.getNodeFilter(),
-                cfg.getNearConfiguration() != null,
-                cfg.getCacheMode());
+                cfg.getNearConfiguration() != null);
         }
 
         if (!F.isEmpty(cachesData.clientNodesMap())) {
@@ -576,6 +624,15 @@ class ClusterCachesInfo {
         gridData = cachesData;
     }
 
+    private CacheGroupDescriptor groupDescriptor(int grpId) {
+        for (CacheGroupDescriptor desc : registeredCacheGrps.values()) {
+            if (desc.groupId() == grpId)
+                return desc;
+        }
+
+        return null;
+    }
+
     void onJoiningNodeDataReceived(DiscoveryDataBag.JoiningNodeDiscoveryData data) {
         if (data.hasJoiningNodeData()) {
             Serializable joiningNodeData = data.joiningNodeData();
@@ -610,7 +667,7 @@ class ClusterCachesInfo {
         }
     }
 
-    private void processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId) {
+    private void processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId, AffinityTopologyVersion topVer) {
         for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.templates().values()) {
             CacheConfiguration cfg = cacheInfo.config();
 
@@ -618,6 +675,7 @@ class ClusterCachesInfo {
                 DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
                     cfg,
                     cacheInfo.cacheType(),
+                    null,
                     true,
                     joinData.cacheDeploymentId(),
                     new QuerySchema(cfg.getQueryEntities()));
@@ -635,9 +693,12 @@ class ClusterCachesInfo {
             CacheConfiguration cfg = cacheInfo.config();
 
             if (!registeredCaches.containsKey(cfg.getName())) {
+                CacheGroupDescriptor grpDesc = registerCacheGroup(cfg, topVer);
+
                 DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
                     cfg,
                     cacheInfo.cacheType(),
+                    grpDesc,
                     false,
                     joinData.cacheDeploymentId(),
                     new QuerySchema(cfg.getQueryEntities()));
@@ -650,16 +711,43 @@ class ClusterCachesInfo {
                 assert old == null : old;
 
                 ctx.discovery().setCacheFilter(
+                    grpDesc.groupId(),
                     cfg.getName(),
-                    cfg.getNodeFilter(),
-                    cfg.getNearConfiguration() != null,
-                    cfg.getCacheMode());
+                    cfg.getNearConfiguration() != null);
             }
 
             ctx.discovery().addClientNode(cfg.getName(), nodeId, cfg.getNearConfiguration() != null);
         }
     }
 
+    private CacheGroupDescriptor registerCacheGroup(CacheConfiguration cfg, AffinityTopologyVersion topVer) {
+        if (cfg.getGroupName() != null) {
+            CacheGroupDescriptor desc = registeredCacheGrps.get(cfg.getGroupName());
+
+            if (desc != null)
+                return desc;
+        }
+
+        int grpId = cacheGrpIdGen++;
+
+        CacheGroupDescriptor grpDesc = new CacheGroupDescriptor(
+            cfg.getGroupName() != null ? cfg.getGroupName() : cfg.getName(),
+            grpId,
+            cfg,
+            topVer);
+
+        ctx.discovery().addCacheGroup(grpDesc, cfg.getNodeFilter(), cfg.getCacheMode());
+
+        return grpDesc;
+    }
+
+    /**
+     * @return Registered cache groups.
+     */
+    ConcurrentMap<String, CacheGroupDescriptor> registeredCacheGroups() {
+        return registeredCacheGrps;
+    }
+
     /**
      * @return Registered caches.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/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 47d0d6e..8fa763c 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
@@ -402,14 +402,6 @@ public class DynamicCacheChangeRequest implements Serializable {
         this.schema = schema != null ? schema.copy() : null;
     }
 
-    public DynamicCacheDescriptor cacheDescriptor() {
-        return cacheDesc;
-    }
-
-    public void cacheDescriptor(DynamicCacheDescriptor cacheDesc) {
-        this.cacheDesc = cacheDesc;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DynamicCacheChangeRequest.class, this, "cacheName", cacheName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/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 f1562d9..ee0c522 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
@@ -99,7 +99,7 @@ public class DynamicCacheDescriptor {
     private QuerySchema schema;
 
     /** */
-    private int grpId;
+    private final CacheGroupDescriptor grpDesc;
 
     /**
      * @param ctx Context.
@@ -112,16 +112,17 @@ public class DynamicCacheDescriptor {
     public DynamicCacheDescriptor(GridKernalContext ctx,
         CacheConfiguration cacheCfg,
         CacheType cacheType,
-        int grpId,
+        CacheGroupDescriptor grpDesc,
         boolean template,
         IgniteUuid deploymentId,
         QuerySchema schema) {
         assert cacheCfg != null;
+        assert grpDesc != null || template;
         assert schema != null;
 
         this.cacheCfg = cacheCfg;
         this.cacheType = cacheType;
-        this.grpId = grpId;
+        this.grpDesc = grpDesc;
         this.template = template;
         this.deploymentId = deploymentId;
 
@@ -137,8 +138,13 @@ public class DynamicCacheDescriptor {
         }
     }
 
-    public int groupId() {
-        return grpId;
+    /**
+     * @return Cache group ID.
+     */
+    public CacheGroupDescriptor groupDescriptor() {
+        assert grpDesc != null : this;
+
+        return grpDesc;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 5652189..bfd28cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -329,7 +329,6 @@ public class GridCacheContext<K, V> implements Externalizable {
         assert locStartTopVer != null : cacheCfg.getName();
 
         assert grp != null;
-
         assert evtMgr != null;
         assert storeMgr != null;
         assert evictMgr != null;
@@ -394,6 +393,9 @@ public class GridCacheContext<K, V> implements Externalizable {
         itHolder = new CacheWeakQueryIteratorsHolder(log);
     }
 
+    /**
+     * @return Cache group infrastructure.
+     */
     public CacheGroupInfrastructure group() {
         return grp;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/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 a007170..4c95a24 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
@@ -678,11 +678,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             CacheType cacheType;
 
             if (CU.isUtilityCache(cfg.getName()))
-            cacheType = CacheType.UTILITY;
-        else if (internalCaches.contains(cfg.getName()))
-            cacheType = CacheType.INTERNAL;
-        else
-            cacheType = CacheType.USER;
+                cacheType = CacheType.UTILITY;
+            else if (internalCaches.contains(cfg.getName()))
+                cacheType = CacheType.INTERNAL;
+            else
+                cacheType = CacheType.USER;
 
             if (cacheType != CacheType.USER && cfg.getMemoryPolicyName() == null)
                 cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
@@ -769,25 +769,21 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         internalCaches.add(CU.ATOMICS_CACHE_NAME);
     }
 
+    /**
+     * @param grpId Group ID.
+     * @return
+     */
     @Nullable public CacheGroupInfrastructure cacheGroup(int grpId) {
         return cacheGrps.get(grpId);
     }
 
+    /**
+     * @return Cache groups.
+     */
     public Collection<CacheGroupInfrastructure> cacheGroups() {
         return cacheGrps.values();
     }
 
-    private int nextCacheGroupId(CacheConfiguration ccfg) {
-        if (ccfg.getGroupName() != null) {
-            for (DynamicCacheDescriptor cacheDesc : registeredCaches.values()) {
-                if (ccfg.getGroupName().equals(cacheDesc.cacheConfiguration().getGroupName()))
-                    return cacheDesc.groupId();
-            }
-        }
-
-        return registeredCaches.size() + 1;
-    }
-
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart(boolean activeOnStart) throws IgniteCheckedException {
@@ -1348,6 +1344,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cfg Cache configuration to use to create cache.
+     * @param grp Cache group infrastructure.
      * @param pluginMgr Cache plugin manager.
      * @param cacheType Cache type.
      * @param cacheObjCtx Cache object context.
@@ -3107,6 +3104,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         return cachesInfo.registeredCaches().values();
     }
 
+    public Collection<CacheGroupDescriptor> cacheGroupDescriptors() {
+        return cachesInfo.registeredCacheGroups().values();
+    }
+
     /**
      * @param cacheId Cache ID.
      * @return Cache descriptor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
index 4f94ae2..7c63c50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
@@ -48,9 +48,6 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFF
  * Future that fetches affinity assignment from remote cache nodes.
  */
 public class GridDhtAssignmentFetchFuture extends GridFutureAdapter<GridDhtAffinityAssignmentResponse> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
@@ -73,21 +70,19 @@ public class GridDhtAssignmentFetchFuture extends GridFutureAdapter<GridDhtAffin
 
     /**
      * @param ctx Context.
-     * @param cacheName Cache name.
      * @param topVer Topology version.
      * @param discoCache Discovery cache.
      */
     public GridDhtAssignmentFetchFuture(
         GridCacheSharedContext ctx,
-        String cacheName,
+        int grpId,
         AffinityTopologyVersion topVer,
         DiscoCache discoCache
     ) {
         this.ctx = ctx;
-        int cacheId = CU.cacheId(cacheName);
-        this.key = new T2<>(cacheId, topVer);
+        this.key = new T2<>(grpId, topVer);
 
-        Collection<ClusterNode> availableNodes = discoCache.cacheAffinityNodes(cacheId);
+        Collection<ClusterNode> availableNodes = discoCache.cacheGroupAffinityNodes(grpId);
 
         LinkedList<ClusterNode> tmp = new LinkedList<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 5b3dfc6..6d45d6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -991,7 +991,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         AffinityTopologyVersion topVer,
         GridDhtPartitionState state,
         GridDhtPartitionState... states) {
-        Collection<UUID> allIds = topVer.topologyVersion() > 0 ? F.nodeIds(discoCache.cacheAffinityNodes(cctx.cacheId())) : null;
+        Collection<UUID> allIds = topVer.topologyVersion() > 0 ?
+            F.nodeIds(discoCache.cacheGroupAffinityNodes(cctx.group().groupId())) :
+            null;
 
         lock.readLock().lock();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/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 bcfd8e0..2fbb042 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
@@ -937,17 +937,18 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     private void warnNoAffinityNodes() {
         List<String> cachesWithoutNodes = null;
 
-        for (String name : cctx.cache().cacheNames()) {
-            if (discoCache.cacheAffinityNodes(name).isEmpty()) {
+        for (DynamicCacheDescriptor cacheDesc : cctx.cache().cacheDescriptors()) {
+            if (cacheDesc.startTopologyVersion().compareTo(topologyVersion()) <= 0 &&
+                discoCache.cacheGroupAffinityNodes(cacheDesc.groupDescriptor().groupId()).isEmpty()) {
                 if (cachesWithoutNodes == null)
                     cachesWithoutNodes = new ArrayList<>();
 
-                cachesWithoutNodes.add(name);
+                cachesWithoutNodes.add(cacheDesc.cacheName());
 
                 // Fire event even if there is no client cache started.
                 if (cctx.gridEvents().isRecordable(EventType.EVT_CACHE_NODES_LEFT)) {
                     Event evt = new CacheEvent(
-                        name,
+                        cacheDesc.cacheName(),
                         cctx.localNode(),
                         cctx.localNode(),
                         "All server nodes have left the cluster.",

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 326429f..eb0981b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -224,10 +224,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
      *
      */
     private void onKernalStart0(boolean activeOnStart){
-        // TODO IGNITE-5075.
-        if (true)
-            return;
-
         if (!activeOnStart && ctx.state().active())
             ctx.event().addLocalEventListener(lsnr, EVT_NODE_LEFT, EVT_NODE_FAILED);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da62febb/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 5ee347e..0c8f857 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -211,10 +211,6 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart(boolean activeOnStart) throws IgniteCheckedException {
-        // TODO IGNITE-5075.
-        if (true)
-            return;
-
         if (ctx.isDaemon() || !ctx.state().active())
             return;
 


[19/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-5075-cacheStart' into ignite-5075

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-5075-cacheStart' into ignite-5075

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java


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

Branch: refs/heads/ignite-5075
Commit: c132480ea069f1bef4f63779a341b4a7f2991bcd
Parents: 91452b1 bf2360d
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 13:18:07 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 13:18:07 2017 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |   36 +
 examples/rest/http-rest-example.php             |    4 +-
 .../discovery/GridDiscoveryManager.java         |   11 +-
 .../affinity/AffinityTopologyVersion.java       |    9 +
 .../cache/CacheAffinitySharedManager.java       |  210 +--
 .../CacheClientReconnectDiscoveryData.java      |  128 ++
 .../internal/processors/cache/CacheData.java    |  169 +++
 .../cache/CacheJoinNodeDiscoveryData.java       |  128 ++
 .../cache/CacheNodeCommonDiscoveryData.java     |   79 ++
 .../processors/cache/ClusterCachesInfo.java     |  730 ++++++++++
 .../cache/DynamicCacheChangeBatch.java          |   83 +-
 .../cache/DynamicCacheChangeRequest.java        |   95 +-
 .../cache/DynamicCacheDescriptor.java           |   26 +-
 .../processors/cache/ExchangeActions.java       |  332 +++++
 .../processors/cache/GridCacheContext.java      |   27 +-
 .../processors/cache/GridCacheIoManager.java    |    9 +-
 .../GridCachePartitionExchangeManager.java      |   58 +-
 .../processors/cache/GridCacheProcessor.java    | 1243 +++++-------------
 .../dht/GridDhtPartitionTopologyImpl.java       |   10 +-
 .../GridDhtPartitionsExchangeFuture.java        |  128 +-
 .../cache/query/GridCacheQueryManager.java      |    3 +-
 .../cluster/GridClusterStateProcessor.java      |   21 +-
 .../ignite/spi/discovery/DiscoveryDataBag.java  |   18 +-
 .../processors/cache/IgniteCacheStartTest.java  |  183 +++
 .../cache/IgniteDynamicCacheStartSelfTest.java  |    2 +-
 .../CacheLateAffinityAssignmentTest.java        |    4 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    3 +
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 modules/hibernate-core/pom.xml                  |    8 +
 pom.xml                                         |    2 +-
 30 files changed, 2442 insertions(+), 1319 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c132480e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index b221e6f,48bc6da..532e3ea
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@@ -356,11 -349,28 +351,28 @@@ public class CacheAffinitySharedManager
                  assert old == null : old;
              }
          }
+     }
  
-         boolean clientOnly = true;
+     /**
+      * Called on exchange initiated for cache start/stop request.
+      *
+      * @param fut Exchange future.
+      * @param crd Coordinator flag.
+      * @param exchActions Cache change requests.
+      * @throws IgniteCheckedException If failed.
+      * @return {@code True} if client-only exchange is needed.
+      */
+     public boolean onCacheChangeRequest(final GridDhtPartitionsExchangeFuture fut,
+         boolean crd,
+         ExchangeActions exchActions)
+         throws IgniteCheckedException
+     {
+         assert exchActions != null && !exchActions.empty() : exchActions;
+ 
+         updateCachesInfo(exchActions);
  
          // Affinity did not change for existing caches.
 -        forAllCaches(crd && lateAffAssign, new IgniteInClosureX<GridAffinityAssignmentCache>() {
 +        forAllCacheGroups(crd && lateAffAssign, new IgniteInClosureX<GridAffinityAssignmentCache>() {
              @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
                  if (fut.stopping(aff.cacheId()))
                      return;
@@@ -377,79 -384,75 +386,75 @@@
  
              Integer cacheId = CU.cacheId(req.cacheName());
  
-             if (req.start()) {
-                 cctx.cache().prepareCacheStart(req, fut.topologyVersion());
+             boolean startCache;
  
-                 if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
-                     if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
-                         U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
-                 }
+             NearCacheConfiguration nearCfg = null;
+ 
+             if (cctx.localNodeId().equals(req.initiatingNodeId())) {
+                 startCache = true;
+ 
+                 nearCfg = req.nearCacheConfiguration();
+             }
+             else {
+                 startCache = cctx.cacheContext(action.descriptor().cacheId()) == null &&
+                     CU.affinityNode(cctx.localNode(), req.startCacheConfiguration().getNodeFilter());
+             }
  
-                 if (!crd || !lateAffAssign) {
-                     GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+             if (startCache)
+                 cctx.cache().prepareCacheStart(req, nearCfg, action.descriptor(), fut.topologyVersion());
  
-                     if (cacheCtx != null && !cacheCtx.isLocal()) {
-                         boolean clientCacheStarted =
-                             req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId());
+             if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
+                 if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
+                     U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
+             }
  
-                         if (clientCacheStarted)
-                             initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign);
-                         else if (!req.clientStartOnly()) {
-                             assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion());
+             if (!crd || !lateAffAssign) {
+                 GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
  
-                             GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache();
+                 if (cacheCtx != null && !cacheCtx.isLocal()) {
+                     boolean clientCacheStarted =
+                         req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId());
  
-                             assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion();
+                     if (clientCacheStarted)
+                         initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign);
+                     else if (!req.clientStartOnly()) {
+                         assert fut.topologyVersion().equals(cacheCtx.cacheStartTopologyVersion());
  
-                             List<List<ClusterNode>> assignment = aff.calculate(fut.topologyVersion(),
-                                 fut.discoveryEvent(), fut.discoCache());
+                         GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache();
  
-                             aff.initialize(fut.topologyVersion(), assignment);
-                         }
+                         assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion();
+ 
+                         List<List<ClusterNode>> assignment = aff.calculate(fut.topologyVersion(),
+                             fut.discoveryEvent(), fut.discoCache());
+ 
+                         aff.initialize(fut.topologyVersion(), assignment);
                      }
                  }
-                 else
-                     initStartedCacheOnCoordinator(fut, req.cacheDescriptor());
              }
-             else if (req.stop() || req.close()) {
-                 cctx.cache().blockGateway(req);
+             else
 -                initStartedCacheOnCoordinator(fut, cacheId);
++                initStartedCacheOnCoordinator(fut, req.cacheDescriptor());
+         }
  
-                 if (crd) {
-                     boolean rmvCache = false;
+         for (DynamicCacheChangeRequest req : exchActions.closeRequests(cctx.localNodeId())) {
+             Integer cacheId = CU.cacheId(req.cacheName());
  
-                     if (req.close() && req.initiatingNodeId().equals(cctx.localNodeId())) {
-                         GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+             cctx.cache().blockGateway(req);
  
-                         rmvCache = cacheCtx != null && !cacheCtx.affinityNode();
-                     }
-                     else if (req.stop())
-                         rmvCache = true;
- 
-                     if (rmvCache) {
-                         // TODO IGNITE-5075.
- //                        CacheHolder cache = caches.remove(cacheId);
- //
- //                        if (cache != null) {
- //                            if (!req.stop()) {
- //                                assert !cache.client();
- //
- //                                cache = CacheHolder2.create(cctx,
- //                                    cctx.cache().cacheDescriptor(cacheId),
- //                                    fut,
- //                                    cache.affinity());
- //
- //                                caches.put(cacheId, cache);
- //                            }
- //                            else {
- //                                if (stoppedCaches == null)
- //                                    stoppedCaches = new HashSet<>();
- //
- //                                stoppedCaches.add(cache.cacheId());
- //
- //                                cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class);
- //                            }
- //                        }
-                     }
+             if (crd) {
+                 GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+ 
+                 // Client cache was stopped, need create 'client' CacheHolder.
+                 if (cacheCtx != null && !cacheCtx.affinityNode()) {
+                     CacheHolder cache = caches.remove(cacheId);
+ 
+                     assert !cache.client() : cache;
+ 
+                     cache = CacheHolder2.create(cctx,
+                         cctx.cache().cacheDescriptor(cacheId),
+                         fut,
+                         cache.affinity());
+ 
+                     caches.put(cacheId, cache);
                  }
              }
          }

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/c132480e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index a62badf,6cb8907..5652189
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@@ -291,8 -291,9 +294,10 @@@ public class GridCacheContext<K, V> imp
          GridKernalContext ctx,
          GridCacheSharedContext sharedCtx,
          CacheConfiguration cacheCfg,
 +        CacheGroupInfrastructure grp,
          CacheType cacheType,
+         AffinityTopologyVersion cacheStartTopVer,
+         AffinityTopologyVersion locStartTopVer,
          boolean affNode,
          boolean updatesAllowed,
          MemoryPolicy memPlc,
@@@ -320,9 -321,9 +325,11 @@@
          assert ctx != null;
          assert sharedCtx != null;
          assert cacheCfg != null;
+         assert cacheStartTopVer != null : cacheCfg.getName();
+         assert locStartTopVer != null : cacheCfg.getName();
  
 +        assert grp != null;
 +
          assert evtMgr != null;
          assert storeMgr != null;
          assert evictMgr != null;
@@@ -338,8 -339,9 +345,10 @@@
          this.ctx = ctx;
          this.sharedCtx = sharedCtx;
          this.cacheCfg = cacheCfg;
 +        this.grp = grp;
          this.cacheType = cacheType;
+         this.locStartTopVer = locStartTopVer;
+         this.cacheStartTopVer = cacheStartTopVer;
          this.affNode = affNode;
          this.updatesAllowed = updatesAllowed;
          this.depEnabled = ctx.deploy().enabled() && !cacheObjects().isBinaryEnabled(cacheCfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c132480e/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 7ef8af5,b4e4b14..a007170
--- 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
@@@ -738,51 -664,25 +667,25 @@@ public class GridCacheProcessor extend
          // Initialize defaults.
          initialize(cfg, cacheObjCtx);
  
-         String cacheName = cfg.getName();
+         boolean template = cfg.getName().endsWith("*");
  
-         if (cacheDescriptor(cfg.getName()) != null) {
-             throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
-                 "assign unique name to each cache): " + cacheName);
-         }
+         if (!template) {
+             if (caches.containsKey(cfg.getName())) {
+                 throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
+                     "assign unique name to each cache): " + cfg.getName());
+             }
  
-         CacheType cacheType;
+             CacheType cacheType;
  
-         if (CU.isUtilityCache(cfg.getName()))
+             if (CU.isUtilityCache(cfg.getName()))
 -                cacheType = CacheType.UTILITY;
 -            else if (internalCaches.contains(cfg.getName()))
 -                cacheType = CacheType.INTERNAL;
 -            else
 -                cacheType = CacheType.USER;
 +            cacheType = CacheType.UTILITY;
 +        else if (internalCaches.contains(cfg.getName()))
 +            cacheType = CacheType.INTERNAL;
 +        else
 +            cacheType = CacheType.USER;
  
-         if (cacheType != CacheType.USER && cfg.getMemoryPolicyName() == null)
-             cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
- 
-         boolean template = cfg.getName() != null && cfg.getName().endsWith("*");
- 
-         DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
-             cfg,
-             cacheType,
-             0,
-             template,
-             IgniteUuid.randomUuid(),
-             new QuerySchema(cfg.getQueryEntities()));
- 
-         desc.locallyConfigured(true);
-         desc.staticallyConfigured(true);
-         desc.receivedFrom(ctx.localNodeId());
- 
-         if (!template) {
-             cacheDescriptor(cfg.getName(), desc);
- 
-             ctx.discovery().setCacheFilter(
-                 cfg.getName(),
-                 cfg.getNodeFilter(),
-                 cfg.getNearConfiguration() != null && cfg.getCacheMode() == PARTITIONED,
-                 cfg.getCacheMode());
- 
-             ctx.discovery().addClientNode(cfg.getName(),
-                 ctx.localNodeId(),
-                 cfg.getNearConfiguration() != null);
+             if (cacheType != CacheType.USER && cfg.getMemoryPolicyName() == null)
+                 cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
  
              if (!cacheType.userCache())
                  stopSeq.addLast(cfg.getName());
@@@ -831,6 -766,6 +769,25 @@@
          internalCaches.add(CU.ATOMICS_CACHE_NAME);
      }
  
++    @Nullable public CacheGroupInfrastructure cacheGroup(int grpId) {
++        return cacheGrps.get(grpId);
++    }
++
++    public Collection<CacheGroupInfrastructure> cacheGroups() {
++        return cacheGrps.values();
++    }
++
++    private int nextCacheGroupId(CacheConfiguration ccfg) {
++        if (ccfg.getGroupName() != null) {
++            for (DynamicCacheDescriptor cacheDesc : registeredCaches.values()) {
++                if (ccfg.getGroupName().equals(cacheDesc.cacheConfiguration().getGroupName()))
++                    return cacheDesc.groupId();
++            }
++        }
++
++        return registeredCaches.size() + 1;
++    }
++
      /** {@inheritDoc} */
      @SuppressWarnings("unchecked")
      @Override public void onKernalStart(boolean activeOnStart) throws IgniteCheckedException {
@@@ -1464,9 -1333,10 +1356,11 @@@
       * @throws IgniteCheckedException If failed to create cache.
       */
      private GridCacheContext createCache(CacheConfiguration<?, ?> cfg,
 +        CacheGroupInfrastructure grp,
          @Nullable CachePluginManager pluginMgr,
          CacheType cacheType,
+         AffinityTopologyVersion cacheStartTopVer,
+         AffinityTopologyVersion locStartTopVer,
          CacheObjectContext cacheObjCtx,
          boolean updatesAllowed)
          throws IgniteCheckedException {
@@@ -1540,8 -1410,9 +1434,10 @@@
              ctx,
              sharedCtx,
              cfg,
 +            grp,
              cacheType,
+             cacheStartTopVer,
+             locStartTopVer,
              affNode,
              updatesAllowed,
              memPlc,
@@@ -1672,8 -1543,9 +1568,10 @@@
                  ctx,
                  sharedCtx,
                  cfg,
 +                grp,
                  cacheType,
+                 cacheStartTopVer,
+                 locStartTopVer,
                  affNode,
                  true,
                  memPlc,
@@@ -1913,23 -1798,9 +1824,23 @@@
              }
          }
  
-         return started;
+         return started != null ? started : Collections.<DynamicCacheDescriptor>emptyList();
      }
  
 +    private CacheGroupInfrastructure startCacheGroup(CacheConfiguration cfg0, int grpId) throws IgniteCheckedException {
 +        CacheConfiguration ccfg = new CacheConfiguration(cfg0);
 +
 +        CacheGroupInfrastructure grp = new CacheGroupInfrastructure(grpId, sharedCtx, ccfg);
 +
 +        grp.start();
 +
 +        CacheGroupInfrastructure old = cacheGrps.put(grpId, grp);
 +
 +        assert old == null;
 +
 +        return grp;
 +    }
 +
      /**
       * @param cfg Start configuration.
       * @param nearCfg Near configuration.
@@@ -1945,70 -1815,39 +1855,60 @@@
          CacheConfiguration cfg,
          NearCacheConfiguration nearCfg,
          CacheType cacheType,
 +        int grpId,
-         boolean clientStartOnly,
-         UUID initiatingNodeId,
          IgniteUuid deploymentId,
-         AffinityTopologyVersion topVer,
+         AffinityTopologyVersion cacheStartTopVer,
+         AffinityTopologyVersion exchTopVer,
          @Nullable QuerySchema schema
      ) throws IgniteCheckedException {
-         CacheConfiguration ccfg = new CacheConfiguration(cfg);
+         assert !caches.containsKey(cfg.getName()) : cfg.getName();
  
-         IgnitePredicate nodeFilter = ccfg.getNodeFilter();
++        String grpName = cfg.getGroupName();
 +
-         ClusterNode locNode = ctx.discovery().localNode();
- 
-         boolean affNodeStart = !clientStartOnly && CU.affinityNode(locNode, nodeFilter);
-         boolean clientNodeStart = locNode.id().equals(initiatingNodeId);
- 
-         if (sharedCtx.cacheContext(CU.cacheId(cfg.getName())) != null)
-             return;
++        CacheGroupInfrastructure grp = null;
 +
-         if (affNodeStart || clientNodeStart || CU.isSystemCache(cfg.getName())) {
-             String grpName = cfg.getGroupName();
++        if (grpName != null) {
++            for (CacheGroupInfrastructure grp0 : cacheGrps.values()) {
++                if (grp0.sharedGroup() && grpName.equals(grp0.groupName())) {
++                    grp = grp0;
 +
-             CacheGroupInfrastructure grp = null;
- 
-             if (grpName != null) {
-                 for (CacheGroupInfrastructure grp0 : cacheGrps.values()) {
-                     if (grp0.sharedGroup() && grpName.equals(grp0.groupName())) {
-                         grp = grp0;
- 
-                         break;
-                     }
++                    break;
 +                }
- 
-                 if (grp == null)
-                     grp = startCacheGroup(cfg, grpId);
 +            }
-             else
++
++            if (grp == null)
 +                grp = startCacheGroup(cfg, grpId);
++        }
++        else
++            grp = startCacheGroup(cfg, grpId);
 +
-             if (clientNodeStart && !affNodeStart) {
-                 if (nearCfg != null)
-                     ccfg.setNearConfiguration(nearCfg);
-                 else
-                     ccfg.setNearConfiguration(null);
-             }
+         CacheConfiguration ccfg = new CacheConfiguration(cfg);
  
-             CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+         if (nearCfg != null)
+             ccfg.setNearConfiguration(nearCfg);
  
-             GridCacheContext cacheCtx = createCache(ccfg, grp, null, cacheType, cacheObjCtx, true);
+         CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
  
-             cacheCtx.startTopologyVersion(topVer);
+         GridCacheContext cacheCtx = createCache(ccfg,
++            grp,
+             null,
+             cacheType,
+             cacheStartTopVer,
+             exchTopVer,
+             cacheObjCtx,
+             true);
  
-             cacheCtx.dynamicDeploymentId(deploymentId);
+         cacheCtx.dynamicDeploymentId(deploymentId);
  
-             GridCacheAdapter cache = cacheCtx.cache();
+         GridCacheAdapter cache = cacheCtx.cache();
  
-             sharedCtx.addCacheContext(cacheCtx);
+         sharedCtx.addCacheContext(cacheCtx);
  
-             caches.put(cacheCtx.name(), cache);
+         caches.put(cacheCtx.name(), cache);
  
-             startCache(cache, schema != null ? schema : new QuerySchema());
+         startCache(cache, schema != null ? schema : new QuerySchema());
  
-             onKernalStart(cache);
-         }
+         onKernalStart(cache);
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c132480e/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 57db6c3,6149586..e301993
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@@ -77,8 -78,9 +78,10 @@@ public class GridCacheTestContext<K, V
                  null
              ),
              defaultCacheConfiguration(),
 +            null,
              CacheType.USER,
+             AffinityTopologyVersion.ZERO,
+             AffinityTopologyVersion.ZERO,
              true,
              true,
              null,


[11/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart


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

Branch: refs/heads/ignite-5075
Commit: 54091d2effffa16769259e75c4c5fba3d00e5c3c
Parents: 0c66d0e 0e8e5dd
Author: sboikov <sb...@gridgain.com>
Authored: Fri Apr 28 15:45:38 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Apr 28 15:45:38 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/igfs/IgfsMetrics.java     |  2 +-
 .../managers/communication/GridIoManager.java   |  2 +-
 .../managers/communication/GridIoMessage.java   |  5 +-
 .../GridDhtPartitionsAbstractMessage.java       |  6 ++
 .../visor/query/VisorQueryConfiguration.java    | 12 ----
 .../CacheMemoryPolicyConfigurationTest.java     | 12 +++-
 .../MemoryPolicyInitializationTest.java         | 18 +++--
 .../Cache/Configuration/CacheConfiguration.cs   |  3 -
 .../IgniteConfigurationSection.xsd              |  5 --
 .../commands/cache/VisorCacheCommand.scala      |  1 -
 .../web-console/backend/app/browsersHandler.js  |  4 +-
 modules/web-console/backend/app/mongo.js        | 71 ++++++++++++++++++--
 .../app/modules/agent/AgentManager.service.js   | 20 +++---
 .../modules/configuration/Version.service.js    |  2 +-
 .../generator/ConfigurationGenerator.js         |  5 --
 .../generator/PlatformGenerator.js              |  3 +-
 .../generator/defaults/Cache.service.js         |  2 -
 .../generator/defaults/IGFS.service.js          |  1 -
 .../frontend/app/modules/demo/Demo.module.js    |  4 +-
 .../states/configuration/caches/query.pug       |  6 --
 .../modules/states/configuration/igfs/misc.pug  |  2 -
 .../agent/handlers/DatabaseListener.java        | 24 +++----
 22 files changed, 127 insertions(+), 83 deletions(-)
----------------------------------------------------------------------



[07/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java


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

Branch: refs/heads/ignite-5075
Commit: 82f3a2fc377494ab7f3291e61f7e5a77052b8f40
Parents: ab92ab2
Author: sboikov <sb...@gridgain.com>
Authored: Thu Apr 27 18:13:42 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Apr 27 18:13:42 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheAffinitySharedManager.java  | 14 ++++++++------
 .../internal/processors/cache/GridCacheProcessor.java |  4 ++--
 2 files changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/82f3a2fc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index d3749f4..60eeef0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -435,16 +435,18 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
             cctx.cache().blockGateway(req);
 
-            CacheHolder cache = caches.remove(cacheId);
+            if (crd) {
+                CacheHolder cache = caches.remove(cacheId);
 
-            assert cache != null : req;
+                assert cache != null : req;
 
-            if (stoppedCaches == null)
-                stoppedCaches = new HashSet<>();
+                if (stoppedCaches == null)
+                    stoppedCaches = new HashSet<>();
 
-            stoppedCaches.add(cache.cacheId());
+                stoppedCaches.add(cache.cacheId());
 
-            cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class);
+                cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class);
+            }
         }
 
         if (stoppedCaches != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/82f3a2fc/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 cfc267e..a68d174 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
@@ -2018,8 +2018,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             sharedCtx.removeCacheContext(ctx);
 
-            assert req.deploymentId().equals(ctx.dynamicDeploymentId()) : "Different deployment IDs [req=" + req +
-                ", ctxDepId=" + ctx.dynamicDeploymentId() + ']';
+//            assert req.deploymentId().equals(ctx.dynamicDeploymentId()) : "Different deployment IDs [req=" + req +
+//                ", ctxDepId=" + ctx.dynamicDeploymentId() + ']';
 
             onKernalStop(cache, req.destroy());
 


[04/26] ignite git commit: ignite-5075 cache start

Posted by sb...@apache.org.
ignite-5075 cache start


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

Branch: refs/heads/ignite-5075
Commit: b71d4ef2d030eb7b59d095e87e72fe5505c9792e
Parents: 07aa9bb
Author: sboikov <sb...@gridgain.com>
Authored: Wed Apr 26 21:53:27 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Apr 26 23:58:49 2017 +0300

----------------------------------------------------------------------
 .../affinity/AffinityTopologyVersion.java       |   8 +
 .../CacheClientReconnectDiscoveryData.java      |  89 ++++++
 .../CacheReconnectClientDiscoveryData.java      |  26 --
 .../processors/cache/ClusterCachesInfo.java     | 161 +++++-----
 .../cache/DynamicCacheChangeBatch.java          |  71 +----
 .../cache/DynamicCacheChangeRequest.java        |  56 +++-
 .../cache/DynamicCacheDescriptor.java           |  20 --
 .../processors/cache/ExchangeActions.java       |  90 ++++++
 .../processors/cache/GridCacheProcessor.java    | 319 ++-----------------
 9 files changed, 350 insertions(+), 490 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index f564e28..4011d9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
+
+import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -72,6 +74,12 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
         this.minorTopVer = minorTopVer;
     }
 
+    public AffinityTopologyVersion nextMinorVersion() {
+        assert topVer > 0;
+
+        return new AffinityTopologyVersion(topVer, minorTopVer + 1);
+    }
+
     /**
      * @return Topology version.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
new file mode 100644
index 0000000..34657b4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
@@ -0,0 +1,89 @@
+/*
+ * 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.configuration.CacheConfiguration;
+import org.apache.ignite.lang.IgniteUuid;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ *
+ */
+public class CacheClientReconnectDiscoveryData implements Serializable {
+    /** */
+    private final Map<String, CacheInfo> clientCaches;
+
+    public CacheClientReconnectDiscoveryData(Map<String, CacheInfo> clientCaches) {
+        this.clientCaches = clientCaches;
+    }
+
+    Map<String, CacheInfo> clientCaches() {
+        return clientCaches;
+    }
+
+    static class CacheInfo implements Serializable {
+        /** */
+        private final CacheConfiguration ccfg;
+
+        /** */
+        private final CacheType cacheType;
+
+        /** */
+        private final IgniteUuid deploymentId;
+
+        /** */
+        private final boolean nearCache;
+
+        /** */
+        private final byte flags;
+
+        public CacheInfo(CacheConfiguration ccfg,
+            CacheType cacheType,
+            IgniteUuid deploymentId,
+            boolean nearCache,
+            byte flags) {
+            assert ccfg != null;
+            assert cacheType != null;
+            assert deploymentId != null;
+
+            this.ccfg = ccfg;
+            this.cacheType = cacheType;
+            this.deploymentId = deploymentId;
+            this.nearCache = nearCache;
+            this.flags = flags;
+        }
+
+        CacheConfiguration config() {
+            return ccfg;
+        }
+
+        CacheType cacheType() {
+            return cacheType;
+        }
+
+        IgniteUuid deploymentId() {
+            return deploymentId;
+        }
+
+        boolean nearCache() {
+            return nearCache;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java
deleted file mode 100644
index 10a8f7e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheReconnectClientDiscoveryData.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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 java.io.Serializable;
-
-/**
- *
- */
-public class CacheReconnectClientDiscoveryData implements Serializable {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index bd4ee1f..8824a48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -17,31 +17,30 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheExistsException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 
 /**
@@ -87,6 +86,8 @@ class ClusterCachesInfo {
      * @return {@code True} if minor topology version should be increased.
      */
     boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTopologyVersion topVer) {
+        ExchangeActions exchangeActions = new ExchangeActions();
+
         boolean incMinorTopVer = false;
 
         List<DynamicCacheDescriptor> added = null;
@@ -163,8 +164,13 @@ class ClusterCachesInfo {
                             req.initiatingNodeId(),
                             req.nearCacheConfiguration() != null);
 
+                        if (added == null)
+                            added = new ArrayList<>();
+
                         added.add(startDesc);
 
+                        exchangeActions.addCacheToStart(req, desc);
+
                         needExchange = true;
                     }
                 }
@@ -192,19 +198,13 @@ class ClusterCachesInfo {
                             needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
                                 req.initiatingNodeId(),
                                 req.nearCacheConfiguration() != null);
-
-                            if (needExchange)
-                                req.clientStartOnly(true);
                         }
                     }
 
                     if (needExchange) {
-                        if (newTopVer == null) {
-                            newTopVer = new AffinityTopologyVersion(topVer.topologyVersion(),
-                                topVer.minorTopologyVersion() + 1);
-                        }
+                        desc.clientCacheStartVersion(topVer.nextMinorVersion());
 
-                        desc.clientCacheStartVersion(newTopVer);
+                        exchangeActions.addClientCacheToStart(req, desc);
                     }
                 }
 
@@ -222,94 +222,68 @@ class ClusterCachesInfo {
 
                 if (desc != null) {
                     if (req.stop()) {
-                        DynamicCacheDescriptor old = cachesInfo.registeredCaches().remove(maskNull(req.cacheName()));
+                        DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
 
                         assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
 
                         ctx.discovery().removeCacheFilter(req.cacheName());
 
                         needExchange = true;
+
+                        exchangeActions.addCacheToStop(req);
                     }
                     else {
                         assert req.close() : req;
 
                         needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
+
+                        if (needExchange)
+                            exchangeActions.addCacheToStop(req);
                     }
                 }
             }
 
-            req.exchangeNeeded(needExchange);
-
             incMinorTopVer |= needExchange;
         }
 
         if (added != null) {
-            AffinityTopologyVersion startTopVer = incMinorTopVer ?
-                new AffinityTopologyVersion(topVer.topologyVersion(), topVer.minorTopologyVersion() + 1) : topVer;
+            AffinityTopologyVersion startTopVer = incMinorTopVer ? topVer.nextMinorVersion() : topVer;
 
             for (DynamicCacheDescriptor desc : added)
                 desc.startTopologyVersion(startTopVer);
         }
 
-        return incMinorTopVer;
-    }
+        if (incMinorTopVer) {
+            assert !exchangeActions.empty() : exchangeActions;
 
-    CacheJoinNodeDiscoveryData joinDiscoveryData() {
-        if (cachesOnDisconnect != null) {
-//            Collection<DynamicCacheChangeRequest> reqs;
-//
-//            Map<String, Map<UUID, Boolean>> clientNodesMap;
-//
-//            reqs = new ArrayList<>(caches.size() + 1);
-//
-//            clientNodesMap = U.newHashMap(caches.size());
-//
-//            collectDataOnReconnectingNode(reqs, clientNodesMap, joiningNodeId);
-
-            // TODO
-            return null;
+            batch.exchangeActions(exchangeActions);
         }
-        else {
-            assert ctx.config().isDaemon() || joinDiscoData != null;
 
-            return joinDiscoData;
-        }
+        return incMinorTopVer;
     }
 
-    /**
-     * @param reqs requests.
-     * @param clientNodesMap Client nodes map.
-     * @param nodeId Node id.
-     */
-    private void collectDataOnReconnectingNode(
-        Collection<GridCacheAdapter> caches,
-        Collection<DynamicCacheChangeRequest> reqs,
-        Map<String, Map<UUID, Boolean>> clientNodesMap,
-        UUID nodeId
-    ) {
-        for (GridCacheAdapter<?, ?> cache : caches) {
-            DynamicCacheDescriptor desc = cachesOnDisconnect.get(cache.name());
-
-            if (desc == null)
-                continue;
-
-            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(null, cache.name(), null);
-
-            req.startCacheConfiguration(desc.cacheConfiguration());
-            req.cacheType(desc.cacheType());
-            req.deploymentId(desc.deploymentId());
-            req.receivedFrom(desc.receivedFrom());
-            req.schema(desc.schema());
+    Serializable joinDiscoveryData() {
+        if (cachesOnDisconnect != null) {
+            Map<String, CacheClientReconnectDiscoveryData.CacheInfo> cachesInfo = new HashMap<>();
 
-            reqs.add(req);
+            for (IgniteInternalCache cache : ctx.cache().caches()) {
+                DynamicCacheDescriptor desc = cachesOnDisconnect.get(cache.name());
 
-            Boolean nearEnabled = cache.isNear();
+                assert desc != null : cache.name();
 
-            Map<UUID, Boolean> map = U.newHashMap(1);
+                cachesInfo.put(cache.name(), new CacheClientReconnectDiscoveryData.CacheInfo(desc.cacheConfiguration(),
+                    desc.cacheType(),
+                    desc.deploymentId(),
+                    cache.context().isNear(),
+                    (byte)0));
+            }
 
-            map.put(nodeId, nearEnabled);
+            return new CacheClientReconnectDiscoveryData(cachesInfo);
+        }
+        else {
+            assert ctx.config().isDaemon() || joinDiscoData != null;
 
-            clientNodesMap.put(cache.name(), map);
+            return joinDiscoData;
         }
     }
 
@@ -505,14 +479,31 @@ class ClusterCachesInfo {
         if (data.hasJoiningNodeData()) {
             Serializable joiningNodeData = data.joiningNodeData();
 
-            if (joiningNodeData instanceof CacheReconnectClientDiscoveryData) {
-                // TODO
-            }
+            if (joiningNodeData instanceof CacheClientReconnectDiscoveryData)
+                processClientReconnectData((CacheClientReconnectDiscoveryData)joiningNodeData, data.joiningNodeId());
             else if (joiningNodeData instanceof CacheJoinNodeDiscoveryData)
                 processJoiningNode((CacheJoinNodeDiscoveryData)joiningNodeData, data.joiningNodeId());
         }
     }
 
+    /**
+     * @param clientNodeId Client node ID.
+     */
+    private void processClientReconnectData(CacheClientReconnectDiscoveryData clientData, UUID clientNodeId) {
+        for (CacheClientReconnectDiscoveryData.CacheInfo cacheInfo : clientData.clientCaches().values()) {
+            String cacheName = cacheInfo.config().getName();
+
+            if (surviveReconnect(cacheName))
+                ctx.discovery().addClientNode(cacheName, clientNodeId, false);
+            else {
+                DynamicCacheDescriptor desc = registeredCaches.get(cacheName);
+
+                if (desc != null && desc.deploymentId().equals(cacheInfo.deploymentId()))
+                    ctx.discovery().addClientNode(cacheName, clientNodeId, cacheInfo.nearCache());
+            }
+        }
+    }
+
     private void processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId) {
         for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.templates().values()) {
             CacheConfiguration cfg = cacheInfo.config();
@@ -591,14 +582,12 @@ class ClusterCachesInfo {
         for(Map.Entry<String, DynamicCacheDescriptor> e : cachesOnDisconnect.entrySet()) {
             DynamicCacheDescriptor desc = e.getValue();
 
-            String name = e.getKey();
+            String cacheName = e.getKey();
 
             boolean stopped;
 
-            boolean sysCache = CU.isUtilityCache(name) || CU.isAtomicsCache(name);
-
-            if (!sysCache) {
-                DynamicCacheDescriptor newDesc = registeredCaches.get(name);
+            if (!surviveReconnect(cacheName)) {
+                DynamicCacheDescriptor newDesc = registeredCaches.get(cacheName);
 
                 stopped = newDesc == null || !desc.deploymentId().equals(newDesc.deploymentId());
             }
@@ -606,7 +595,7 @@ class ClusterCachesInfo {
                 stopped = false;
 
             if (stopped)
-                stoppedCaches.add(name);
+                stoppedCaches.add(cacheName);
         }
 
         cachesOnDisconnect = null;
@@ -614,6 +603,10 @@ class ClusterCachesInfo {
         return stoppedCaches;
     }
 
+    private boolean surviveReconnect(String cacheName) {
+        return CU.isUtilityCache(cacheName) || CU.isAtomicsCache(cacheName);
+    }
+
     void clearCaches() {
         registeredCaches.clear();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index a250063..e44bfcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
@@ -33,26 +34,22 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Custom message ID. */
+    private IgniteUuid id = IgniteUuid.randomUuid();
+
     /** Change requests. */
     @GridToStringInclude
     private Collection<DynamicCacheChangeRequest> reqs;
 
-    /** Client nodes map. Used in discovery data exchange. */
-    @GridToStringInclude
-    private Map<String, Map<UUID, Boolean>> clientNodes;
-
-    /** Custom message ID. */
-    private IgniteUuid id = IgniteUuid.randomUuid();
-
     /** */
-    private boolean clientReconnect;
+    private transient ExchangeActions exchangeActions;
 
     /**
      * @param reqs Requests.
      */
-    public DynamicCacheChangeBatch(
-        Collection<DynamicCacheChangeRequest> reqs
-    ) {
+    public DynamicCacheChangeBatch(Collection<DynamicCacheChangeRequest> reqs) {
+        assert !F.isEmpty(reqs) : reqs;
+
         this.reqs = reqs;
     }
 
@@ -61,34 +58,6 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
         return id;
     }
 
-    /**
-     * @param id Message ID.
-     */
-    public void id(IgniteUuid id) {
-        this.id = id;
-    }
-
-    /**
-     * @return Collection of change requests.
-     */
-    public Collection<DynamicCacheChangeRequest> requests() {
-        return reqs;
-    }
-
-    /**
-     * @return Client nodes map.
-     */
-    public Map<String, Map<UUID, Boolean>> clientNodes() {
-        return clientNodes;
-    }
-
-    /**
-     * @param clientNodes Client nodes map.
-     */
-    public void clientNodes(Map<String, Map<UUID, Boolean>> clientNodes) {
-        this.clientNodes = clientNodes;
-    }
-
     /** {@inheritDoc} */
     @Nullable @Override public DiscoveryCustomMessage ackMessage() {
         return null;
@@ -100,31 +69,21 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     }
 
     /**
-     * @param clientReconnect {@code True} if this is discovery data sent on client reconnect.
-     */
-    public void clientReconnect(boolean clientReconnect) {
-        this.clientReconnect = clientReconnect;
-    }
-
-    /**
-     * @return {@code True} if this is discovery data sent on client reconnect.
+     * @return Collection of change requests.
      */
-    public boolean clientReconnect() {
-        return clientReconnect;
+    public Collection<DynamicCacheChangeRequest> requests() {
+        return reqs;
     }
 
     /**
      * @return {@code True} if request should trigger partition exchange.
      */
     public boolean exchangeNeeded() {
-        if (reqs != null) {
-            for (DynamicCacheChangeRequest req : reqs) {
-                if (req.exchangeNeeded())
-                    return true;
-            }
-        }
+        return exchangeActions != null && !exchangeActions.empty();
+    }
 
-        return false;
+    void exchangeActions(ExchangeActions exchangeActions) {
+        this.exchangeActions = exchangeActions;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/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 9d2563d..642ec04 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
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -89,9 +90,6 @@ public class DynamicCacheChangeRequest implements Serializable {
     private QuerySchema schema;
 
     /** */
-    private transient boolean exchangeNeeded;
-
-    /** */
     private transient AffinityTopologyVersion cacheFutTopVer;
 
     /**
@@ -106,6 +104,44 @@ public class DynamicCacheChangeRequest implements Serializable {
         this.initiatingNodeId = initiatingNodeId;
     }
 
+    static DynamicCacheChangeRequest resetLostPartitions(GridKernalContext ctx, String cacheName) {
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
+
+        req.markResetLostPartitions();
+
+        return req;
+    }
+
+    static DynamicCacheChangeRequest addTemplateRequest(GridKernalContext ctx, CacheConfiguration<?, ?> cfg0) {
+        CacheConfiguration<?, ?> cfg = new CacheConfiguration<>(cfg0);
+
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cfg.getName(), ctx.localNodeId());
+
+        req.template(true);
+        req.startCacheConfiguration(cfg);
+        req.schema(new QuerySchema(cfg.getQueryEntities()));
+        req.deploymentId(IgniteUuid.randomUuid());
+
+        return req;
+    }
+
+    static DynamicCacheChangeRequest closeRequest(GridKernalContext ctx, String cacheName) {
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
+
+        req.close(true);
+
+        return req;
+    }
+
+    static DynamicCacheChangeRequest stopRequest(GridKernalContext ctx, String cacheName, boolean destroy) {
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
+
+        req.stop(true);
+        req.destroy(destroy);
+
+        return req;
+    }
+
     /**
      * @return Request ID.
      */
@@ -114,13 +150,6 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
-     * @return {@code True} if request should trigger partition exchange.
-     */
-    public boolean exchangeNeeded() {
-        return exchangeNeeded;
-    }
-
-    /**
      * @return State.
      */
     public ClusterState state() {
@@ -156,13 +185,6 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
-     * @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/b71d4ef2/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 88985fd..536da79 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
@@ -206,26 +206,6 @@ public class DynamicCacheDescriptor {
     }
 
     /**
-     * @return {@code True} if started flag was flipped by this call.
-     */
-    public boolean onStart() {
-        if (!started) {
-            started = true;
-
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * @return Started flag.
-     */
-    public boolean started() {
-        return started;
-    }
-
-    /**
      * @return Cache configuration.
      */
     public CacheConfiguration cacheConfiguration() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
new file mode 100644
index 0000000..5adafc9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -0,0 +1,90 @@
+/*
+ * 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.internal.util.typedef.F;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class ExchangeActions {
+    /** */
+    private List<ActionData<DynamicCacheDescriptor>> cachesToStart;
+
+    /** */
+    private List<ActionData<DynamicCacheDescriptor>> clientCachesToStart;
+
+    /** */
+    private List<ActionData<String>> cachesToStop;
+
+    /** */
+    private List<ActionData<String>> cachesToClose;
+
+    void addCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        if (cachesToStart == null)
+            cachesToStart = new ArrayList<>();
+
+        cachesToStart.add(new ActionData<>(req, desc));
+    }
+
+    void addClientCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        if (clientCachesToStart == null)
+            clientCachesToStart = new ArrayList<>();
+
+        clientCachesToStart.add(new ActionData<>(req, desc));
+    }
+
+    void addCacheToStop(DynamicCacheChangeRequest req) {
+        if (cachesToStop == null)
+            cachesToStop = new ArrayList<>();
+
+        cachesToStop.add(new ActionData<>(req, req.cacheName()));
+    }
+
+    void addCacheToClose(DynamicCacheChangeRequest req) {
+        if (cachesToClose == null)
+            cachesToClose = new ArrayList<>();
+
+        cachesToClose.add(new ActionData<>(req, req.cacheName()));
+    }
+
+    boolean empty() {
+        return F.isEmpty(cachesToStart) &&
+            F.isEmpty(clientCachesToStart) &&
+            F.isEmpty(cachesToStop) &&
+            F.isEmpty(cachesToClose);
+    }
+
+    void addFutureToComplete() {
+
+    }
+
+    static class ActionData<T> {
+        DynamicCacheChangeRequest req;
+
+        T data;
+
+        public ActionData(DynamicCacheChangeRequest req, T data) {
+            this.req = req;
+            this.data = data;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b71d4ef2/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 ea9dfa5..4f86dac 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
@@ -200,7 +200,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private final CountDownLatch cacheStartedLatch = new CountDownLatch(1);
 
     /** */
-    private Map<UUID, DynamicCacheChangeBatch> clientReconnectReqs;
+    //private Map<UUID, DynamicCacheChangeBatch> clientReconnectReqs;
 
     /** Internal cache names. */
     private final Set<String> internalCaches;
@@ -1224,13 +1224,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 }
             }
         }
-
-        if (clientReconnectReqs != null) {
-            for (Map.Entry<UUID, DynamicCacheChangeBatch> e : clientReconnectReqs.entrySet())
-                processClientReconnectData(e.getKey(), e.getValue());
-
-            clientReconnectReqs = null;
-        }
+//
+//        if (clientReconnectReqs != null) {
+//            for (Map.Entry<UUID, DynamicCacheChangeBatch> e : clientReconnectReqs.entrySet())
+//                processClientReconnectData(e.getKey(), e.getValue());
+//
+//            clientReconnectReqs = null;
+//        }
 
         sharedCtx.onReconnected();
 
@@ -2087,14 +2087,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param req Request to complete future for.
      */
     public void completeStartFuture(DynamicCacheChangeRequest req) {
-        DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
+        if (req.initiatingNodeId().equals(ctx.localNodeId())) {
+            DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 
-        assert req.deploymentId() != null || req.globalStateChange() || req.resetLostPartitions();
-        assert fut == null || fut.deploymentId != null || req.globalStateChange() || req.resetLostPartitions();
+            assert req.deploymentId() != null || req.globalStateChange() || req.resetLostPartitions();
+            assert fut == null || req.globalStateChange() || req.resetLostPartitions();
 
-        if (fut != null && F.eq(fut.deploymentId(), req.deploymentId()) &&
-            F.eq(req.initiatingNodeId(), ctx.localNodeId()))
-            fut.onDone();
+            if (fut != null)
+                fut.onDone();
+        }
     }
 
     /**
@@ -2173,39 +2174,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param clientNodeId Client node ID.
-     * @param batch Cache change batch.
-     */
-    private void processClientReconnectData(UUID clientNodeId, DynamicCacheChangeBatch batch) {
-        assert batch.clientReconnect() : batch;
-
-        for (DynamicCacheChangeRequest req : batch.requests()) {
-            assert !req.template() : req;
-
-            initReceivedCacheConfiguration(req);
-
-            String name = req.cacheName();
-
-            boolean sysCache = CU.isUtilityCache(name) || CU.isAtomicsCache(name);
-
-            if (!sysCache) {
-                DynamicCacheDescriptor desc = cacheDescriptor(req.cacheName());
-
-                if (desc != null && desc.deploymentId().equals(req.deploymentId())) {
-                    Map<UUID, Boolean> nodes = batch.clientNodes().get(name);
-
-                    assert nodes != null : req;
-                    assert nodes.containsKey(clientNodeId) : nodes;
-
-                    ctx.discovery().addClientNode(req.cacheName(), clientNodeId, nodes.get(clientNodeId));
-                }
-            }
-            else
-                ctx.discovery().addClientNode(req.cacheName(), clientNodeId, false);
-        }
-    }
-
-    /**
      * Dynamically starts cache using template configuration.
      *
      * @param cacheName Cache name.
@@ -2469,12 +2437,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (checkThreadTx)
             checkEmptyTransactions();
 
-        DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
+        DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true);
 
-        t.stop(true);
-        t.destroy(true);
-
-        return F.first(initiateCacheChanges(F.asList(t), false));
+        return F.first(initiateCacheChanges(F.asList(req), false));
     }
 
     /**
@@ -2489,11 +2454,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         List<DynamicCacheChangeRequest> reqs = new ArrayList<>(cacheNames.size());
 
         for (String cacheName : cacheNames) {
-            DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
+            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true);
 
-            t.stop(true);
-
-            reqs.add(t);
+            reqs.add(req);
         }
 
         GridCompoundFuture<?, ?> compoundFut = new GridCompoundFuture<>();
@@ -2520,9 +2483,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         checkEmptyTransactions();
 
-        DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
-
-        t.close(true);
+        DynamicCacheChangeRequest t = DynamicCacheChangeRequest.closeRequest(ctx, cacheName);
 
         return F.first(initiateCacheChanges(F.asList(t), false));
     }
@@ -2551,10 +2512,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 continue;
             }
 
-            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(
-                UUID.randomUUID(), cacheName, ctx.localNodeId());
-
-            req.markResetLostPartitions();
+            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.resetLostPartitions(ctx, cacheName);
 
             reqs.add(req);
         }
@@ -2607,14 +2565,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         List<DynamicCacheChangeRequest> reqs = new ArrayList<>();
 
         for (String cacheName : cacheNames()) {
-            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(
-                UUID.randomUUID(), cacheName, ctx.localNodeId());
-
-            DynamicCacheDescriptor desc = cacheDescriptor(cacheName);
-
-            req.deploymentId(desc.deploymentId());
-            req.stop(true);
-            req.destroy(false);
+            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false);
 
             reqs.add(req);
         }
@@ -2642,11 +2593,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         String cacheName = cfg.getName();
 
-        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(
-            UUID.randomUUID(), cacheName, ctx.localNodeId());
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 
         req.startCacheConfiguration(cfg);
-        req.template(cfg.getName() != null && cfg.getName().endsWith("*"));
+        req.template(cfg.getName().endsWith("*"));
         req.nearCacheConfiguration(cfg.getNearConfiguration());
         req.deploymentId(IgniteUuid.randomUuid());
         req.schema(new QuerySchema(cfg.getQueryEntities()));
@@ -2676,7 +2626,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         Collection<DynamicCacheChangeRequest> sndReqs = new ArrayList<>(reqs.size());
 
         for (DynamicCacheChangeRequest req : reqs) {
-            DynamicCacheStartFuture fut = new DynamicCacheStartFuture(req.cacheName(), req.deploymentId(), req);
+            DynamicCacheStartFuture fut = new DynamicCacheStartFuture(req.cacheName(), req);
 
             try {
                 if (req.stop() || req.close()) {
@@ -2690,17 +2640,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                         if (req.close() && desc.cacheConfiguration().getCacheMode() == LOCAL) {
                             req.close(false);
-
                             req.stop(true);
+                            req.destroy(true);
                         }
-
-                        IgniteUuid dynamicDeploymentId = desc.deploymentId();
-
-                        assert dynamicDeploymentId != null : desc;
-
-                        // Save deployment ID to avoid concurrent stops.
-                        req.deploymentId(dynamicDeploymentId);
-                        fut.deploymentId = dynamicDeploymentId;
                     }
                 }
 
@@ -2710,7 +2652,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 DynamicCacheStartFuture old = (DynamicCacheStartFuture)pendingFuts.putIfAbsent(
                     req.requestId(), fut);
 
-                assert old == null; //TODO : check failIfExists.
+                assert old == null;
 
                 if (fut.isDone())
                     continue;
@@ -2803,187 +2745,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (ctx.localNodeId().equals(req.initiatingNodeId())) {
             DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 
-            if (fut != null && F.eq(req.deploymentId(), fut.deploymentId()))
+            if (fut != null)
                 fut.onDone(err);
         }
     }
 
     /**
-     * @param batch Change request batch.
-     * @param topVer Current topology version.
-     * @return {@code True} if minor topology version should be increased.
-     */
-    private boolean onCacheChangeRequested(
-        DynamicCacheChangeBatch batch,
-        AffinityTopologyVersion topVer
-    ) {
-        AffinityTopologyVersion newTopVer = null;
-
-        boolean incMinorTopVer = false;
-
-        for (DynamicCacheChangeRequest req : batch.requests()) {
-            initReceivedCacheConfiguration(req);
-
-            if (req.template()) {
-                CacheConfiguration ccfg = req.startCacheConfiguration();
-
-                assert ccfg != null : req;
-
-                DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(req.cacheName());
-
-                if (desc == null) {
-                    DynamicCacheDescriptor templateDesc = new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), true,
-                        req.deploymentId(), req.schema());
-
-                    DynamicCacheDescriptor old = cachesInfo.registeredTemplates().put(ccfg.getName(), templateDesc);
-
-                    assert old == null :
-                        "Dynamic cache map was concurrently modified [new=" + templateDesc + ", old=" + old + ']';
-                }
-
-                TemplateConfigurationFuture fut =
-                    (TemplateConfigurationFuture)pendingTemplateFuts.get(ccfg.getName());
-
-                if (fut != null && fut.deploymentId().equals(req.deploymentId()))
-                    fut.onDone();
-
-                continue;
-            }
-
-            DynamicCacheDescriptor desc = cacheDescriptor(req.cacheName());
-
-            DynamicCacheStartFuture fut = null;
-
-            if (ctx.localNodeId().equals(req.initiatingNodeId())) {
-                fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
-
-                if (fut != null && !F.eq(req.deploymentId(), fut.deploymentId()))
-                    fut = null;
-            }
-
-            boolean needExchange = false;
-
-            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;
-
-                        DynamicCacheDescriptor startDesc = new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false,
-                            req.deploymentId(), req.schema());
-
-                        if (newTopVer == null) {
-                            newTopVer = new AffinityTopologyVersion(topVer.topologyVersion(),
-                                topVer.minorTopologyVersion() + 1);
-                        }
-
-                        startDesc.startTopologyVersion(newTopVer);
-
-                        // TODO
-                        DynamicCacheDescriptor old = null;//cacheDescriptor(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());
-
-                        ctx.discovery().addClientNode(req.cacheName(),
-                            req.initiatingNodeId(),
-                            req.nearCacheConfiguration() != null);
-
-                        needExchange = true;
-                    }
-                }
-                else {
-                    assert req.initiatingNodeId() != null : req;
-
-                    // Cache already exists, exchange is needed only if client cache should be created.
-                    ClusterNode node = ctx.discovery().node(req.initiatingNodeId());
-
-                    boolean clientReq = node != null &&
-                        !ctx.discovery().cacheAffinityNode(node, req.cacheName());
-
-                    if (req.clientStartOnly()) {
-                        needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
-                            req.initiatingNodeId(),
-                            req.nearCacheConfiguration() != null);
-                    }
-                    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 {
-                            needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
-                                req.initiatingNodeId(),
-                                req.nearCacheConfiguration() != null);
-
-                            if (needExchange)
-                                req.clientStartOnly(true);
-                        }
-                    }
-
-                    if (needExchange) {
-                        if (newTopVer == null) {
-                            newTopVer = new AffinityTopologyVersion(topVer.topologyVersion(),
-                                topVer.minorTopologyVersion() + 1);
-                        }
-
-                        desc.clientCacheStartVersion(newTopVer);
-                    }
-                }
-
-                if (!needExchange && desc != null) {
-                    if (desc.clientCacheStartVersion() != null)
-                        req.cacheFutureTopologyVersion(desc.clientCacheStartVersion());
-                    else
-                        req.cacheFutureTopologyVersion(desc.startTopologyVersion());
-                }
-            }
-            else if (req.globalStateChange() || req.resetLostPartitions())
-                needExchange = true;
-            else {
-                assert req.stop() ^ req.close() : req;
-
-                if (desc != null) {
-                    if (req.stop()) {
-                        DynamicCacheDescriptor old = cachesInfo.registeredCaches().remove(req.cacheName());
-
-                        assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
-
-                        ctx.discovery().removeCacheFilter(req.cacheName());
-
-                        needExchange = true;
-                    }
-                    else {
-                        assert req.close() : req;
-
-                        needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
-                    }
-                }
-            }
-
-            req.exchangeNeeded(needExchange);
-
-            incMinorTopVer |= needExchange;
-        }
-
-        return incMinorTopVer;
-    }
-
-    /**
      * @param req Cache change request.
      */
     private void initReceivedCacheConfiguration(DynamicCacheChangeRequest req) {
@@ -3448,23 +3215,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public void addCacheConfiguration(CacheConfiguration cacheCfg) throws IgniteCheckedException {
         assert cacheCfg.getName() != null;
 
-        String masked = cacheCfg.getName();
+        String name = cacheCfg.getName();
 
-        DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(masked);
+        DynamicCacheDescriptor desc = cachesInfo.registeredTemplates().get(name);
 
         if (desc != null)
             return;
 
-        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheCfg.getName(), ctx.localNodeId());
-
-        CacheConfiguration cfg = new CacheConfiguration(cacheCfg);
-
-        req.template(true);
-
-        req.startCacheConfiguration(cfg);
-        req.schema(new QuerySchema(cfg.getQueryEntities()));
-
-        req.deploymentId(IgniteUuid.randomUuid());
+        DynamicCacheChangeRequest req = DynamicCacheChangeRequest.addTemplateRequest(ctx, cacheCfg);
 
         TemplateConfigurationFuture fut = new TemplateConfigurationFuture(req.cacheName(), req.deploymentId());
 
@@ -3937,10 +3695,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
     private class DynamicCacheStartFuture extends GridFutureAdapter<Object> {
-        /** Start ID. */
-        @GridToStringInclude
-        private IgniteUuid deploymentId;
-
         /** Cache name. */
         private String cacheName;
 
@@ -3950,23 +3704,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         /**
          * @param cacheName Cache name.
-         * @param deploymentId Deployment ID.
          * @param req Cache start request.
          */
-        private DynamicCacheStartFuture(String cacheName, IgniteUuid deploymentId, DynamicCacheChangeRequest req) {
-            this.deploymentId = deploymentId;
+        private DynamicCacheStartFuture(String cacheName, DynamicCacheChangeRequest req) {
             this.cacheName = cacheName;
             this.req = req;
         }
 
         /**
-         * @return Start ID.
-         */
-        public IgniteUuid deploymentId() {
-            return deploymentId;
-        }
-
-        /**
          * @return Request.
          */
         public DynamicCacheChangeRequest request() {


[08/26] ignite git commit: start cache

Posted by sb...@apache.org.
start cache


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

Branch: refs/heads/ignite-5075
Commit: dfad84346a84d4d8eb470b14325e9e753d976147
Parents: 82f3a2f
Author: sboikov <sb...@gridgain.com>
Authored: Thu Apr 27 18:26:02 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Apr 27 18:26:02 2017 +0300

----------------------------------------------------------------------
 .../affinity/AffinityTopologyVersion.java       |  4 +++-
 .../cache/CacheAffinitySharedManager.java       |  2 --
 .../CacheClientReconnectDiscoveryData.java      |  9 ++++++++
 .../internal/processors/cache/CacheData.java    |  2 +-
 .../cache/CacheJoinNodeDiscoveryData.java       |  2 +-
 .../cache/CacheNodeCommonDiscoveryData.java     |  7 ++++++-
 .../cache/DynamicCacheChangeBatch.java          |  2 --
 .../cache/DynamicCacheChangeRequest.java        | 22 +++++---------------
 .../processors/cache/ExchangeActions.java       |  3 ++-
 .../processors/cache/GridCacheContext.java      |  3 ++-
 .../processors/cache/GridCacheIoManager.java    |  2 +-
 .../processors/cache/GridCacheProcessor.java    |  3 ---
 12 files changed, 30 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index 4011d9b..0a6d965 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -23,7 +23,6 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 
-import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -74,6 +73,9 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
         this.minorTopVer = minorTopVer;
     }
 
+    /**
+     * @return Topology version with incremented minor version.
+     */
     public AffinityTopologyVersion nextMinorVersion() {
         assert topVer > 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 60eeef0..e599231 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -356,8 +356,6 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             assert old == null : old;
         }
 
-        boolean clientOnly = true;
-
         // Affinity did not change for existing caches.
         forAllCaches(crd && lateAffAssign, new IgniteInClosureX<GridAffinityAssignmentCache>() {
             @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
index 34657b4..b791e35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClientReconnectDiscoveryData.java
@@ -30,14 +30,23 @@ public class CacheClientReconnectDiscoveryData implements Serializable {
     /** */
     private final Map<String, CacheInfo> clientCaches;
 
+    /**
+     * @param clientCaches Information about caches started on re-joining client node.
+     */
     public CacheClientReconnectDiscoveryData(Map<String, CacheInfo> clientCaches) {
         this.clientCaches = clientCaches;
     }
 
+    /**
+     * @return Information about caches started on re-joining client node.
+     */
     Map<String, CacheInfo> clientCaches() {
         return clientCaches;
     }
 
+    /**
+     *
+     */
     static class CacheInfo implements Serializable {
         /** */
         private final CacheConfiguration ccfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
index 4579c27..39c6e90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 
 /**
- *
+ * Cache information sent in discovery data to joining node.
  */
 public class CacheData implements Serializable {
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
index 0624217..e04e739 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
@@ -23,7 +23,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.lang.IgniteUuid;
 
 /**
- *
+ * Information about configured caches sent from joining node.
  */
 class CacheJoinNodeDiscoveryData implements Serializable {
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
index 10df452..a2a7b30 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheNodeCommonDiscoveryData.java
@@ -22,7 +22,7 @@ import java.util.Map;
 import java.util.UUID;
 
 /**
- *
+ * Cache information sent in discovery data to joining node.
  */
 class CacheNodeCommonDiscoveryData implements Serializable {
     /** */
@@ -34,6 +34,11 @@ class CacheNodeCommonDiscoveryData implements Serializable {
     /** */
     private final Map<String, Map<UUID, Boolean>> clientNodesMap;
 
+    /**
+     * @param caches
+     * @param templates
+     * @param clientNodesMap
+     */
     CacheNodeCommonDiscoveryData(Map<String, CacheData> caches,
         Map<String, CacheData> templates,
         Map<String, Map<UUID, Boolean>> clientNodesMap) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index f69246e..56639b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -18,8 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.Collection;
-import java.util.Map;
-import java.util.UUID;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/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 642ec04..ee316ab 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
@@ -89,16 +89,18 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** Dynamic schema. */
     private QuerySchema schema;
 
-    /** */
-    private transient AffinityTopologyVersion cacheFutTopVer;
-
     /**
      * Constructor creates cache stop request.
      *
+     * @param reqId Unique request ID.
      * @param cacheName Cache stop name.
      * @param initiatingNodeId Initiating node ID.
      */
     public DynamicCacheChangeRequest(UUID reqId, String cacheName, UUID initiatingNodeId) {
+        assert reqId != null;
+        assert cacheName != null;
+        assert initiatingNodeId != null;
+
         this.reqId = reqId;
         this.cacheName = cacheName;
         this.initiatingNodeId = initiatingNodeId;
@@ -171,20 +173,6 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
-     * @param cacheFutTopVer Ready topology version when dynamic cache future should be completed.
-     */
-    public void cacheFutureTopologyVersion(AffinityTopologyVersion cacheFutTopVer) {
-        this.cacheFutTopVer = cacheFutTopVer;
-    }
-
-    /**
-     * @return Ready topology version when dynamic cache future should be completed.
-     */
-    @Nullable public AffinityTopologyVersion cacheFutureTopologyVersion() {
-        return cacheFutTopVer;
-    }
-
-    /**
      * @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/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index a7b62ca..3d14f23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -23,7 +23,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.internal.util.typedef.F;
 
 import java.util.ArrayList;
@@ -211,8 +210,10 @@ public class ExchangeActions {
      *
      */
     static class ActionData {
+        /** */
         private DynamicCacheChangeRequest req;
 
+        /** */
         private DynamicCacheDescriptor desc;
 
         public ActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 67f25b5..1f6391b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -234,9 +234,10 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** */
     private CountDownLatch startLatch = new CountDownLatch(1);
 
-    /** Start topology version. */
+    /** Topology version when cache was started on local node. */
     private AffinityTopologyVersion startTopVer;
 
+    /** Global cache start topology version. */
     private AffinityTopologyVersion cacheStartTopVer;
 
     /** Dynamic cache deployment ID. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index b9c066b..a80213d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -150,7 +150,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                     DynamicCacheDescriptor cacheDesc = cctx.cache().cacheDescriptor(cacheMsg.cacheId());
 
-                    // TODO: should be specified on request since cache desc can be removed,
+                    // TODO: should be specified in request since cache desc can be removed,
                     if (cacheDesc != null)
                         startTopVer = cacheDesc.startTopologyVersion();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dfad8434/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 a68d174..d486d3a 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
@@ -199,9 +199,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Count down latch for caches. */
     private final CountDownLatch cacheStartedLatch = new CountDownLatch(1);
 
-    /** */
-    //private Map<UUID, DynamicCacheChangeBatch> clientReconnectReqs;
-
     /** Internal cache names. */
     private final Set<String> internalCaches;
 


[10/26] ignite git commit: ignite-5075

Posted by sb...@apache.org.
ignite-5075


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

Branch: refs/heads/ignite-5075
Commit: 0c66d0e01b81f9a33efe067c31712725f8dee498
Parents: ea022d0
Author: sboikov <sb...@gridgain.com>
Authored: Fri Apr 28 11:12:33 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Apr 28 15:31:17 2017 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  11 +-
 .../affinity/AffinityTopologyVersion.java       |   1 -
 .../cache/CacheAffinitySharedManager.java       |  80 ++++--
 .../internal/processors/cache/CacheData.java    |   8 +-
 .../processors/cache/ClusterCachesInfo.java     | 164 ++++++++----
 .../cache/DynamicCacheChangeBatch.java          |  10 +-
 .../cache/DynamicCacheChangeRequest.java        |  11 +-
 .../cache/DynamicCacheDescriptor.java           |  13 +
 .../processors/cache/ExchangeActions.java       |  82 +++++-
 .../processors/cache/GridCacheContext.java      |   3 +
 .../processors/cache/GridCacheProcessor.java    | 256 ++++---------------
 .../ignite/spi/discovery/DiscoveryDataBag.java  |   4 +
 .../processors/cache/IgniteCacheStartTest.java  | 176 +++++++++++++
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   2 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 15 files changed, 524 insertions(+), 299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/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 e5f2278..24c7283 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
@@ -315,8 +315,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         boolean nearEnabled,
         CacheMode cacheMode
     ) {
-        if (!registeredCaches.containsKey(cacheName))
+        if (!registeredCaches.containsKey(cacheName)) {
+            if (cacheMode == CacheMode.REPLICATED)
+                nearEnabled = false;
+            
             registeredCaches.put(cacheName, new CachePredicate(filter, nearEnabled, cacheMode));
+        }
     }
 
     /**
@@ -2737,7 +2741,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if this node is a data node for given cache.
          */
         public boolean dataNode(ClusterNode node) {
-            return !node.isDaemon() && CU.affinityNode(node, cacheFilter);
+            return CU.affinityNode(node, cacheFilter);
         }
 
         /**
@@ -2753,9 +2757,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if near cache is present on the given nodes.
          */
         public boolean nearNode(ClusterNode node) {
-            if (node.isDaemon())
-                return false;
-
             if (CU.affinityNode(node, cacheFilter))
                 return nearEnabled;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index 0a6d965..8669530 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
-
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index e599231..6b99e22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -32,6 +32,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -318,21 +319,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         }
     }
 
-    /**
-     * Called on exchange initiated for cache start/stop request.
-     *
-     * @param fut Exchange future.
-     * @param crd Coordinator flag.
-     * @param exchActions Cache change requests.
-     * @throws IgniteCheckedException If failed.
-     * @return {@code True} if client-only exchange is needed.
-     */
-    public boolean onCacheChangeRequest(final GridDhtPartitionsExchangeFuture fut,
-        boolean crd,
-        ExchangeActions exchActions)
-        throws IgniteCheckedException {
-        assert exchActions != null && !exchActions.empty() : fut;
-
+    private void updateCachesInfo(ExchangeActions exchActions) {
         for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
             Integer cacheId = CU.cacheId(req.cacheName());
 
@@ -341,7 +328,9 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             assert desc != null : cacheId;
         }
 
-        for (DynamicCacheChangeRequest req : exchActions.startRequests()) {
+        for (ExchangeActions.ActionData action : exchActions.newCachesStartRequests()) {
+            DynamicCacheChangeRequest req = action.request();
+
             Integer cacheId = CU.cacheId(req.cacheName());
 
             DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(),
@@ -355,6 +344,25 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
             assert old == null : old;
         }
+    }
+
+    /**
+     * Called on exchange initiated for cache start/stop request.
+     *
+     * @param fut Exchange future.
+     * @param crd Coordinator flag.
+     * @param exchActions Cache change requests.
+     * @throws IgniteCheckedException If failed.
+     * @return {@code True} if client-only exchange is needed.
+     */
+    public boolean onCacheChangeRequest(final GridDhtPartitionsExchangeFuture fut,
+        boolean crd,
+        ExchangeActions exchActions)
+        throws IgniteCheckedException
+    {
+        assert exchActions != null && !exchActions.empty() : exchActions;
+
+        updateCachesInfo(exchActions);
 
         // Affinity did not change for existing caches.
         forAllCaches(crd && lateAffAssign, new IgniteInClosureX<GridAffinityAssignmentCache>() {
@@ -366,10 +374,27 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             }
         });
 
-        for (DynamicCacheChangeRequest req : exchActions.startRequests()) {
+        for (ExchangeActions.ActionData action : exchActions.newAndClientCachesStartRequests()) {
+            DynamicCacheChangeRequest req = action.request();
+
             Integer cacheId = CU.cacheId(req.cacheName());
 
-            cctx.cache().prepareCacheStart(req, fut.topologyVersion());
+            boolean startCache;
+
+            NearCacheConfiguration nearCfg = null;
+
+            if (cctx.localNodeId().equals(req.initiatingNodeId())) {
+                startCache = true;
+
+                nearCfg = req.nearCacheConfiguration();
+            }
+            else {
+                startCache = cctx.cacheContext(action.descriptor().cacheId()) == null &&
+                    CU.affinityNode(cctx.localNode(), req.startCacheConfiguration().getNodeFilter());
+            }
+
+            if (startCache)
+                cctx.cache().prepareCacheStart(req, nearCfg, action.descriptor(), fut.topologyVersion());
 
             if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
                 if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
@@ -411,18 +436,19 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             if (crd) {
                 GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
 
-                assert cacheCtx != null : req;
+                // Client cache was stopped, need create 'client' CacheHolder.
+                if (cacheCtx != null && !cacheCtx.affinityNode()) {
+                    CacheHolder cache = caches.remove(cacheId);
 
-                CacheHolder cache = caches.remove(cacheId);
+                    assert !cache.client() : cache;
 
-                assert !cache.client();
+                    cache = CacheHolder2.create(cctx,
+                        cctx.cache().cacheDescriptor(cacheId),
+                        fut,
+                        cache.affinity());
 
-                cache = CacheHolder2.create(cctx,
-                    cctx.cache().cacheDescriptor(cacheId),
-                    fut,
-                    cache.affinity());
-
-                caches.put(cacheId, cache);
+                    caches.put(cacheId, cache);
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
index 39c6e90..b38e03f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -66,10 +66,10 @@ public class CacheData implements Serializable {
         boolean staticCfg,
         boolean template) {
         assert cacheCfg != null;
-        assert rcvdFrom != null;
-        assert startTopVer != null;
-        assert deploymentId != null;
-        assert template || cacheId != 0;
+        assert rcvdFrom != null : cacheCfg.getName();
+        assert startTopVer != null : cacheCfg.getName();
+        assert deploymentId != null : cacheCfg.getName();
+        assert template || cacheId != 0 : cacheCfg.getName();
 
         this.cacheCfg = cacheCfg;
         this.cacheId = cacheId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index cd2cd77..6cc09a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -17,11 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collections;
 import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheExistsException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -46,6 +48,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC;
 
 /**
  *
@@ -61,13 +64,19 @@ class ClusterCachesInfo {
     private final ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates = new ConcurrentHashMap<>();
 
     /** */
+    private Map<String, DynamicCacheDescriptor> cachesOnDisconnect;
+
+    /** */
     private CacheJoinNodeDiscoveryData joinDiscoData;
 
     /** */
     private CacheNodeCommonDiscoveryData gridData;
 
     /** */
-    private List<DynamicCacheDescriptor> locJoinStartCaches;
+    private List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> locJoinStartCaches;
+
+    /** */
+    private Map<UUID, CacheJoinNodeDiscoveryData> joiningNodesDiscoData = new HashMap<>();
 
     /**
      * @param ctx Context.
@@ -81,7 +90,7 @@ class ClusterCachesInfo {
     }
 
     void onKernalStart() throws IgniteCheckedException {
-
+        // TODO: validate cache configurations.
     }
 
     /**
@@ -104,7 +113,7 @@ class ClusterCachesInfo {
 
                 assert ccfg != null : req;
 
-                DynamicCacheDescriptor desc = registeredTemplates().get(req.cacheName());
+                DynamicCacheDescriptor desc = registeredTemplates.get(req.cacheName());
 
                 if (desc == null) {
                     DynamicCacheDescriptor templateDesc = new DynamicCacheDescriptor(ctx,
@@ -204,6 +213,8 @@ class ClusterCachesInfo {
                     }
 
                     if (needExchange) {
+                        req.clientStartOnly(true);
+
                         desc.clientCacheStartVersion(topVer.nextMinorVersion());
 
                         exchangeActions.addClientCacheToStart(req, desc);
@@ -222,39 +233,37 @@ class ClusterCachesInfo {
             else if (req.globalStateChange())
                 needExchange = true;
             else if (req.resetLostPartitions()) {
-                needExchange = desc != null;
+                if (desc != null) {
+                    needExchange = true;
 
-                if (needExchange)
                     exchangeActions.addCacheToResetLostPartitions(req, desc);
+                }
             }
-            else {
+            else if (req.stop()) {
                 assert req.stop() ^ req.close() : req;
 
                 if (desc != null) {
-                    if (req.stop()) {
-                        DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
-
-                        assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
+                    DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
 
-                        ctx.discovery().removeCacheFilter(req.cacheName());
-
-                        needExchange = true;
+                    assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
 
-                        exchangeActions.addCacheToStop(req, desc);
-                    }
-                    else {
-                        assert req.close() : req;
+                    ctx.discovery().removeCacheFilter(req.cacheName());
 
-                        needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
+                    needExchange = true;
 
-                        if (needExchange) {
-                            exchangeActions.addCacheToStop(req, desc);
+                    exchangeActions.addCacheToStop(req, desc);
+                }
+            }
+            else if (req.close()) {
+                if (desc != null) {
+                    needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
 
-                            exchangeActions.addCacheToClose(req, desc);
-                        }
-                    }
+                    if (needExchange)
+                        exchangeActions.addCacheToClose(req, desc);
                 }
             }
+            else
+                assert false : req;
 
             if (!needExchange) {
                 if (req.initiatingNodeId().equals(ctx.localNodeId()))
@@ -267,8 +276,11 @@ class ClusterCachesInfo {
         if (!F.isEmpty(addedDescs)) {
             AffinityTopologyVersion startTopVer = incMinorTopVer ? topVer.nextMinorVersion() : topVer;
 
-            for (DynamicCacheDescriptor desc : addedDescs)
+            for (DynamicCacheDescriptor desc : addedDescs) {
+                assert desc.template() || incMinorTopVer;
+
                 desc.startTopologyVersion(startTopVer);
+            }
         }
 
         if (!F.isEmpty(reqsToComplete)) {
@@ -306,7 +318,15 @@ class ClusterCachesInfo {
         return incMinorTopVer;
     }
 
-    Serializable joinDiscoveryData() {
+    void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        if (!ctx.isDaemon())
+        dataBag.addJoiningNodeData(CACHE_PROC.ordinal(), joinDiscoveryData());
+    }
+
+    /**
+     * @return Discovery date sent on local node join.
+     */
+    private Serializable joinDiscoveryData() {
         if (cachesOnDisconnect != null) {
             Map<String, CacheClientReconnectDiscoveryData.CacheInfo> cachesInfo = new HashMap<>();
 
@@ -336,16 +356,23 @@ class ClusterCachesInfo {
      *
      * @return Caches to be started when this node starts.
      */
-    List<DynamicCacheDescriptor> cachesToStartOnLocalJoin() {
+    List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> cachesToStartOnLocalJoin() {
+        if (ctx.isDaemon())
+            return Collections.emptyList();
+
         assert locJoinStartCaches != null;
 
-        List<DynamicCacheDescriptor> locJoinStartCaches = this.locJoinStartCaches;
+        List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> locJoinStartCaches = this.locJoinStartCaches;
 
         this.locJoinStartCaches = null;
 
         return locJoinStartCaches;
     }
 
+    /**
+     * @param joinedNodeId Joined node ID.
+     * @return New caches received from joined node.
+     */
     List<DynamicCacheDescriptor> cachesReceivedFromJoin(UUID joinedNodeId) {
         assert joinedNodeId != null;
 
@@ -369,7 +396,7 @@ class ClusterCachesInfo {
             }
         }
 
-        return started;
+        return started != null ? started : Collections.<DynamicCacheDescriptor>emptyList();
     }
 
     /**
@@ -380,16 +407,16 @@ class ClusterCachesInfo {
      * @param topVer Topology version.
      */
     void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
-        if (type == EVT_NODE_JOINED) {
+        if (type == EVT_NODE_JOINED && !ctx.isDaemon()) {
             if (node.id().equals(ctx.discovery().localNode().id())) {
                 if (gridData == null) { // First node starts.
                     assert registeredCaches.isEmpty();
                     assert registeredTemplates.isEmpty();
                     assert joinDiscoData != null;
-
-                    processJoiningNode(joinDiscoData, node.id());
                 }
 
+                processJoiningNode(joinDiscoData, node.id());
+
                 assert locJoinStartCaches == null;
 
                 locJoinStartCaches = new ArrayList<>();
@@ -397,14 +424,24 @@ class ClusterCachesInfo {
                 for (DynamicCacheDescriptor desc : registeredCaches.values()) {
                     CacheConfiguration cfg = desc.cacheConfiguration();
 
-                    boolean locCfg = joinDiscoData.caches().containsKey(cfg.getName());
+                    CacheJoinNodeDiscoveryData.CacheInfo locCfg = joinDiscoData.caches().get(cfg.getName());
+
+                    boolean affNode = CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter());
 
-                    if (locCfg || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()))
-                        locJoinStartCaches.add(desc);
+                    NearCacheConfiguration nearCfg = (!affNode && locCfg != null) ? locCfg.config().getNearConfiguration() : null;
+
+                    if (locCfg != null || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()))
+                        locJoinStartCaches.add(new T2<>(desc, nearCfg));
                 }
 
                 joinDiscoData = null;
             }
+            else {
+                CacheJoinNodeDiscoveryData discoData = joiningNodesDiscoData.remove(node.id());
+
+                if (discoData != null)
+                    processJoiningNode(discoData, node.id());
+            }
 
             initStartVersionOnJoin(registeredCaches.values(), node, topVer);
 
@@ -412,6 +449,11 @@ class ClusterCachesInfo {
         }
     }
 
+    /**
+     * @param descs Cache descriptors.
+     * @param joinedNode Joined node.
+     * @param topVer Current topology version.
+     */
     private void initStartVersionOnJoin(Collection<DynamicCacheDescriptor> descs,
         ClusterNode joinedNode,
         AffinityTopologyVersion topVer) {
@@ -421,7 +463,18 @@ class ClusterCachesInfo {
         }
     }
 
-    CacheNodeCommonDiscoveryData collectCommonDiscoveryData() {
+    void collectGridNodeData(DiscoveryDataBag dataBag) {
+        if (ctx.isDaemon())
+            return;
+
+        if (!dataBag.commonDataCollectedFor(CACHE_PROC.ordinal()))
+            dataBag.addGridCommonData(CACHE_PROC.ordinal(), collectCommonDiscoveryData());
+    }
+
+    /**
+     * @return Information about started caches.
+     */
+    private CacheNodeCommonDiscoveryData collectCommonDiscoveryData() {
         Map<String, CacheData> caches = new HashMap<>();
 
         for (DynamicCacheDescriptor desc : registeredCaches.values()) {
@@ -435,7 +488,7 @@ class ClusterCachesInfo {
                 desc.staticallyConfigured(),
                 false);
 
-            caches.put(desc.cacheConfiguration().getName(), cacheData);
+            caches.put(desc.cacheName(), cacheData);
         }
 
         Map<String, CacheData> templates = new HashMap<>();
@@ -451,13 +504,19 @@ class ClusterCachesInfo {
                 desc.staticallyConfigured(),
                 true);
 
-            templates.put(desc.cacheConfiguration().getName(), cacheData);
+            templates.put(desc.cacheName(), cacheData);
         }
 
         return new CacheNodeCommonDiscoveryData(caches, templates, ctx.discovery().clientNodesMap());
     }
 
+    /**
+     * @param data Discovery data.
+     */
     void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) {
+        if (ctx.isDaemon() || data.commonData() == null)
+            return;
+
         assert joinDiscoData != null;
         assert data.commonData() instanceof CacheNodeCommonDiscoveryData : data;
 
@@ -525,12 +584,17 @@ class ClusterCachesInfo {
 
             if (joiningNodeData instanceof CacheClientReconnectDiscoveryData)
                 processClientReconnectData((CacheClientReconnectDiscoveryData)joiningNodeData, data.joiningNodeId());
-            else if (joiningNodeData instanceof CacheJoinNodeDiscoveryData)
-                processJoiningNode((CacheJoinNodeDiscoveryData)joiningNodeData, data.joiningNodeId());
+            else if (joiningNodeData instanceof CacheJoinNodeDiscoveryData) {
+                CacheJoinNodeDiscoveryData old =
+                    joiningNodesDiscoData.put(data.joiningNodeId(), (CacheJoinNodeDiscoveryData)joiningNodeData);
+
+                assert old == null : old;
+            }
         }
     }
 
     /**
+     * @param clientData Discovery data.
      * @param clientNodeId Client node ID.
      */
     private void processClientReconnectData(CacheClientReconnectDiscoveryData clientData, UUID clientNodeId) {
@@ -594,23 +658,27 @@ class ClusterCachesInfo {
                     cfg.getCacheMode());
             }
 
-            ctx.discovery().addClientNode(cfg.getName(),
-                nodeId,
-                cfg.getNearConfiguration() != null);
+            ctx.discovery().addClientNode(cfg.getName(), nodeId, cfg.getNearConfiguration() != null);
         }
     }
 
+    /**
+     * @return Registered caches.
+     */
     ConcurrentMap<String, DynamicCacheDescriptor> registeredCaches() {
         return registeredCaches;
     }
 
+    /**
+     * @return Registered cache templates.
+     */
     ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates() {
         return registeredTemplates;
     }
 
-    /** */
-    private Map<String, DynamicCacheDescriptor> cachesOnDisconnect;
-
+    /**
+     *
+     */
     void onDisconnect() {
         cachesOnDisconnect = new HashMap<>(registeredCaches);
 
@@ -618,6 +686,9 @@ class ClusterCachesInfo {
         registeredTemplates.clear();
     }
 
+    /**
+     * @return Stopped caches names.
+     */
     Set<String> onReconnected() {
         assert cachesOnDisconnect != null;
 
@@ -651,6 +722,9 @@ class ClusterCachesInfo {
         return CU.isUtilityCache(cacheName) || CU.isAtomicsCache(cacheName);
     }
 
+    /**
+     *
+     */
     void clearCaches() {
         registeredCaches.clear();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index 56639b7..e27d5af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -39,7 +39,7 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     @GridToStringInclude
     private Collection<DynamicCacheChangeRequest> reqs;
 
-    /** */
+    /** Cache updates to be executed on exchange. */
     private transient ExchangeActions exchangeActions;
 
     /**
@@ -77,13 +77,19 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
      * @return {@code True} if request should trigger partition exchange.
      */
     public boolean exchangeNeeded() {
-        return exchangeActions != null && !exchangeActions.empty();
+        return exchangeActions != null;
     }
 
+    /**
+     * @return Cache updates to be executed on exchange.
+     */
     ExchangeActions exchangeActions() {
         return exchangeActions;
     }
 
+    /**
+     * @param exchangeActions Cache updates to be executed on exchange.
+     */
     void exchangeActions(ExchangeActions exchangeActions) {
         assert !exchangeActions.empty() : exchangeActions;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/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 ee316ab..e4c95a7 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
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -106,6 +105,11 @@ public class DynamicCacheChangeRequest implements Serializable {
         this.initiatingNodeId = initiatingNodeId;
     }
 
+    /**
+     * @param ctx Context.
+     * @param cacheName Cache name.
+     * @return Request to reset lost partitions.
+     */
     static DynamicCacheChangeRequest resetLostPartitions(GridKernalContext ctx, String cacheName) {
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 
@@ -114,6 +118,11 @@ public class DynamicCacheChangeRequest implements Serializable {
         return req;
     }
 
+    /**
+     * @param ctx Context.
+     * @param cfg0 Template configuration.
+     * @return Request to add template.
+     */
     static DynamicCacheChangeRequest addTemplateRequest(GridKernalContext ctx, CacheConfiguration<?, ?> cfg0) {
         CacheConfiguration<?, ?> cfg = new CacheConfiguration<>(cfg0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/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 536da79..bae711a 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
@@ -21,6 +21,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -123,6 +124,9 @@ public class DynamicCacheDescriptor {
 
         cacheId = CU.cacheId(cacheCfg.getName());
 
+        if (cacheCfg.getCacheMode() == CacheMode.REPLICATED)
+            cacheCfg.setNearConfiguration(null);
+
         synchronized (schemaMux) {
             this.schema = schema.copy();
         }
@@ -206,6 +210,15 @@ public class DynamicCacheDescriptor {
     }
 
     /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        assert cacheCfg != null : this;
+
+        return cacheCfg.getName();
+    }
+
+    /**
      * @return Cache configuration.
      */
     public CacheConfiguration cacheConfiguration() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index 3d14f23..6de02b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -27,6 +28,7 @@ import org.apache.ignite.internal.util.typedef.F;
 
 import java.util.ArrayList;
 import java.util.List;
+import org.jetbrains.annotations.Nullable;
 
 /**
  *
@@ -56,6 +58,10 @@ public class ExchangeActions {
             F.isEmpty(cachesToResetLostParts);
     }
 
+    /**
+     * @param nodeId Local node ID.
+     * @return Close cache requests.
+     */
     public List<DynamicCacheChangeRequest> closeRequests(UUID nodeId) {
         List<DynamicCacheChangeRequest> res = null;
 
@@ -73,19 +79,35 @@ public class ExchangeActions {
         return res != null ? res : Collections.<DynamicCacheChangeRequest>emptyList();
     }
 
-    public List<DynamicCacheChangeRequest> startRequests() {
-        List<DynamicCacheChangeRequest> res = null;
+    /**
+     * @return Start cache requests.
+     */
+    Collection<ActionData> newAndClientCachesStartRequests() {
+        if (cachesToStart != null || clientCachesToStart != null) {
+            List<ActionData> res = new ArrayList<>();
 
-        if (cachesToStart != null) {
-            res = new ArrayList<>(cachesToStart.size());
+            if (cachesToStart != null)
+                res.addAll(cachesToStart.values());
 
-            for (ActionData req : cachesToStart.values())
-                res.add(req.req);
+            if (clientCachesToStart != null)
+                res.addAll(clientCachesToStart.values());
+
+            return res;
         }
 
-        return res != null ? res : Collections.<DynamicCacheChangeRequest>emptyList();
+        return Collections.emptyList();
+    }
+
+    /**
+     * @return Start cache requests.
+     */
+    Collection<ActionData> newCachesStartRequests() {
+        return cachesToStart != null ? cachesToStart.values() : Collections.<ActionData>emptyList();
     }
 
+    /**
+     * @return Stop cache requests.
+     */
     public List<DynamicCacheChangeRequest> stopRequests() {
         List<DynamicCacheChangeRequest> res = null;
 
@@ -99,11 +121,14 @@ public class ExchangeActions {
         return res != null ? res : Collections.<DynamicCacheChangeRequest>emptyList();
     }
 
+    /**
+     * @param ctx Context.
+     */
     public void completeRequestFutures(GridCacheSharedContext ctx) {
         completeRequestFutures(cachesToStart, ctx);
-        completeRequestFutures(clientCachesToStart, ctx);
         completeRequestFutures(cachesToStop, ctx);
         completeRequestFutures(cachesToClose, ctx);
+        completeRequestFutures(clientCachesToStart, ctx);
         completeRequestFutures(cachesToResetLostParts, ctx);
     }
 
@@ -114,10 +139,16 @@ public class ExchangeActions {
         }
     }
 
+    /**
+     * @return {@code True} if have cache stop requests.
+     */
     public boolean hasStop() {
         return !F.isEmpty(cachesToStop);
     }
 
+    /**
+     * @return
+     */
     public Set<String> cachesToResetLostPartitions() {
         Set<String> caches = null;
         
@@ -149,6 +180,10 @@ public class ExchangeActions {
         return false;
     }
 
+    /**
+     * @param nodeId Local node ID.
+     * @return {@code True} if client cache was started.
+     */
     public boolean clientCacheStarted(UUID nodeId) {
         if (clientCachesToStart != null) {
             for (ActionData cache : clientCachesToStart.values()) {
@@ -160,7 +195,10 @@ public class ExchangeActions {
         return false;
     }
 
-    public ClusterState newClusterState() {
+    /**
+     * @return New cluster state if state change was requested.
+     */
+    @Nullable public ClusterState newClusterState() {
         return newState;
     }
 
@@ -179,25 +217,38 @@ public class ExchangeActions {
     }
 
     void addCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        assert req.start() : req;
+
         cachesToStart = add(cachesToStart, req, desc);
     }
 
     void addClientCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        assert req.start() : req;
+
         clientCachesToStart = add(clientCachesToStart, req, desc);
     }
 
     void addCacheToStop(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        assert req.stop() : req;
+
         cachesToStop = add(cachesToStop, req, desc);
     }
 
     void addCacheToClose(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        assert req.close() : req;
+
         cachesToClose = add(cachesToClose, req, desc);
     }
 
     void addCacheToResetLostPartitions(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        assert req.resetLostPartitions() : req;
+
         cachesToResetLostParts = add(cachesToResetLostParts, req, desc);
     }
 
+    /**
+     * @return {@code True} if there are no cache change actions.
+     */
     public boolean empty() {
         return F.isEmpty(cachesToStart) &&
             F.isEmpty(clientCachesToStart) &&
@@ -216,9 +267,20 @@ public class ExchangeActions {
         /** */
         private DynamicCacheDescriptor desc;
 
-        public ActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        ActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+            assert req != null;
+            assert desc != null;
+
             this.req = req;
             this.desc = desc;
         }
+
+        public DynamicCacheChangeRequest request() {
+            return req;
+        }
+
+        public DynamicCacheDescriptor descriptor() {
+            return desc;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 1f6391b..72adeaf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -461,6 +461,9 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.startTopVer = startTopVer;
     }
 
+    /**
+     * @param cacheStartTopVer Global cache start topology version.
+     */
     public void cacheStartTopologyVersion(AffinityTopologyVersion cacheStartTopVer) {
         this.cacheStartTopVer = cacheStartTopVer;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/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 d486d3a..0951676 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
@@ -117,6 +117,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -657,6 +658,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
         Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates) throws IgniteCheckedException {
         CU.validateCacheName(cfg.getName());
+
         cloneCheckSerializable(cfg);
 
         CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
@@ -674,7 +676,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             CacheType cacheType;
 
-        if (CU.isUtilityCache(cfg.getName()))
+            if (CU.isUtilityCache(cfg.getName()))
                 cacheType = CacheType.UTILITY;
             else if (internalCaches.contains(cfg.getName()))
                 cacheType = CacheType.INTERNAL;
@@ -748,96 +750,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param cfg Cache configuration.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
-//        cloneCheckSerializable(cfg);
-//
-//        CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
-//
-//        // Initialize defaults.
-//        initialize(cfg, cacheObjCtx);
-//
-//        String masked = maskNull(cfg.getName());
-//
-//        if (cacheDescriptor(cfg.getName()) != null) {
-//            String cacheName = cfg.getName();
-//
-//            if (cacheName != null)
-//                throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
-//                    "assign unique name to each cache): " + U.maskName(cacheName));
-//            else
-//                throw new IgniteCheckedException("Default cache has already been configured (check configuration and " +
-//                    "assign unique name to each cache).");
-//        }
-//
-//        CacheType cacheType;
-//
-//        if (CU.isUtilityCache(cfg.getName()))
-//            cacheType = CacheType.UTILITY;
-//        else if (internalCaches.contains(maskNull(cfg.getName())))
-//            cacheType = CacheType.INTERNAL;
-//        else
-//            cacheType = CacheType.USER;
-//
-//        if (cacheType != CacheType.USER && cfg.getMemoryPolicyName() == null)
-//            cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
-//
-//        boolean template = cfg.getName() != null && cfg.getName().endsWith("*");
-//
-//        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
-//            cfg,
-//            cacheType,
-//            template,
-//            IgniteUuid.randomUuid(),
-//            new QuerySchema(cfg.getQueryEntities()));
-//
-//        desc.locallyConfigured(true);
-//        desc.staticallyConfigured(true);
-//        desc.receivedFrom(ctx.localNodeId());
-//
-//        if (!template) {
-//            cacheDescriptor(cfg.getName(), desc);
-//
-//            ctx.discovery().setCacheFilter(
-//                cfg.getName(),
-//                cfg.getNodeFilter(),
-//                cfg.getNearConfiguration() != null && cfg.getCacheMode() == PARTITIONED,
-//                cfg.getCacheMode());
-//
-//            ctx.discovery().addClientNode(cfg.getName(),
-//                ctx.localNodeId(),
-//                cfg.getNearConfiguration() != null);
-//
-//            if (!cacheType.userCache())
-//                stopSeq.addLast(cfg.getName());
-//            else
-//                stopSeq.addFirst(cfg.getName());
-//        }
-//        else {
-//            if (log.isDebugEnabled())
-//                log.debug("Use cache configuration as template: " + cfg);
-//
-//            registeredTemplates.put(masked, desc);
-//        }
-//
-//        if (cfg.getName() == null) { // Use cache configuration with null name as template.
-//            DynamicCacheDescriptor desc0 = new DynamicCacheDescriptor(ctx,
-//                cfg,
-//                cacheType,
-//                true,
-//                IgniteUuid.randomUuid(),
-//                new QuerySchema(cfg.getQueryEntities()));
-//
-//            desc0.locallyConfigured(true);
-//            desc0.staticallyConfigured(true);
-//
-//            registeredTemplates.put(masked, desc0);
-//        }
-    }
-
-    /**
      * Initialize internal cache names
      */
     private void initializeInternalCacheNames() {
@@ -908,57 +820,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             ctx.query().onCacheKernalStart();
 
-            // Start dynamic caches received from collect discovery data.
-//            for (DynamicCacheDescriptor desc : cacheDescriptors()) {
-//                if (ctx.config().isDaemon())
-//                    continue;
-//
-//                desc.clearRemoteConfigurations();
-//
-//                CacheConfiguration ccfg = desc.cacheConfiguration();
-//
-//                IgnitePredicate filter = ccfg.getNodeFilter();
-//
-//                boolean loc = desc.locallyConfigured();
-//
-//                if (loc || (desc.receivedOnDiscovery() && CU.affinityNode(locNode, filter))) {
-//                    boolean started = desc.onStart();
-//
-//                    assert started : "Failed to change started flag for locally configured cache: " + desc;
-//
-//                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
-//
-//                    CachePluginManager pluginMgr = desc.pluginManager();
-//
-//                    GridCacheContext ctx = createCache(
-//                        ccfg, pluginMgr, desc.cacheType(), cacheObjCtx, desc.updatesAllowed());
-//
-//                    ctx.dynamicDeploymentId(desc.deploymentId());
-//
-//                    sharedCtx.addCacheContext(ctx);
-//
-//                    GridCacheAdapter cache = ctx.cache();
-//
-//                    String name = ccfg.getName();
-//
-//                    caches.put(name, cache);
-//
-//                    startCache(cache, desc.schema());
-//
-//                    jCacheProxies.put(name, new IgniteCacheProxy(ctx, cache, null, false));
-//                }
-//            }
+            // Must call onKernalStart on shared managers after creation of fetched caches.
+            for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers()) {
+                if (sharedCtx.database() != mgr)
+                    mgr.onKernalStart(false);
+            }
         }
         finally {
             cacheStartedLatch.countDown();
         }
 
-        // Must call onKernalStart on shared managers after creation of fetched caches.
-        for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers()) {
-            if (sharedCtx.database() != mgr)
-                mgr.onKernalStart(false);
-        }
-
         // Escape if start active on start false
         if (!activeOnStart)
             return;
@@ -1221,7 +1092,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 }
             }
         }
-//
+// TODO
 //        if (clientReconnectReqs != null) {
 //            for (Map.Entry<UUID, DynamicCacheChangeBatch> e : clientReconnectReqs.entrySet())
 //                processClientReconnectData(e.getKey(), e.getValue());
@@ -1835,39 +1706,44 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param req Cache start request.
-     * @param topVer Topology version.
+     * @param nearCfg Near cache configuration.
+     * @param desc Cache descriptor.
+     * @param exchTopVer Current exchange version.
      * @throws IgniteCheckedException If failed.
      */
-    public void prepareCacheStart(DynamicCacheChangeRequest req, AffinityTopologyVersion topVer)
+    void prepareCacheStart(DynamicCacheChangeRequest req,
+        @Nullable NearCacheConfiguration nearCfg,
+        DynamicCacheDescriptor desc,
+        AffinityTopologyVersion exchTopVer)
         throws IgniteCheckedException {
         assert req.start() : req;
         assert req.cacheType() != null : req;
 
-        DynamicCacheDescriptor desc = cacheDescriptor(req.cacheName());
-
         prepareCacheStart(
             req.startCacheConfiguration(),
-            req.nearCacheConfiguration(),
+            nearCfg,
             req.cacheType(),
-            req.clientStartOnly(),
-            req.initiatingNodeId(),
             req.deploymentId(),
             desc.startTopologyVersion(),
-            topVer,
-            desc != null ? desc.schema() : null
+            exchTopVer,
+            desc.schema()
         );
     }
 
+    /**
+     * @param exchTopVer Current exchange version.
+     * @throws IgniteCheckedException If failed.
+     */
     public void startCachesOnLocalJoin(AffinityTopologyVersion exchTopVer) throws IgniteCheckedException {
-        List<DynamicCacheDescriptor> caches = cachesInfo.cachesToStartOnLocalJoin();
+        List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> caches = cachesInfo.cachesToStartOnLocalJoin();
+
+        for (T2<DynamicCacheDescriptor, NearCacheConfiguration> t : caches) {
+            DynamicCacheDescriptor desc = t.get1();
 
-        for (DynamicCacheDescriptor desc : caches) {
             prepareCacheStart(
                 desc.cacheConfiguration(),
-                null,
+                t.get2(),
                 desc.cacheType(),
-                false,
-                null,
                 desc.deploymentId(),
                 desc.startTopologyVersion(),
                 exchTopVer,
@@ -1894,8 +1770,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     desc.cacheConfiguration(),
                     null,
                     desc.cacheType(),
-                    false,
-                    null,
                     desc.deploymentId(),
                     desc.startTopologyVersion(),
                     exchTopVer,
@@ -1911,8 +1785,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param cfg Start configuration.
      * @param nearCfg Near configuration.
      * @param cacheType Cache type.
-     * @param clientStartOnly Client only start request.
-     * @param initiatingNodeId Initiating node ID.
      * @param deploymentId Deployment ID.
      * @param cacheStartTopVer Cache start topology version.
      * @param exchTopVer Current exchange version.
@@ -1923,51 +1795,37 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         CacheConfiguration cfg,
         NearCacheConfiguration nearCfg,
         CacheType cacheType,
-        boolean clientStartOnly,
-        UUID initiatingNodeId,
         IgniteUuid deploymentId,
         AffinityTopologyVersion cacheStartTopVer,
         AffinityTopologyVersion exchTopVer,
         @Nullable QuerySchema schema
     ) throws IgniteCheckedException {
-        CacheConfiguration ccfg = new CacheConfiguration(cfg);
-
-        IgnitePredicate nodeFilter = ccfg.getNodeFilter();
-
-        ClusterNode locNode = ctx.discovery().localNode();
+        assert !caches.containsKey(cfg.getName()) : cfg.getName();
 
-        boolean affNodeStart = !clientStartOnly && CU.affinityNode(locNode, nodeFilter);
-        boolean clientNodeStart = locNode.id().equals(initiatingNodeId);
+        CacheConfiguration ccfg = new CacheConfiguration(cfg);
 
-        assert !caches.containsKey(ccfg.getName()) : ccfg.getName();
+        if (nearCfg != null)
+            ccfg.setNearConfiguration(nearCfg);
 
-        if (affNodeStart || clientNodeStart || CU.isSystemCache(cfg.getName())) {
-            if (clientNodeStart && !affNodeStart) {
-                if (nearCfg != null)
-                    ccfg.setNearConfiguration(nearCfg);
-                else
-                    ccfg.setNearConfiguration(null);
-            }
+        CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-            CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+        GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx, true);
 
-            GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx, true);
+        cacheCtx.startTopologyVersion(exchTopVer);
 
-            cacheCtx.startTopologyVersion(exchTopVer);
-            cacheCtx.cacheStartTopologyVersion(cacheStartTopVer);
+        cacheCtx.cacheStartTopologyVersion(cacheStartTopVer);
 
-            cacheCtx.dynamicDeploymentId(deploymentId);
+        cacheCtx.dynamicDeploymentId(deploymentId);
 
-            GridCacheAdapter cache = cacheCtx.cache();
+        GridCacheAdapter cache = cacheCtx.cache();
 
-            sharedCtx.addCacheContext(cacheCtx);
+        sharedCtx.addCacheContext(cacheCtx);
 
-            caches.put(cacheCtx.name(), cache);
+        caches.put(cacheCtx.name(), cache);
 
-            startCache(cache, schema != null ? schema : new QuerySchema());
+        startCache(cache, schema != null ? schema : new QuerySchema());
 
-            onKernalStart(cache);
-        }
+        onKernalStart(cache);
     }
 
     /**
@@ -2079,9 +1937,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
     }
 
-    void completeTemplateAddFuture(String name, IgniteUuid deploymentId) {
+    /**
+     * @param cacheName Cache name.
+     * @param deploymentId
+     */
+    void completeTemplateAddFuture(String cacheName, IgniteUuid deploymentId) {
         GridCacheProcessor.TemplateConfigurationFuture fut =
-            (GridCacheProcessor.TemplateConfigurationFuture)pendingTemplateFuts.get(name);
+            (GridCacheProcessor.TemplateConfigurationFuture)pendingTemplateFuts.get(cacheName);
 
         if (fut != null && fut.deploymentId().equals(deploymentId))
             fut.onDone();
@@ -2089,8 +1951,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param req Request to complete future for.
+     * @param err Error if any.
      */
-    public void completeCacheStartFuture(DynamicCacheChangeRequest req, Exception err) {
+    public void completeCacheStartFuture(DynamicCacheChangeRequest req, @Nullable Exception err) {
         if (req.initiatingNodeId().equals(ctx.localNodeId())) {
             DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 
@@ -2155,13 +2018,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
-        dataBag.addJoiningNodeData(CACHE_PROC.ordinal(), cachesInfo.joinDiscoveryData());
+        cachesInfo.collectJoiningNodeData(dataBag);
     }
 
     /** {@inheritDoc} */
     @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
-        if (!dataBag.commonDataCollectedFor(CACHE_PROC.ordinal()))
-            dataBag.addGridCommonData(CACHE_PROC.ordinal(), cachesInfo.collectCommonDiscoveryData());
+        cachesInfo.collectGridNodeData(dataBag);
     }
 
     /** {@inheritDoc} */
@@ -2474,7 +2336,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param cacheName Cache name to close.
      * @return Future that will be completed when cache is closed.
      */
-    public IgniteInternalFuture<?> dynamicCloseCache(String cacheName) {
+    IgniteInternalFuture<?> dynamicCloseCache(String cacheName) {
         assert cacheName != null;
 
         IgniteCacheProxy<?, ?> proxy = jCacheProxies.get(cacheName);
@@ -2734,18 +2596,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param req Cache change request.
-     */
-    private void initReceivedCacheConfiguration(DynamicCacheChangeRequest req) {
-        if (req.startCacheConfiguration() != null) {
-            CacheConfiguration ccfg = req.startCacheConfiguration();
-
-            if (ccfg.isStoreKeepBinary() == null)
-                ccfg.setStoreKeepBinary(CacheConfiguration.DFLT_STORE_KEEP_BINARY);
-        }
-    }
-
-    /**
      * Checks that preload-order-dependant caches has SYNC or ASYNC preloading mode.
      *
      * @param cfgs Caches.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
index 96df255..1ca4ac7 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
@@ -218,6 +218,10 @@ public class DiscoveryDataBag {
         return newJoinerData;
     }
 
+    void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+
+    }
+
     /**
      * @param cmpId component ID.
      * @param data Data.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java
new file mode 100644
index 0000000..da34424
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStartTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.Ignite;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteCacheStartTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** */
+    private CacheConfiguration ccfg;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        if (ccfg != null)
+            cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartAndNodeJoin() throws Exception {
+        Ignite node0 = startGrid(0);
+
+        checkCache(0, "c1", false);
+
+        node0.createCache(cacheConfiguration("c1"));
+
+        checkCache(0, "c1", true);
+
+        startGrid(1);
+
+        checkCache(0, "c1", true);
+        checkCache(1, "c1", true);
+
+        client = true;
+
+        startGrid(2);
+
+        checkCache(0, "c1", true);
+        checkCache(1, "c1", true);
+        checkCache(2, "c1", false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartFromJoiningNode1() throws Exception {
+        checkStartFromJoiningNode(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartFromJoiningNode2() throws Exception {
+        checkStartFromJoiningNode(true);
+    }
+
+    /**
+     * @param joinClient {@code True} if client node joins.
+     * @throws Exception If failed.
+     */
+    private void checkStartFromJoiningNode(boolean joinClient) throws Exception {
+        startGrid(0);
+        startGrid(1);
+
+        client = true;
+
+        startGrid(2);
+
+        ccfg = cacheConfiguration("c1");
+        client = joinClient;
+
+        startGrid(3);
+
+        checkCache(0, "c1", true);
+        checkCache(1, "c1", true);
+        checkCache(2, "c1", false);
+        checkCache(3, "c1", true);
+
+        client = false;
+        ccfg = null;
+
+        startGrid(4);
+
+        checkCache(0, "c1", true);
+        checkCache(1, "c1", true);
+        checkCache(2, "c1", false);
+        checkCache(3, "c1", true);
+        checkCache(4, "c1", true);
+
+        client = true;
+
+        startGrid(5);
+
+        checkCache(0, "c1", true);
+        checkCache(1, "c1", true);
+        checkCache(2, "c1", false);
+        checkCache(3, "c1", true);
+        checkCache(4, "c1", true);
+        checkCache(5, "c1", false);
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration(String cacheName) {
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName(cacheName);
+
+        return ccfg;
+    }
+
+    /**
+     * @param idx Node index.
+     * @param cacheName Cache name.
+     * @param expCache {@code True} if cache should be created.
+     */
+    private void checkCache(int idx, String cacheName, boolean expCache) {
+        IgniteKernal node = (IgniteKernal)ignite(idx);
+
+        if (expCache)
+            assertNotNull(node.context().cache().cache(cacheName));
+        else
+            assertNull(node.context().cache().cache(cacheName));
+
+        assertNotNull(node.context().cache().cache(CU.UTILITY_CACHE_NAME));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/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 4a34a1d..e7c5ca5 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
@@ -1027,7 +1027,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
      * @param nearOnly Near only flag.
      * @throws Exception If failed.
      */
-    public void checkGetOrCreateNear(final boolean nearOnly) throws Exception {
+    private void checkGetOrCreateNear(final boolean nearOnly) throws Exception {
         try {
             final AtomicInteger cnt = new AtomicInteger(nodeCount());
             final AtomicReference<Throwable> err = new AtomicReference<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c66d0e0/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 8340cd7..72f13d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -64,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughS
 import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLoadRebalanceEvictionSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheReadThroughStoreCallTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheStartTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxLocalPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxNearPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxPeekModesTest;
@@ -209,6 +210,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(IgniteCacheTxPreloadNoWriteTest.class);
 
+        suite.addTestSuite(IgniteCacheStartTest.class);
         suite.addTestSuite(IgniteDynamicCacheStartSelfTest.class);
         suite.addTestSuite(IgniteDynamicCacheWithConfigStartSelfTest.class);
         suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class);


[09/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart


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

Branch: refs/heads/ignite-5075
Commit: ea022d0d5546b452ea84d5ea1a9fb7055ae99f2b
Parents: dfad843 c096059
Author: sboikov <sb...@gridgain.com>
Authored: Fri Apr 28 09:25:38 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Apr 28 09:25:38 2017 +0300

----------------------------------------------------------------------
 examples/config/example-memory-policies.xml     |  22 +-
 .../datagrid/MemoryPoliciesExample.java         |  16 +-
 .../benchmarks/jmh/tree/BPlusTreeBenchmark.java |  15 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   4 +-
 .../java/org/apache/ignite/MemoryMetrics.java   |  89 ++--
 .../org/apache/ignite/cache/CacheMetrics.java   |   8 -
 .../configuration/DataPageEvictionMode.java     |   4 +-
 .../configuration/MemoryConfiguration.java      | 117 +++++-
 .../MemoryPolicyConfiguration.java              |  88 +++-
 .../apache/ignite/internal/IgniteKernal.java    |  11 +-
 .../ignite/internal/mem/DirectMemory.java       |  55 ---
 .../internal/mem/DirectMemoryProvider.java      |  19 +-
 .../mem/file/MappedFileMemoryProvider.java      | 153 ++-----
 .../mem/unsafe/UnsafeMemoryProvider.java        |  69 ++--
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 408 +++++++++++++------
 .../cache/CacheClusterMetricsMXBeanImpl.java    |   5 -
 .../cache/CacheLocalMetricsMXBeanImpl.java      |   5 -
 .../processors/cache/CacheMetricsImpl.java      |   5 -
 .../processors/cache/CacheMetricsSnapshot.java  |  13 -
 .../cache/binary/BinaryMetadataHolder.java      |   7 +
 .../cache/binary/BinaryMetadataTransport.java   |  34 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  27 +-
 .../binary/ClientMetadataRequestFuture.java     |   5 +
 .../IgniteCacheDatabaseSharedManager.java       | 235 +++++++----
 .../cache/database/MemoryMetricsImpl.java       |  24 +-
 .../cache/database/MemoryMetricsMXBeanImpl.java | 108 +++++
 .../cache/database/MemoryMetricsSnapshot.java   |  85 ++++
 .../processors/cache/database/MemoryPolicy.java |   7 +-
 .../evict/FairFifoPageEvictionTracker.java      |   6 +-
 .../evict/PageAbstractEvictionTracker.java      |  85 +---
 .../evict/Random2LruPageEvictionTracker.java    |   6 +-
 .../evict/RandomLruPageEvictionTracker.java     |   6 +-
 .../processors/igfs/IgfsDataManager.java        |   2 +-
 .../platform/cache/PlatformCache.java           |   1 -
 .../utils/PlatformConfigurationUtils.java       |  18 +-
 .../service/GridServiceProcessor.java           |   9 +-
 .../ignite/internal/util/IgniteUtils.java       |  28 +-
 .../visor/node/VisorMemoryConfiguration.java    |   2 +-
 .../node/VisorMemoryPolicyConfiguration.java    |   2 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |   4 -
 .../ignite/mxbean/MemoryMetricsMXBean.java      |  94 +++--
 .../internal/ClusterNodeMetricsSelfTest.java    |   6 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  18 +-
 .../cache/CacheConfigurationLeakTest.java       |   2 +-
 .../CacheMemoryPolicyConfigurationTest.java     |  10 +-
 .../cache/MemoryPolicyConfigValidationTest.java | 121 +++++-
 .../MemoryPolicyInitializationTest.java         |  16 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |   2 +-
 .../paged/PageEvictionAbstractTest.java         |   4 +-
 .../TxPessimisticDeadlockDetectionTest.java     |   2 +-
 .../processors/database/BPlusTreeSelfTest.java  |  26 +-
 .../database/FreeListImplSelfTest.java          |  26 +-
 .../database/IgniteDbDynamicCacheSelfTest.java  |   2 +-
 .../database/MemoryMetricsSelfTest.java         |   5 +-
 .../database/MetadataStorageSelfTest.java       |  21 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   2 +-
 .../platform/PlatformCacheWriteMetricsTask.java |   5 -
 ...stributedPartitionQueryAbstractSelfTest.java |   5 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   7 +-
 .../index/DynamicIndexAbstractSelfTest.java     |   6 +-
 .../h2/database/InlineIndexHelperTest.java      |  40 +-
 .../cpp/core-test/config/cache-identity-32.xml  |   7 +-
 .../cpp/core-test/config/cache-query-32.xml     |   6 +-
 .../config/cache-query-continuous-32.xml        |   6 +-
 .../cpp/core-test/config/cache-store-32.xml     |   6 +-
 .../cpp/core-test/config/cache-test-32.xml      |   6 +-
 .../cpp/odbc-test/config/queries-test-32.xml    |   6 +-
 .../odbc-test/config/queries-test-noodbc-32.xml |   6 +-
 .../Cache/CacheConfigurationTest.cs             |   3 +-
 .../Cache/CacheMetricsTest.cs                   |   1 -
 .../Config/spring-test.xml                      |  14 +
 .../IgniteConfigurationSerializerTest.cs        |  17 +-
 .../IgniteConfigurationTest.cs                  |  32 +-
 .../Cache/Configuration/DataPageEvictionMode.cs |   4 +-
 .../Cache/Configuration/MemoryConfiguration.cs  |  28 +-
 .../Configuration/MemoryPolicyConfiguration.cs  |  34 +-
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |   8 -
 .../IgniteConfigurationSection.xsd              |  16 +-
 .../Impl/Cache/CacheMetricsImpl.cs              |   7 -
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |  44 ++
 pom.xml                                         |  32 +-
 81 files changed, 1587 insertions(+), 923 deletions(-)
----------------------------------------------------------------------



[14/26] ignite git commit: IGNITE-5120 Fixed LGPL build.

Posted by sb...@apache.org.
IGNITE-5120 Fixed LGPL build.


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

Branch: refs/heads/ignite-5075
Commit: 777078d377ba2f94b8f41086ecf0ad6056cdace5
Parents: 6ab579c
Author: Alexey Kuznetsov <ak...@gridgain.com>
Authored: Sat Apr 29 21:01:47 2017 +0700
Committer: Alexey Kuznetsov <ak...@gridgain.com>
Committed: Sat Apr 29 21:01:47 2017 +0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/777078d3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 722f5eb..5fd1005 100644
--- a/pom.xml
+++ b/pom.xml
@@ -198,9 +198,9 @@
         <profile>
             <id>lgpl</id>
             <modules>
+                <module>modules/hibernate-core</module>
                 <module>modules/hibernate-4.2</module>
                 <module>modules/hibernate-5.1</module>
-                <module>modules/hibernate-core</module>
                 <module>modules/geospatial</module>
                 <module>modules/schedule</module>
             </modules>


[17/26] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-5075-cacheStart


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

Branch: refs/heads/ignite-5075
Commit: e2cf06a5b62a07281bb8234e029a670fafb338c7
Parents: 54091d2 a04aa10
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 2 09:59:05 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 2 09:59:05 2017 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                   | 36 ++++++++++++++++++++++++++++++++
 examples/rest/http-rest-example.php |  4 ++--
 modules/hibernate-core/pom.xml      |  8 +++++++
 pom.xml                             |  2 +-
 4 files changed, 47 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[05/26] ignite git commit: cache discovery data refactoring

Posted by sb...@apache.org.
cache discovery data refactoring


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

Branch: refs/heads/ignite-5075
Commit: 0022f6b6ed8585ca3c56b7282e91a373fcf54117
Parents: b71d4ef
Author: sboikov <sb...@gridgain.com>
Authored: Thu Apr 27 12:40:57 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Apr 27 17:57:40 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       | 154 +++++++--------
 .../processors/cache/ClusterCachesInfo.java     |  88 ++++++---
 .../cache/DynamicCacheChangeBatch.java          |   6 +
 .../processors/cache/ExchangeActions.java       | 195 ++++++++++++++++---
 .../GridCachePartitionExchangeManager.java      |  38 +---
 .../processors/cache/GridCacheProcessor.java    |  76 +++-----
 .../GridDhtPartitionsExchangeFuture.java        | 115 +++--------
 .../cluster/GridClusterStateProcessor.java      |  22 +--
 8 files changed, 391 insertions(+), 303 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 0958208..d3749f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -323,36 +323,37 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      *
      * @param fut Exchange future.
      * @param crd Coordinator flag.
-     * @param reqs Cache change requests.
+     * @param exchActions Cache change requests.
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if client-only exchange is needed.
      */
     public boolean onCacheChangeRequest(final GridDhtPartitionsExchangeFuture fut,
         boolean crd,
-        Collection<DynamicCacheChangeRequest> reqs)
+        ExchangeActions exchActions)
         throws IgniteCheckedException {
-        assert !F.isEmpty(reqs) : fut;
+        assert exchActions != null && !exchActions.empty() : fut;
 
-        for (DynamicCacheChangeRequest req : reqs) {
+        for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
             Integer cacheId = CU.cacheId(req.cacheName());
 
-            if (req.stop()) {
-                DynamicCacheDescriptor desc = registeredCaches.remove(cacheId);
+            DynamicCacheDescriptor desc = registeredCaches.remove(cacheId);
 
-                assert desc != null : cacheId;
-            }
-            else if (req.start() && !req.clientStartOnly()) {
-                DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(),
-                    req.startCacheConfiguration(),
-                    req.cacheType(),
-                    false,
-                    req.deploymentId(),
-                    req.schema());
+            assert desc != null : cacheId;
+        }
 
-                DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc);
+        for (DynamicCacheChangeRequest req : exchActions.startRequests()) {
+            Integer cacheId = CU.cacheId(req.cacheName());
 
-                assert old == null : old;
-            }
+            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(),
+                req.startCacheConfiguration(),
+                req.cacheType(),
+                false,
+                req.deploymentId(),
+                req.schema());
+
+            DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc);
+
+            assert old == null : old;
         }
 
         boolean clientOnly = true;
@@ -367,90 +368,85 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             }
         });
 
-        Set<Integer> stoppedCaches = null;
-
-        for (DynamicCacheChangeRequest req : reqs) {
-            if (!(req.clientStartOnly() || req.close()))
-                clientOnly = false;
-
+        for (DynamicCacheChangeRequest req : exchActions.startRequests()) {
             Integer cacheId = CU.cacheId(req.cacheName());
 
-            if (req.start()) {
-                cctx.cache().prepareCacheStart(req, fut.topologyVersion());
+            cctx.cache().prepareCacheStart(req, fut.topologyVersion());
 
-                if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
-                    if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
-                        U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
-                }
+            if (fut.isCacheAdded(cacheId, fut.topologyVersion())) {
+                if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty())
+                    U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
+            }
 
-                if (!crd || !lateAffAssign) {
-                    GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+            if (!crd || !lateAffAssign) {
+                GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
 
-                    if (cacheCtx != null && !cacheCtx.isLocal()) {
-                        boolean clientCacheStarted =
-                            req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId());
+                if (cacheCtx != null && !cacheCtx.isLocal()) {
+                    boolean clientCacheStarted =
+                        req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId());
 
-                        if (clientCacheStarted)
-                            initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign);
-                        else if (!req.clientStartOnly()) {
-                            assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion());
+                    if (clientCacheStarted)
+                        initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign);
+                    else if (!req.clientStartOnly()) {
+                        assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion());
 
-                            GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache();
+                        GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache();
 
-                            assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion();
+                        assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion();
 
-                            List<List<ClusterNode>> assignment = aff.calculate(fut.topologyVersion(),
-                                fut.discoveryEvent(), fut.discoCache());
+                        List<List<ClusterNode>> assignment = aff.calculate(fut.topologyVersion(),
+                            fut.discoveryEvent(), fut.discoCache());
 
-                            aff.initialize(fut.topologyVersion(), assignment);
-                        }
+                        aff.initialize(fut.topologyVersion(), assignment);
                     }
                 }
-                else
-                    initStartedCacheOnCoordinator(fut, cacheId);
             }
-            else if (req.stop() || req.close()) {
-                cctx.cache().blockGateway(req);
-
-                if (crd) {
-                    boolean rmvCache = false;
+            else
+                initStartedCacheOnCoordinator(fut, cacheId);
+        }
 
-                    if (req.close() && req.initiatingNodeId().equals(cctx.localNodeId())) {
-                        GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+        for (DynamicCacheChangeRequest req : exchActions.closeRequests(cctx.localNodeId())) {
+            Integer cacheId = CU.cacheId(req.cacheName());
 
-                        rmvCache = cacheCtx != null && !cacheCtx.affinityNode();
-                    }
-                    else if (req.stop())
-                        rmvCache = true;
+            cctx.cache().blockGateway(req);
 
-                    if (rmvCache) {
-                        CacheHolder cache = caches.remove(cacheId);
+            if (crd) {
+                GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
 
-                        if (cache != null) {
-                            if (!req.stop()) {
-                                assert !cache.client();
+                assert cacheCtx != null : req;
 
-                                cache = CacheHolder2.create(cctx,
-                                    cctx.cache().cacheDescriptor(cacheId),
-                                    fut,
-                                    cache.affinity());
+                CacheHolder cache = caches.remove(cacheId);
 
-                                caches.put(cacheId, cache);
-                            }
-                            else {
-                                if (stoppedCaches == null)
-                                    stoppedCaches = new HashSet<>();
+                assert !cache.client();
 
-                                stoppedCaches.add(cache.cacheId());
+                cache = CacheHolder2.create(cctx,
+                    cctx.cache().cacheDescriptor(cacheId),
+                    fut,
+                    cache.affinity());
 
-                                cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class);
-                            }
-                        }
-                    }
-                }
+                caches.put(cacheId, cache);
             }
         }
 
+        Set<Integer> stoppedCaches = null;
+
+        for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
+            Integer cacheId = CU.cacheId(req.cacheName());
+
+            cctx.cache().blockGateway(req);
+
+            CacheHolder cache = caches.remove(cacheId);
+
+            assert cache != null : req;
+
+            if (stoppedCaches == null)
+                stoppedCaches = new HashSet<>();
+
+            stoppedCaches.add(cache.cacheId());
+
+            cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class);
+        }
+
         if (stoppedCaches != null) {
             boolean notify = false;
 
@@ -479,7 +475,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             }
         }
 
-        return clientOnly;
+        return exchActions.clientOnlyExchange();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 8824a48..cd2cd77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -17,15 +17,19 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheExistsException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 
@@ -90,7 +94,9 @@ class ClusterCachesInfo {
 
         boolean incMinorTopVer = false;
 
-        List<DynamicCacheDescriptor> added = null;
+        List<DynamicCacheDescriptor> addedDescs = new ArrayList<>();
+
+        final List<T2<DynamicCacheChangeRequest, AffinityTopologyVersion>> reqsToComplete = new ArrayList<>();
 
         for (DynamicCacheChangeRequest req : batch.requests()) {
             if (req.template()) {
@@ -114,10 +120,7 @@ class ClusterCachesInfo {
 
                     assert old == null;
 
-                    if (added == null)
-                        added = new ArrayList<>();
-
-                    added.add(templateDesc);
+                    addedDescs.add(templateDesc);
                 }
 
                 ctx.cache().completeTemplateAddFuture(ccfg.getName(), req.deploymentId());
@@ -129,6 +132,8 @@ class ClusterCachesInfo {
 
             boolean needExchange = false;
 
+            AffinityTopologyVersion waitTopVer = null;
+
             if (req.start()) {
                 if (desc == null) {
                     if (req.clientStartOnly()) {
@@ -164,12 +169,9 @@ class ClusterCachesInfo {
                             req.initiatingNodeId(),
                             req.nearCacheConfiguration() != null);
 
-                        if (added == null)
-                            added = new ArrayList<>();
+                        addedDescs.add(startDesc);
 
-                        added.add(startDesc);
-
-                        exchangeActions.addCacheToStart(req, desc);
+                        exchangeActions.addCacheToStart(req, startDesc);
 
                         needExchange = true;
                     }
@@ -208,15 +210,23 @@ class ClusterCachesInfo {
                     }
                 }
 
-                if (!needExchange && desc != null) {
-                    if (desc.clientCacheStartVersion() != null)
-                        req.cacheFutureTopologyVersion(desc.clientCacheStartVersion());
-                    else
-                        req.cacheFutureTopologyVersion(desc.startTopologyVersion());
+                if (!needExchange) {
+                    if (desc != null) {
+                        if (desc.clientCacheStartVersion() != null)
+                            waitTopVer = desc.clientCacheStartVersion();
+                        else
+                            waitTopVer = desc.startTopologyVersion();
+                    }
                 }
             }
-            else if (req.globalStateChange() || req.resetLostPartitions())
+            else if (req.globalStateChange())
                 needExchange = true;
+            else if (req.resetLostPartitions()) {
+                needExchange = desc != null;
+
+                if (needExchange)
+                    exchangeActions.addCacheToResetLostPartitions(req, desc);
+            }
             else {
                 assert req.stop() ^ req.close() : req;
 
@@ -230,29 +240,63 @@ class ClusterCachesInfo {
 
                         needExchange = true;
 
-                        exchangeActions.addCacheToStop(req);
+                        exchangeActions.addCacheToStop(req, desc);
                     }
                     else {
                         assert req.close() : req;
 
                         needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId());
 
-                        if (needExchange)
-                            exchangeActions.addCacheToStop(req);
+                        if (needExchange) {
+                            exchangeActions.addCacheToStop(req, desc);
+
+                            exchangeActions.addCacheToClose(req, desc);
+                        }
                     }
                 }
             }
 
-            incMinorTopVer |= needExchange;
+            if (!needExchange) {
+                if (req.initiatingNodeId().equals(ctx.localNodeId()))
+                    reqsToComplete.add(new T2<>(req, waitTopVer));
+            }
+            else
+                incMinorTopVer = true;
         }
 
-        if (added != null) {
+        if (!F.isEmpty(addedDescs)) {
             AffinityTopologyVersion startTopVer = incMinorTopVer ? topVer.nextMinorVersion() : topVer;
 
-            for (DynamicCacheDescriptor desc : added)
+            for (DynamicCacheDescriptor desc : addedDescs)
                 desc.startTopologyVersion(startTopVer);
         }
 
+        if (!F.isEmpty(reqsToComplete)) {
+            ctx.closure().callLocalSafe(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    for (T2<DynamicCacheChangeRequest, AffinityTopologyVersion> t :reqsToComplete) {
+                        final DynamicCacheChangeRequest req = t.get1();
+                        AffinityTopologyVersion waitTopVer = t.get2();
+
+                        IgniteInternalFuture<?> fut = waitTopVer != null ?
+                            ctx.cache().context().exchange().affinityReadyFuture(waitTopVer) : null;
+
+                        if (fut == null || fut.isDone())
+                            ctx.cache().completeCacheStartFuture(req, null);
+                        else {
+                            fut.listen(new IgniteInClosure<IgniteInternalFuture<?>>() {
+                                @Override public void apply(IgniteInternalFuture<?> fut) {
+                                    ctx.cache().completeCacheStartFuture(req, null);
+                                }
+                            });
+                        }
+                    }
+
+                    return null;
+                }
+            });
+        }
+
         if (incMinorTopVer) {
             assert !exchangeActions.empty() : exchangeActions;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index e44bfcc..f69246e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -82,7 +82,13 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
         return exchangeActions != null && !exchangeActions.empty();
     }
 
+    ExchangeActions exchangeActions() {
+        return exchangeActions;
+    }
+
     void exchangeActions(ExchangeActions exchangeActions) {
+        assert !exchangeActions.empty() : exchangeActions;
+
         this.exchangeActions = exchangeActions;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index 5adafc9..a7b62ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -17,6 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.internal.util.typedef.F;
 
 import java.util.ArrayList;
@@ -27,64 +34,190 @@ import java.util.List;
  */
 public class ExchangeActions {
     /** */
-    private List<ActionData<DynamicCacheDescriptor>> cachesToStart;
+    private Map<String, ActionData> cachesToStart;
 
     /** */
-    private List<ActionData<DynamicCacheDescriptor>> clientCachesToStart;
+    private Map<String, ActionData> clientCachesToStart;
 
     /** */
-    private List<ActionData<String>> cachesToStop;
+    private Map<String, ActionData> cachesToStop;
 
     /** */
-    private List<ActionData<String>> cachesToClose;
+    private Map<String, ActionData> cachesToClose;
 
-    void addCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
-        if (cachesToStart == null)
-            cachesToStart = new ArrayList<>();
+    /** */
+    private Map<String, ActionData> cachesToResetLostParts;
+
+    /** */
+    private ClusterState newState;
 
-        cachesToStart.add(new ActionData<>(req, desc));
+    public boolean clientOnlyExchange() {
+        return F.isEmpty(cachesToStart) &&
+            F.isEmpty(cachesToStop) &&
+            F.isEmpty(cachesToResetLostParts);
     }
 
-    void addClientCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
-        if (clientCachesToStart == null)
-            clientCachesToStart = new ArrayList<>();
+    public List<DynamicCacheChangeRequest> closeRequests(UUID nodeId) {
+        List<DynamicCacheChangeRequest> res = null;
+
+        if (cachesToClose != null) {
+            for (ActionData req : cachesToClose.values()) {
+                if (nodeId.equals(req.req.initiatingNodeId())) {
+                    if (res == null)
+                        res = new ArrayList<>(cachesToClose.size());
+
+                    res.add(req.req);
+                }
+            }
+        }
+
+        return res != null ? res : Collections.<DynamicCacheChangeRequest>emptyList();
+    }
+
+    public List<DynamicCacheChangeRequest> startRequests() {
+        List<DynamicCacheChangeRequest> res = null;
+
+        if (cachesToStart != null) {
+            res = new ArrayList<>(cachesToStart.size());
+
+            for (ActionData req : cachesToStart.values())
+                res.add(req.req);
+        }
 
-        clientCachesToStart.add(new ActionData<>(req, desc));
+        return res != null ? res : Collections.<DynamicCacheChangeRequest>emptyList();
     }
 
-    void addCacheToStop(DynamicCacheChangeRequest req) {
-        if (cachesToStop == null)
-            cachesToStop = new ArrayList<>();
+    public List<DynamicCacheChangeRequest> stopRequests() {
+        List<DynamicCacheChangeRequest> res = null;
+
+        if (cachesToStop != null) {
+            res = new ArrayList<>(cachesToStop.size());
+
+            for (ActionData req : cachesToStop.values())
+                res.add(req.req);
+        }
 
-        cachesToStop.add(new ActionData<>(req, req.cacheName()));
+        return res != null ? res : Collections.<DynamicCacheChangeRequest>emptyList();
     }
 
-    void addCacheToClose(DynamicCacheChangeRequest req) {
-        if (cachesToClose == null)
-            cachesToClose = new ArrayList<>();
+    public void completeRequestFutures(GridCacheSharedContext ctx) {
+        completeRequestFutures(cachesToStart, ctx);
+        completeRequestFutures(clientCachesToStart, ctx);
+        completeRequestFutures(cachesToStop, ctx);
+        completeRequestFutures(cachesToClose, ctx);
+        completeRequestFutures(cachesToResetLostParts, ctx);
+    }
 
-        cachesToClose.add(new ActionData<>(req, req.cacheName()));
+    private void completeRequestFutures(Map<String, ActionData> map, GridCacheSharedContext ctx) {
+        if (map != null) {
+            for (ActionData req : map.values())
+                ctx.cache().completeCacheStartFuture(req.req, null);
+        }
     }
 
-    boolean empty() {
-        return F.isEmpty(cachesToStart) &&
-            F.isEmpty(clientCachesToStart) &&
-            F.isEmpty(cachesToStop) &&
-            F.isEmpty(cachesToClose);
+    public boolean hasStop() {
+        return !F.isEmpty(cachesToStop);
+    }
+
+    public Set<String> cachesToResetLostPartitions() {
+        Set<String> caches = null;
+        
+        if (cachesToResetLostParts != null)
+            caches = new HashSet<>(cachesToResetLostParts.keySet());
+
+        return caches != null ? caches : Collections.<String>emptySet();
+    }
+    
+    public boolean cacheStopped(int cacheId) {
+        if (cachesToStop != null) {
+            for (ActionData cache : cachesToStop.values()) {
+                if (cache.desc.cacheId() == cacheId)
+                    return true;
+            }
+        }
+
+        return false;
     }
 
-    void addFutureToComplete() {
+    public boolean cacheStarted(int cacheId) {
+        if (cachesToStart != null) {
+            for (ActionData cache : cachesToStart.values()) {
+                if (cache.desc.cacheId() == cacheId)
+                    return true;
+            }
+        }
+
+        return false;
+    }
+
+    public boolean clientCacheStarted(UUID nodeId) {
+        if (clientCachesToStart != null) {
+            for (ActionData cache : clientCachesToStart.values()) {
+                if (nodeId.equals(cache.req.initiatingNodeId()))
+                    return true;
+            }
+        }
+
+        return false;
+    }
+
+    public ClusterState newClusterState() {
+        return newState;
+    }
+
+    private Map<String, ActionData> add(Map<String, ActionData> map, DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        assert req != null;
+        assert desc != null;
+
+        if (map == null)
+            map = new HashMap<>();
+
+        ActionData old = map.put(req.cacheName(), new ActionData(req, desc));
+
+        assert old == null : old;
+
+        return map;
+    }
 
+    void addCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        cachesToStart = add(cachesToStart, req, desc);
+    }
+
+    void addClientCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        clientCachesToStart = add(clientCachesToStart, req, desc);
+    }
+
+    void addCacheToStop(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        cachesToStop = add(cachesToStop, req, desc);
+    }
+
+    void addCacheToClose(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        cachesToClose = add(cachesToClose, req, desc);
+    }
+
+    void addCacheToResetLostPartitions(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        cachesToResetLostParts = add(cachesToResetLostParts, req, desc);
+    }
+
+    public boolean empty() {
+        return F.isEmpty(cachesToStart) &&
+            F.isEmpty(clientCachesToStart) &&
+            F.isEmpty(cachesToStop) &&
+            F.isEmpty(cachesToClose) &&
+            F.isEmpty(cachesToResetLostParts);
     }
 
-    static class ActionData<T> {
-        DynamicCacheChangeRequest req;
+    /**
+     *
+     */
+    static class ActionData {
+        private DynamicCacheChangeRequest req;
 
-        T data;
+        private DynamicCacheDescriptor desc;
 
-        public ActionData(DynamicCacheChangeRequest req, T data) {
+        public ActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
             this.req = req;
-            this.data = data;
+            this.desc = desc;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/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 4775ea1..c2b0e27 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
@@ -235,34 +235,12 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     if (customMsg instanceof DynamicCacheChangeBatch) {
                         DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)customMsg;
 
-                        Collection<DynamicCacheChangeRequest> valid = new ArrayList<>(batch.requests().size());
-
-                        // Validate requests to check if event should trigger partition exchange.
-                        for (final DynamicCacheChangeRequest req : batch.requests()) {
-                            if (req.exchangeNeeded())
-                                valid.add(req);
-                            else {
-                                IgniteInternalFuture<?> fut = null;
-
-                                if (req.cacheFutureTopologyVersion() != null)
-                                    fut = affinityReadyFuture(req.cacheFutureTopologyVersion());
-
-                                if (fut == null || fut.isDone())
-                                    cctx.cache().completeStartFuture(req);
-                                else {
-                                    fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                                        @Override public void apply(IgniteInternalFuture<?> fut) {
-                                            cctx.cache().completeStartFuture(req);
-                                        }
-                                    });
-                                }
-                            }
-                        }
+                        ExchangeActions exchActions = batch.exchangeActions();
 
-                        if (!F.isEmpty(valid) && !(valid.size() == 1 && valid.iterator().next().globalStateChange())) {
+                        if (exchActions != null) {
                             exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
 
-                            exchFut = exchangeFuture(exchId, evt, cache, valid, null);
+                            exchFut = exchangeFuture(exchId, evt, cache, exchActions, null);
                         }
                     }
                     else if (customMsg instanceof CacheAffinityChangeMessage) {
@@ -1123,25 +1101,25 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @param exchId Exchange ID.
      * @param discoEvt Discovery event.
      * @param cache Discovery data cache.
-     * @param reqs Cache change requests.
+     * @param exchActions Cache change actions.
      * @param affChangeMsg Affinity change message.
      * @return Exchange future.
      */
     private GridDhtPartitionsExchangeFuture exchangeFuture(GridDhtPartitionExchangeId exchId,
         @Nullable DiscoveryEvent discoEvt,
         @Nullable DiscoCache cache,
-        @Nullable Collection<DynamicCacheChangeRequest> reqs,
+        @Nullable ExchangeActions exchActions,
         @Nullable CacheAffinityChangeMessage affChangeMsg) {
         GridDhtPartitionsExchangeFuture fut;
 
         GridDhtPartitionsExchangeFuture old = exchFuts.addx(
-            fut = new GridDhtPartitionsExchangeFuture(cctx, busyLock, exchId, reqs, affChangeMsg));
+            fut = new GridDhtPartitionsExchangeFuture(cctx, busyLock, exchId, exchActions, affChangeMsg));
 
         if (old != null) {
             fut = old;
 
-            if (reqs != null)
-                fut.cacheChangeRequests(reqs);
+            if (exchActions != null)
+                fut.exchangeActions(exchActions);
 
             if (affChangeMsg != null)
                 fut.affinityChangeMessage(affChangeMsg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/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 4f86dac..55690b2 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
@@ -2031,13 +2031,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Callback invoked when first exchange future for dynamic cache is completed.
      *
      * @param topVer Completed topology version.
-     * @param reqs Change requests.
+     * @param exchActions Change requests.
      * @param err Error.
      */
     @SuppressWarnings("unchecked")
     public void onExchangeDone(
         AffinityTopologyVersion topVer,
-        Collection<DynamicCacheChangeRequest> reqs,
+        ExchangeActions exchActions,
         Throwable err
     ) {
         for (GridCacheAdapter<?, ?> cache : caches.values()) {
@@ -2053,48 +2053,52 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
         }
 
-        if (!F.isEmpty(reqs) && err == null) {
-            for (DynamicCacheChangeRequest req : reqs) {
-                String masked = req.cacheName();
+        if (exchActions != null && err == null) {
+            for (DynamicCacheChangeRequest req : exchActions.stopRequests()) {
+                stopGateway(req);
 
-                if (req.stop()) {
-                    stopGateway(req);
+                prepareCacheStop(req);
+            }
 
-                    prepareCacheStop(req);
-                }
-                else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) {
-                    IgniteCacheProxy<?, ?> proxy = jCacheProxies.remove(masked);
+            for (DynamicCacheChangeRequest req : exchActions.closeRequests(ctx.localNodeId())) {
+                String cacheName = req.cacheName();
 
-                    if (proxy != null) {
-                        if (proxy.context().affinityNode()) {
-                            GridCacheAdapter<?, ?> cache = caches.get(masked);
+                IgniteCacheProxy<?, ?> proxy = jCacheProxies.remove(cacheName);
 
-                            if (cache != null)
-                                jCacheProxies.put(masked, new IgniteCacheProxy(cache.context(), cache, null, false));
-                        }
-                        else {
-                            proxy.context().gate().onStopped();
+                if (proxy != null) {
+                    if (proxy.context().affinityNode()) {
+                        GridCacheAdapter<?, ?> cache = caches.get(cacheName);
 
-                            prepareCacheStop(req);
-                        }
+                        if (cache != null)
+                            jCacheProxies.put(cacheName, new IgniteCacheProxy(cache.context(), cache, null, false));
+                    }
+                    else {
+                        proxy.context().gate().onStopped();
+
+                        prepareCacheStop(req);
                     }
                 }
             }
         }
     }
 
+    void completeTemplateAddFuture(String name, IgniteUuid deploymentId) {
+        GridCacheProcessor.TemplateConfigurationFuture fut =
+            (GridCacheProcessor.TemplateConfigurationFuture)pendingTemplateFuts.get(name);
+
+        if (fut != null && fut.deploymentId().equals(deploymentId))
+            fut.onDone();
+    }
+
     /**
      * @param req Request to complete future for.
      */
-    public void completeStartFuture(DynamicCacheChangeRequest req) {
+    public void completeCacheStartFuture(DynamicCacheChangeRequest req, Exception err) {
         if (req.initiatingNodeId().equals(ctx.localNodeId())) {
             DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 
-            assert req.deploymentId() != null || req.globalStateChange() || req.resetLostPartitions();
-            assert fut == null || req.globalStateChange() || req.resetLostPartitions();
-
             if (fut != null)
-                fut.onDone();
+                fut.onDone(null, err);
         }
     }
 
@@ -2506,8 +2510,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             DynamicCacheDescriptor desc = cacheDescriptor(cacheName);
 
             if (desc == null) {
-                log.warning("Reset lost partition will not be executed, " +
-                    "because cache with name:" + cacheName + " doesn't not exist");
+                U.warn(log, "Failed to find cache for reset lost partition request, cache does not exist: " + cacheName);
 
                 continue;
             }
@@ -2733,23 +2736,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         return false;
     }
 
-    void completeTemplateAddFuture(String name, IgniteUuid deploymentId) {
-        GridCacheProcessor.TemplateConfigurationFuture fut =
-            (GridCacheProcessor.TemplateConfigurationFuture)pendingTemplateFuts.get(name);
-
-        if (fut != null && fut.deploymentId().equals(deploymentId))
-            fut.onDone();
-    }
-
-    void completeCacheStartFuture(DynamicCacheChangeRequest req, Exception err) {
-        if (ctx.localNodeId().equals(req.initiatingNodeId())) {
-            DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
-
-            if (fut != null)
-                fut.onDone(err);
-        }
-    }
-
     /**
      * @param req Cache change request.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/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 3c056fd..320480c 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
@@ -61,6 +61,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
 import org.apache.ignite.internal.processors.cache.CachePartitionExchangeWorkerTask;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.cache.ExchangeActions;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
@@ -110,9 +111,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     public static final int DUMP_PENDING_OBJECTS_THRESHOLD =
         IgniteSystemProperties.getInteger(IgniteSystemProperties.IGNITE_DUMP_PENDING_OBJECTS_THRESHOLD, 10);
 
-    /** */
-    private static final long serialVersionUID = 0L;
-
     /** Dummy flag. */
     private final boolean dummy;
 
@@ -190,8 +188,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Dynamic cache change requests. */
-    private Collection<DynamicCacheChangeRequest> reqs;
+    /** Cache change requests. */
+    private ExchangeActions exchActions;
 
     /** */
     private CacheAffinityChangeMessage affChangeMsg;
@@ -284,19 +282,20 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @param cctx Cache context.
      * @param busyLock Busy lock.
      * @param exchId Exchange ID.
-     * @param reqs Cache change requests.
+     * @param exchActions Cache change requests.
      * @param affChangeMsg Affinity change message.
      */
     public GridDhtPartitionsExchangeFuture(
         GridCacheSharedContext cctx,
         ReadWriteLock busyLock,
         GridDhtPartitionExchangeId exchId,
-        Collection<DynamicCacheChangeRequest> reqs,
+        ExchangeActions exchActions,
         CacheAffinityChangeMessage affChangeMsg
     ) {
         assert busyLock != null;
         assert exchId != null;
         assert exchId.topologyVersion() != null;
+        assert exchActions == null || !exchActions.empty();
 
         dummy = false;
         forcePreload = false;
@@ -305,7 +304,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         this.cctx = cctx;
         this.busyLock = busyLock;
         this.exchId = exchId;
-        this.reqs = reqs;
+        this.exchActions = exchActions;
         this.affChangeMsg = affChangeMsg;
 
         log = cctx.logger(getClass());
@@ -317,10 +316,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
-     * @param reqs Cache change requests.
+     * @param exchActions Exchange actions.
      */
-    public void cacheChangeRequests(Collection<DynamicCacheChangeRequest> reqs) {
-        this.reqs = reqs;
+    public void exchangeActions(ExchangeActions exchActions) {
+        assert exchActions == null || !exchActions.empty();
+        assert evtLatch != null && evtLatch.getCount() == 1L : this;
+
+        this.exchActions = exchActions;
     }
 
     /**
@@ -396,16 +398,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @return {@code True} if non-client cache was added during this exchange.
      */
     public boolean cacheStarted(int cacheId) {
-        if (!F.isEmpty(reqs)) {
-            for (DynamicCacheChangeRequest req : reqs) {
-                if (req.start() && !req.clientStartOnly()) {
-                    if (CU.cacheId(req.cacheName()) == cacheId)
-                        return true;
-                }
-            }
-        }
+        return exchActions != null && exchActions.cacheStarted(cacheId);
 
-        return false;
     }
 
     /**
@@ -435,14 +429,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      *
      */
     public ClusterState newClusterState() {
-        if (!F.isEmpty(reqs)) {
-            for (DynamicCacheChangeRequest req : reqs) {
-                if (req.globalStateChange())
-                    return req.state();
-            }
-        }
-
-        return null;
+        return exchActions != null ? exchActions.newClusterState() : null;
     }
 
     /**
@@ -524,7 +511,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)discoEvt).customMessage();
 
                 if (msg instanceof DynamicCacheChangeBatch){
-                    assert !F.isEmpty(reqs);
+                    assert exchActions != null && !exchActions.empty();
 
                     exchange = onCacheChangeRequest(crdNode);
                 }
@@ -558,20 +545,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
             updateTopologies(crdNode);
 
-            if (!F.isEmpty(reqs)) {
-                boolean hasStop = false;
-
-                for (DynamicCacheChangeRequest req : reqs) {
-                    if (req.stop()) {
-                        hasStop = true;
-
-                        break;
-                    }
-                }
-
-                if (hasStop)
-                    cctx.cache().context().database().beforeCachesStop();
-            }
+            if (exchActions != null && exchActions.hasStop())
+                cctx.cache().context().database().beforeCachesStop();
 
             switch (exchange) {
                 case ALL: {
@@ -679,32 +654,21 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @throws IgniteCheckedException If failed.
      */
     private ExchangeType onCacheChangeRequest(boolean crd) throws IgniteCheckedException {
-        assert !F.isEmpty(reqs) : this;
+        assert exchActions != null && !exchActions.empty() : this;
 
         GridClusterStateProcessor stateProc = cctx.kernalContext().state();
 
-        if (exchangeOnChangeGlobalState = stateProc.changeGlobalState(reqs, topologyVersion())) {
+        if (exchangeOnChangeGlobalState = stateProc.changeGlobalState(exchActions, topologyVersion())) {
             changeGlobalStateE = stateProc.onChangeGlobalState();
 
             if (crd && changeGlobalStateE != null)
                 changeGlobalStateExceptions.put(cctx.localNodeId(), changeGlobalStateE);
         }
 
-        boolean clientOnly = cctx.affinity().onCacheChangeRequest(this, crd, reqs);
+        boolean clientOnly = cctx.affinity().onCacheChangeRequest(this, crd, exchActions);
 
-        if (clientOnly) {
-            boolean clientCacheStarted = false;
-
-            for (DynamicCacheChangeRequest req : reqs) {
-                if (req.start() && req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId())) {
-                    clientCacheStarted = true;
-
-                    break;
-                }
-            }
-
-            return clientCacheStarted ? ExchangeType.CLIENT : ExchangeType.NONE;
-        }
+        if (clientOnly)
+            return exchActions.clientCacheStarted(cctx.localNodeId()) ? ExchangeType.CLIENT : ExchangeType.NONE;
         else
             return cctx.kernalContext().clientNode() ? ExchangeType.CLIENT : ExchangeType.ALL;
     }
@@ -1051,19 +1015,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @return {@code True} if cache is stopping by this exchange.
      */
     public boolean stopping(int cacheId) {
-        boolean stopping = false;
-
-        if (!F.isEmpty(reqs)) {
-            for (DynamicCacheChangeRequest req : reqs) {
-                if (cacheId == CU.cacheId(req.cacheName())) {
-                    stopping = req.stop();
-
-                    break;
-                }
-            }
-        }
-
-        return stopping;
+        return exchActions != null && exchActions.cacheStopped(cacheId);
     }
 
     /**
@@ -1074,13 +1026,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         assert node != null;
 
         // Reset lost partition before send local partition to coordinator.
-        if (!F.isEmpty(reqs)) {
-            Set<String> caches = new HashSet<>();
-
-            for (DynamicCacheChangeRequest req : reqs) {
-                if (req.resetLostPartitions())
-                    caches.add(req.cacheName());
-            }
+        if (exchActions != null) {
+            Set<String> caches = exchActions.cachesToResetLostPartitions();
 
             if (!F.isEmpty(caches))
                 resetLostPartitions(caches);
@@ -1208,14 +1155,12 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             cacheValidRes = m;
         }
 
-        cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err);
+        cctx.cache().onExchangeDone(exchId.topologyVersion(), exchActions, err);
 
         cctx.exchange().onExchangeDone(this, err);
 
-        if (!F.isEmpty(reqs) && err == null) {
-            for (DynamicCacheChangeRequest req : reqs)
-                cctx.cache().completeStartFuture(req);
-        }
+        if (exchActions != null && err == null)
+            exchActions.completeRequestFutures(cctx);
 
         if (exchangeOnChangeGlobalState && err == null)
             cctx.kernalContext().state().onExchangeDone();
@@ -1232,7 +1177,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     cacheCtx.config().getAffinity().removeNode(exchId.nodeId());
             }
 
-            reqs = null;
+            exchActions = null;
 
             if (discoEvt instanceof DiscoveryCustomEvent)
                 ((DiscoveryCustomEvent)discoEvt).customMessage(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0022f6b6/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
index 1286ba9..a20ee41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
@@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.cache.ChangeGlobalStateMessage;
 import org.apache.ignite.internal.processors.cache.ClusterState;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest;
+import org.apache.ignite.internal.processors.cache.ExchangeActions;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridChangeGlobalStateMessageResponse;
@@ -329,26 +330,25 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param reqs Requests.
+     * @param exchActions Requests.
+     * @param topVer Exchange topology version.
      */
     public boolean changeGlobalState(
-        Collection<DynamicCacheChangeRequest> reqs,
+        ExchangeActions exchActions,
         AffinityTopologyVersion topVer
     ) {
-        assert !F.isEmpty(reqs);
+        assert exchActions != null;
         assert topVer != null;
 
-        for (DynamicCacheChangeRequest req : reqs)
-            if (req.globalStateChange()) {
-                ChangeGlobalStateContext cgsCtx = lastCgsCtx;
+        if (exchActions.newClusterState() != null) {
+            ChangeGlobalStateContext cgsCtx = lastCgsCtx;
 
-                assert cgsCtx != null : "reqs: " + Arrays.toString(reqs.toArray());
+            assert cgsCtx != null : exchActions;
 
-                cgsCtx.topologyVersion(topVer);
-
-                return true;
-            }
+            cgsCtx.topologyVersion(topVer);
 
+            return true;
+        }
 
         return false;
     }