Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 7D02D200C72 for ; Fri, 12 May 2017 09:37:43 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 7BC30160BA3; Fri, 12 May 2017 07:37:43 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id D411A160BCF for ; Fri, 12 May 2017 09:37:41 +0200 (CEST) Received: (qmail 60894 invoked by uid 500); 12 May 2017 07:37:40 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 60576 invoked by uid 99); 12 May 2017 07:37:39 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 12 May 2017 07:37:39 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2681EEE68E; Fri, 12 May 2017 07:37:39 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Fri, 12 May 2017 07:37:47 -0000 Message-Id: In-Reply-To: <15f05078982f48a1a78b27c396b5bd38@git.apache.org> References: <15f05078982f48a1a78b27c396b5bd38@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [10/11] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-5075-cacheStart' into ignite-5075 archived-at: Fri, 12 May 2017 07:37:43 -0000 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/GridCacheAdapter.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/63655a70 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/63655a70 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/63655a70 Branch: refs/heads/ignite-5075 Commit: 63655a707b2f9d0b7693116624420c507fc6b4a4 Parents: 19a4c4d 3762b0d Author: sboikov Authored: Fri May 12 10:36:42 2017 +0300 Committer: sboikov Committed: Fri May 12 10:36:42 2017 +0300 ---------------------------------------------------------------------- .../cache/CacheAffinitySharedManager.java | 7 +- .../internal/processors/cache/CacheData.java | 1 - .../cache/DynamicCacheDescriptor.java | 2 - .../processors/cache/ExchangeActions.java | 2 +- .../processors/cache/GridCacheAdapter.java | 131 ++++-- .../cache/GridCacheConcurrentMap.java | 2 - .../cache/GridCacheConcurrentMapImpl.java | 5 +- .../processors/cache/GridCacheContext.java | 24 ++ .../processors/cache/GridCacheEventManager.java | 32 ++ .../processors/cache/GridCacheMapEntry.java | 14 +- .../cache/GridCacheMapEntryFactory.java | 6 +- .../processors/cache/GridCacheProcessor.java | 46 ++- .../processors/cache/GridNoStorageCacheMap.java | 9 +- .../cache/IgniteCacheOffheapManager.java | 7 + .../cache/IgniteCacheOffheapManagerImpl.java | 17 + .../distributed/GridDistributedCacheEntry.java | 8 +- .../dht/GridCachePartitionedConcurrentMap.java | 11 +- .../dht/GridClientPartitionTopology.java | 2 +- .../distributed/dht/GridDhtCacheAdapter.java | 8 +- .../distributed/dht/GridDhtCacheEntry.java | 8 +- .../distributed/dht/GridDhtLocalPartition.java | 9 +- .../dht/GridDhtPartitionTopologyImpl.java | 5 +- .../dht/GridPartitionedGetFuture.java | 158 +++++--- .../dht/GridPartitionedSingleGetFuture.java | 141 ++++--- .../dht/atomic/GridDhtAtomicCache.java | 226 ++++++----- .../dht/atomic/GridDhtAtomicCacheEntry.java | 11 +- .../dht/colocated/GridDhtColocatedCache.java | 222 ++++++----- .../colocated/GridDhtColocatedCacheEntry.java | 11 +- .../colocated/GridDhtDetachedCacheEntry.java | 10 +- .../GridDhtPartitionsExchangeFuture.java | 16 +- .../distributed/near/GridNearCacheAdapter.java | 6 +- .../distributed/near/GridNearCacheEntry.java | 8 +- .../cache/distributed/near/GridNearTxLocal.java | 7 +- .../processors/cache/local/GridLocalCache.java | 6 +- .../cache/local/GridLocalCacheEntry.java | 8 +- .../local/atomic/GridLocalAtomicCache.java | 188 +++++---- .../GridCacheAtomicSequenceImpl.java | 101 +---- .../ignite/spi/discovery/DiscoveryDataBag.java | 2 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 2 + .../internal/GridNodeMetricsLogSelfTest.java | 13 +- .../cache/IgniteCacheNoSyncForGetTest.java | 395 +++++++++++++++++++ .../IgniteCrossCacheTxStoreSelfTest.java | 1 - .../IgniteCacheExpiryPolicyAbstractTest.java | 2 +- .../loadtests/hashmap/GridCacheTestContext.java | 2 + .../loadtests/hashmap/GridHashMapLoadTest.java | 4 +- .../testsuites/IgniteCacheTestSuite2.java | 3 + .../query/h2/twostep/GridMergeIndexSorted.java | 3 + .../query/IgniteSqlSplitterSelfTest.java | 68 ++++ .../IgniteStartStopTest.cs | 2 + .../dotnet/Apache.Ignite.Core/Ignition.cs | 16 +- .../cache/IgniteGetFromComputeBenchmark.java | 167 ++++++++ 51 files changed, 1498 insertions(+), 657 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/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 90e6c8e,d186cdb..28e2d12 --- 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 @@@ -320,19 -320,32 +320,19 @@@ public class CacheAffinitySharedManager } /** - * @param exchActions Cache change requests to execte on exchange. + * @param exchActions Cache change requests to execute 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 = registeredGrps.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()); - - DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(), - req.startCacheConfiguration(), - req.cacheType(), - false, - req.deploymentId(), - req.schema()); + for (CacheGroupDescriptor startDesc : exchActions.cacheGroupsToStart()) { + CacheGroupDescriptor old = registeredGrps.put(startDesc.groupId(), startDesc); - DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc); - - assert old == null : old; - } + assert old == null : old; } } @@@ -386,66 -399,56 +386,66 @@@ if (startCache) cctx.cache().prepareCacheStart(cacheDesc, nearCfg, fut.topologyVersion()); - if (fut.isCacheAdded(cacheDesc.cacheId(), fut.topologyVersion())) { + if (fut.cacheAddedOnExchange(cacheDesc.cacheId(), cacheDesc.receivedFrom())) { - if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty()) + if (fut.discoCache().cacheGroupAffinityNodes(cacheDesc.groupDescriptor().groupId()).isEmpty()) U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName()); } + } - if (!crd || !lateAffAssign) { - GridCacheContext cacheCtx = cctx.cacheContext(cacheDesc.cacheId()); + for (CacheGroupInfrastructure grp : cctx.cache().cacheGroups()) { + if (grp.affinity().lastVersion().equals(AffinityTopologyVersion.NONE)) + initAffinity(registeredGrps.get(grp.groupId()), grp.affinity(), fut, lateAffAssign); + } - if (cacheCtx != null && !cacheCtx.isLocal()) { - boolean clientCacheStarted = - req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId()); + if (crd) { + for (CacheGroupDescriptor grpDesc : exchActions.cacheGroupsToStart()) + initStartedGroupOnCoordinator(fut, grpDesc); + } - if (clientCacheStarted) - initAffinity(cacheDesc, cacheCtx.affinity().affinityCache(), fut, lateAffAssign); - else if (!req.clientStartOnly()) { - GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache(); + List closeReqs = exchActions.closeRequests(cctx.localNodeId()); - assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion(); + for (ExchangeActions.ActionData req : closeReqs) { + cctx.cache().blockGateway(req.request()); - List> assignment = aff.calculate(fut.topologyVersion(), - fut.discoveryEvent(), fut.discoCache()); + if (crd) { + CacheGroupInfrastructure grp = cctx.cache().cacheGroup(req.descriptor().groupDescriptor().groupId()); - aff.initialize(fut.topologyVersion(), assignment); - } - } - } - else - initStartedCacheOnCoordinator(fut, cacheDesc.cacheId()); - } + assert grp != null; - for (DynamicCacheChangeRequest req : exchActions.closeRequests(cctx.localNodeId())) { - Integer cacheId = CU.cacheId(req.cacheName()); + if (grp.affinityNode()) + continue; - cctx.cache().blockGateway(req); + boolean grpClosed = false; - if (crd) { - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + if (grp.sharedGroup()) { + boolean cacheRemaining = false; + + for (GridCacheContext ctx : cctx.cacheContexts()) { + if (ctx.group() == grp && !cacheClosed(ctx.cacheId(), closeReqs)) { + cacheRemaining = true; + + break; + } + } + + if (!cacheRemaining) + grpClosed = true; + } + else + grpClosed = true; - // Client cache was stopped, need create 'client' CacheHolder. - if (cacheCtx != null && !cacheCtx.affinityNode()) { - CacheHolder cache = caches.remove(cacheId); + // All client caches were stopped, need create 'client' CacheGroupHolder. + if (grpClosed) { + CacheGroupHolder grpHolder = grpHolders.remove(grp.groupId()); - assert !cache.client() : cache; + assert !grpHolder.client() : grpHolder; - cache = CacheHolder2.create(cctx, - cctx.cache().cacheDescriptor(cacheId), + grpHolder = CacheGroupHolder2.create(cctx, + registeredGrps.get(grp.groupId()), fut, - cache.affinity()); + grp.affinity()); - caches.put(cacheId, cache); + grpHolders.put(grp.groupId(), grpHolder); } } } @@@ -888,15 -875,13 +888,14 @@@ return true; // If local node did not initiate exchange or local node is the only cache node in grid. - Collection affNodes = cctx.discovery().cacheAffinityNodes(aff.cacheId(), fut.topologyVersion()); + Collection affNodes = + cctx.discovery().cacheGroupAffinityNodes(aff.groupId(), fut.topologyVersion()); - DynamicCacheDescriptor cacheDesc = registeredCaches.get(aff.cacheId()); + CacheGroupDescriptor grpDesc = registeredGrps.get(aff.groupId()); - assert cacheDesc != null : aff.cacheName(); + assert grpDesc != null : aff.groupName(); - return fut.cacheGroupStarting(aff.groupId()) || - cctx.localNodeId().equals(grpDesc.receivedFrom()) || - return fut.cacheAddedOnExchange(aff.cacheId(), cacheDesc.receivedFrom()) || ++ return fut.cacheGroupAddedOnExchange(aff.groupId(), grpDesc.receivedFrom()) || !fut.exchangeId().nodeId().equals(cctx.localNodeId()) || (affNodes.size() == 1 && affNodes.contains(cctx.localNode())); } http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java index e1da75a,694f4b0..aa43bdd --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java @@@ -560,9 -560,9 +560,9 @@@ public abstract class GridCacheAdapter< * @throws IgniteCheckedException If start failed. */ public void start() throws IgniteCheckedException { - // TODO: IGNITE-5075: make abstract? - if (map == null) { - map = new GridCacheLocalConcurrentMap(ctx, entryFactory(), DFLT_START_CACHE_SIZE); - } ++ // TODO: IGNITE-5075: make start abstract? + if (map == null) + map = new GridCacheLocalConcurrentMap(entryFactory(), DFLT_START_CACHE_SIZE); } /** @@@ -944,7 -944,7 +944,7 @@@ * @return Entry (never {@code null}). */ public GridCacheEntryEx entryEx(KeyCacheObject key, AffinityTopologyVersion topVer) { - GridCacheEntryEx e = map.putEntryIfObsoleteOrAbsent(ctx, topVer, key, null, true, false); - GridCacheEntryEx e = map.putEntryIfObsoleteOrAbsent(topVer, key, true, false); ++ GridCacheEntryEx e = map.putEntryIfObsoleteOrAbsent(ctx, topVer, key, true, false); assert e != null; @@@ -964,10 -964,8 +964,9 @@@ if (cur == null || cur.obsolete()) { cur = map.putEntryIfObsoleteOrAbsent( + ctx, topVer, key, - null, create, touch); } http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java index e00b9f8,0fe5c9f..a7bad0b --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java @@@ -45,10 -44,8 +44,9 @@@ public interface GridCacheConcurrentMa * couldn't be created. */ @Nullable public GridCacheMapEntry putEntryIfObsoleteOrAbsent( + GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable CacheObject val, boolean create, boolean touch); http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java index d569005,2c262df..6f461fb --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java @@@ -98,11 -105,8 +98,10 @@@ public abstract class GridCacheConcurre } /** {@inheritDoc} */ - @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(final AffinityTopologyVersion topVer, + @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent( + GridCacheContext ctx, + final AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable final CacheObject val, final boolean create, final boolean touch) { GridCacheMapEntry cur = null; http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/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 d5128f5,aa503b1..58a3775 --- 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 @@@ -277,11 -291,14 +280,12 @@@ public class GridCacheContext imp GridKernalContext ctx, GridCacheSharedContext sharedCtx, CacheConfiguration cacheCfg, + CacheGroupInfrastructure grp, CacheType cacheType, AffinityTopologyVersion locStartTopVer, + UUID rcvdFrom, boolean affNode, boolean updatesAllowed, - MemoryPolicy memPlc, - FreeList freeList, - ReuseList reuseList, /* * Managers in starting order! @@@ -322,9 -338,9 +326,10 @@@ this.ctx = ctx; this.sharedCtx = sharedCtx; this.cacheCfg = cacheCfg; + this.grp = grp; this.cacheType = cacheType; this.locStartTopVer = locStartTopVer; + this.rcvdFrom = rcvdFrom; this.affNode = affNode; this.updatesAllowed = updatesAllowed; this.depEnabled = ctx.deploy().enabled() && !cacheObjects().isBinaryEnabled(cacheCfg); http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/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 a242649,bc6a7d6..5137ed1 --- 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 @@@ -1362,18 -1327,18 +1361,20 @@@ public class GridCacheProcessor extend /** * @param cfg Cache configuration to use to create cache. + * @param grp Cache group infrastructure. * @param pluginMgr Cache plugin manager. - * @param cacheType Cache type. + * @param desc Cache descriptor. + * @param locStartTopVer Current topology version. * @param cacheObjCtx Cache object context. + * @param affNode {@code True} if local node affinity node. * @param updatesAllowed Updates allowed flag. * @return Cache context. * @throws IgniteCheckedException If failed to create cache. */ private GridCacheContext createCache(CacheConfiguration cfg, + CacheGroupInfrastructure grp, @Nullable CachePluginManager pluginMgr, - CacheType cacheType, + DynamicCacheDescriptor desc, AffinityTopologyVersion locStartTopVer, CacheObjectContext cacheObjCtx, boolean affNode, @@@ -1442,11 -1412,14 +1443,12 @@@ ctx, sharedCtx, cfg, + grp, - cacheType, + desc.cacheType(), locStartTopVer, + desc.receivedFrom(), affNode, updatesAllowed, - memPlc, - freeList, - reuseList, /* * Managers in starting order! @@@ -1572,11 -1545,14 +1574,12 @@@ ctx, sharedCtx, cfg, + grp, - cacheType, + desc.cacheType(), locStartTopVer, + desc.receivedFrom(), affNode, true, - memPlc, - freeList, - reuseList, /* * Managers in starting order! @@@ -1746,11 -1722,9 +1749,10 @@@ AffinityTopologyVersion exchTopVer) throws IgniteCheckedException { prepareCacheStart( + cacheDesc.groupDescriptor(), cacheDesc.cacheConfiguration(), nearCfg, - cacheDesc.cacheType(), - cacheDesc.deploymentId(), + cacheDesc, exchTopVer, cacheDesc.schema() ); @@@ -1768,11 -1742,9 +1770,10 @@@ DynamicCacheDescriptor desc = t.get1(); prepareCacheStart( + desc.groupDescriptor(), desc.cacheConfiguration(), t.get2(), - desc.cacheType(), - desc.deploymentId(), + desc, exchTopVer, desc.schema() ); @@@ -1798,11 -1770,9 +1799,10 @@@ if (CU.affinityNode(ctx.discovery().localNode(), filter)) { prepareCacheStart( + desc.groupDescriptor(), desc.cacheConfiguration(), null, - desc.cacheType(), - desc.deploymentId(), + desc, exchTopVer, desc.schema() ); @@@ -1823,11 -1792,9 +1822,10 @@@ * @throws IgniteCheckedException If failed. */ private void prepareCacheStart( + CacheGroupDescriptor grpDesc, CacheConfiguration startCfg, @Nullable NearCacheConfiguration reqNearCfg, - CacheType cacheType, - IgniteUuid deploymentId, + DynamicCacheDescriptor desc, AffinityTopologyVersion exchTopVer, @Nullable QuerySchema schema ) throws IgniteCheckedException { @@@ -1852,48 -1819,15 +1850,48 @@@ ccfg.setNearConfiguration(reqNearCfg); } + String grpName = startCfg.getGroupName(); + + CacheGroupInfrastructure grp = null; + + if (grpName != null) { + for (CacheGroupInfrastructure grp0 : cacheGrps.values()) { + if (grp0.sharedGroup() && grpName.equals(grp0.name())) { + grp = grp0; + + break; + } + } + + if (grp == null) { + grp = startCacheGroup(grpDesc, + cacheType, + affNode, + cacheObjCtx, + exchTopVer); + } + } + else { + grp = startCacheGroup(grpDesc, + cacheType, + affNode, + cacheObjCtx, + exchTopVer); + } + GridCacheContext cacheCtx = createCache(ccfg, + grp, null, - cacheType, + desc, exchTopVer, cacheObjCtx, affNode, true); + if (!grp.sharedGroup()) + grp.cacheContext(cacheCtx); + - cacheCtx.dynamicDeploymentId(deploymentId); + cacheCtx.dynamicDeploymentId(desc.deploymentId()); GridCacheAdapter cache = cacheCtx.cache(); http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java index 8faf52c,14a8482..90f628e --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java @@@ -22,7 -22,7 +22,6 @@@ import java.util.Collections import java.util.Set; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; --import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; import org.jetbrains.annotations.Nullable; /** @@@ -35,10 -45,12 +34,12 @@@ public class GridNoStorageCacheMap impl } /** {@inheritDoc} */ - @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable CacheObject val, boolean create, boolean touch) { - @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(AffinityTopologyVersion topVer, ++ @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(GridCacheContext ctx, AffinityTopologyVersion topVer, + KeyCacheObject key, + boolean create, + boolean touch) { if (create) - return new GridDhtCacheEntry(ctx, topVer, key, key.hashCode(), val); + return new GridDhtCacheEntry(ctx, topVer, key); else return null; } http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java index 1fa909b,f021b65..51594a9 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java @@@ -25,8 -25,7 +25,7 @@@ import java.util.NoSuchElementException import java.util.Set; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; -import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheGroupInfrastructure; - import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@@ -80,15 -79,17 +79,17 @@@ public class GridCachePartitionedConcur } /** {@inheritDoc} */ - @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable CacheObject val, boolean create, boolean touch) { - @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(AffinityTopologyVersion topVer, ++ @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(GridCacheContext ctx, AffinityTopologyVersion topVer, + KeyCacheObject key, + boolean create, + boolean touch) { while (true) { - GridDhtLocalPartition part = localPartition(key, topVer, create); + GridDhtLocalPartition part = localPartition(ctx, key, topVer, create); if (part == null) return null; - GridCacheMapEntry res = part.putEntryIfObsoleteOrAbsent(ctx, topVer, key, val, create, touch); - GridCacheMapEntry res = part.putEntryIfObsoleteOrAbsent(topVer, key, create, touch); ++ GridCacheMapEntry res = part.putEntryIfObsoleteOrAbsent(ctx, topVer, key, create, touch); if (res != null || !create) return res; http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java index 63df21b,4208a98..dc1ad31 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java @@@ -932,15 -899,10 +932,10 @@@ public class GridDhtLocalPartition exte try { CacheDataRow row = it0.next(); - GridCacheMapEntry cached = putEntryIfObsoleteOrAbsent(cctx.affinity().affinityTopologyVersion(), - row.key(), - true, - false); + if (grp.sharedGroup() && (cctx == null || cctx.cacheId() != row.cacheId())) - cctx = ctx.cacheContext(row.cacheId()); - - GridCacheMapEntry cached = putEntryIfObsoleteOrAbsent(cctx, ++ cctx = ctx.cacheContext(row.cacheId());GridCacheMapEntry cached = putEntryIfObsoleteOrAbsent(cctx, + grp.affinity().lastVersion(), - row.key(), - null, - true, - false); ++ row.key(), true, false); if (cached instanceof GridDhtCacheEntry && ((GridDhtCacheEntry)cached).clearInternal(clearVer, extras)) { if (rec) { http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java index 9168d7f,0410dda..c5582bc --- 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 @@@ -396,15 -388,14 +396,14 @@@ public class GridDhtPartitionTopologyIm ", futVer=" + exchFut.topologyVersion() + ", fut=" + exchFut + ']'; - List> aff = cctx.affinity().assignments(exchFut.topologyVersion()); + List> aff = grp.affinity().assignments(exchFut.topologyVersion()); - int num = cctx.affinity().partitions(); + int num = grp.affinity().partitions(); - if (cctx.rebalanceEnabled()) { - boolean added = exchFut.cacheAddedOnExchange(cctx.cacheId(), cctx.receivedFrom()); + if (grp.rebalanceEnabled()) { - boolean added = exchFut.cacheGroupStarting(grp.groupId()) || - (exchId.isJoined() && exchId.nodeId().equals(grp.receivedFrom())); ++ boolean added = exchFut.cacheGroupAddedOnExchange(grp.groupId(), grp.receivedFrom()); - boolean first = (loc.equals(oldest) && loc.id().equals(exchId.nodeId()) && exchId.isJoined()) || added; + boolean first = added || (loc.equals(oldest) && loc.id().equals(exchId.nodeId()) && exchId.isJoined()); if (first) { assert exchId.isJoined() || added; @@@ -550,7 -541,7 +549,7 @@@ cntrMap.clear(); // If this is the oldest node. - if (oldest != null && (loc.equals(oldest) || grp.localStartVersion().equals(exchId.topologyVersion()))) { - if (oldest != null && (loc.equals(oldest) || exchFut.cacheAddedOnExchange(cctx.cacheId(), cctx.receivedFrom()))) { ++ if (oldest != null && (loc.equals(oldest) || exchFut.cacheGroupAddedOnExchange(grp.groupId(), grp.receivedFrom())) { if (node2part == null) { node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq); http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index d15f069,b4cb3c1..8f0fce6 --- 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 @@@ -382,22 -380,13 +381,17 @@@ public class GridDhtPartitionsExchangeF /** * @param cacheId Cache ID to check. - * @param topVer Topology version. + * @param rcvdFrom Topology version. * @return {@code True} if cache was added during this exchange. */ - public boolean isCacheAdded(int cacheId, AffinityTopologyVersion topVer) { - if (cacheStarted(cacheId)) - return true; - - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - - return cacheCtx != null && F.eq(cacheCtx.startTopologyVersion(), topVer); + public boolean cacheAddedOnExchange(int cacheId, UUID rcvdFrom) { + return dynamicCacheStarted(cacheId) || (exchId.isJoined() && exchId.nodeId().equals(rcvdFrom)); } + public boolean cacheGroupStarting(int grpId) { + return exchActions != null && exchActions.cacheGroupStarting(grpId); + } + /** * @param cacheId Cache ID. * @return {@code True} if non-client cache was added during this exchange. http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63655a70/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 9126fd2,2096179..7de8774 --- 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 @@@ -78,11 -79,14 +79,12 @@@ public class GridCacheTestContext