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:27 UTC

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

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.
      */