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/12 07:37:46 UTC

[09/11] 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/3762b0dd
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3762b0dd
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3762b0dd

Branch: refs/heads/ignite-5075
Commit: 3762b0dd85796f3b3875a4b8f67b21f68e586c15
Parents: 0f25894
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 12 09:51:31 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 12 10:13:39 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       | 17 ++----
 .../internal/processors/cache/CacheData.java    |  1 -
 .../cache/DynamicCacheDescriptor.java           |  2 -
 .../processors/cache/ExchangeActions.java       |  2 +-
 .../processors/cache/GridCacheContext.java      | 12 +++++
 .../processors/cache/GridCacheIoManager.java    |  4 +-
 .../processors/cache/GridCacheProcessor.java    | 56 ++++++++------------
 .../dht/GridClientPartitionTopology.java        |  2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  4 +-
 .../GridDhtPartitionsExchangeFuture.java        | 13 ++---
 .../ignite/spi/discovery/DiscoveryDataBag.java  |  2 +-
 .../internal/GridNodeMetricsLogSelfTest.java    |  2 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |  1 -
 .../loadtests/hashmap/GridCacheTestContext.java |  2 +
 14 files changed, 52 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 bd41ccc..d186cdb 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
@@ -118,16 +118,6 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param cacheId Cache ID.
-     * @return Cache start topology version.
-     */
-    public AffinityTopologyVersion localStartVersion(int cacheId) {
-        DynamicCacheDescriptor desc = registeredCaches.get(cacheId);
-
-        return desc != null ? desc.localStartVersion() : null;
-    }
-
-    /**
      * Callback invoked from discovery thread when discovery message is received.
      *
      * @param type Event type.
@@ -330,7 +320,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @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()) {
@@ -409,7 +399,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             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())
                     U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
             }
@@ -891,9 +881,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         assert cacheDesc != null : aff.cacheName();
 
-        return fut.cacheStarted(aff.cacheId()) ||
+        return fut.cacheAddedOnExchange(aff.cacheId(), cacheDesc.receivedFrom()) ||
             !fut.exchangeId().nodeId().equals(cctx.localNodeId()) ||
-            cctx.localNodeId().equals(cacheDesc.receivedFrom()) ||
             (affNodes.size() == 1 && affNodes.contains(cctx.localNode()));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 0c97ab0..4768a9a 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
@@ -20,7 +20,6 @@ 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;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 130ebde..5c7060c 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
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheMode;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 eed60ac..99fd29d 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.
+ * Cache change requests to execute when receive {@link DynamicCacheChangeBatch} event.
  */
 public class ExchangeActions {
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 5a6f793..aa503b1 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,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Topology version when cache was started on local node. */
     private AffinityTopologyVersion locStartTopVer;
 
+    /** */
+    private UUID rcvdFrom;
+
     /** Dynamic cache deployment ID. */
     private IgniteUuid dynamicDeploymentId;
 
@@ -290,6 +293,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         CacheConfiguration cacheCfg,
         CacheType cacheType,
         AffinityTopologyVersion locStartTopVer,
+        UUID rcvdFrom,
         boolean affNode,
         boolean updatesAllowed,
         MemoryPolicy memPlc,
@@ -336,6 +340,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.cacheCfg = cacheCfg;
         this.cacheType = cacheType;
         this.locStartTopVer = locStartTopVer;
+        this.rcvdFrom = rcvdFrom;
         this.affNode = affNode;
         this.updatesAllowed = updatesAllowed;
         this.depEnabled = ctx.deploy().enabled() && !cacheObjects().isBinaryEnabled(cacheCfg);
@@ -455,6 +460,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Node ID cache was received from.
+     */
+    public UUID receivedFrom() {
+        return rcvdFrom;
+    }
+
+    /**
      * @return Topology version when cache was started on local node.
      */
     public AffinityTopologyVersion startTopologyVersion() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 348d9d8..a8a4dcd 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
@@ -146,7 +146,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                 if (cacheMsg instanceof GridDhtAffinityAssignmentRequest) {
                     assert cacheMsg.topologyVersion() != null : cacheMsg;
 
-                    AffinityTopologyVersion waitVer = cctx.affinity().localStartVersion(cacheMsg.cacheId());
+                    DynamicCacheDescriptor cacheDesc = cctx.cache().cacheDescriptor(cacheMsg.cacheId());
+
+                    AffinityTopologyVersion waitVer = cacheDesc != null ? cacheDesc.localStartVersion() : null;
 
                     if (waitVer == null)
                         waitVer = new AffinityTopologyVersion(cctx.localNode().order());

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 f9b015d..bc6a7d6 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
@@ -897,7 +897,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     *
+     * @throws IgniteCheckedException if check failed.
      */
     private void checkConsistency() throws IgniteCheckedException {
         for (ClusterNode n : ctx.discovery().remoteNodes()) {
@@ -909,9 +909,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             DeploymentMode locDepMode = ctx.config().getDeploymentMode();
             DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
 
-            CU.checkAttributeMismatch(
-                    log, null, n.id(), "deploymentMode", "Deployment mode",
-                    locDepMode, rmtDepMode, true);
+            CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
+                locDepMode, rmtDepMode, true);
         }
     }
 
@@ -1101,13 +1100,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 }
             }
         }
-// TODO
-//        if (clientReconnectReqs != null) {
-//            for (Map.Entry<UUID, DynamicCacheChangeBatch> e : clientReconnectReqs.entrySet())
-//                processClientReconnectData(e.getKey(), e.getValue());
-//
-//            clientReconnectReqs = null;
-//        }
 
         sharedCtx.onReconnected();
 
@@ -1336,15 +1328,17 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * @param cfg Cache configuration to use to create cache.
      * @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,
         @Nullable CachePluginManager pluginMgr,
-        CacheType cacheType,
+        DynamicCacheDescriptor desc,
         AffinityTopologyVersion locStartTopVer,
         CacheObjectContext cacheObjCtx,
         boolean affNode,
@@ -1365,7 +1359,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         QueryUtils.prepareCacheConfiguration(cfg);
 
-        validate(ctx.config(), cfg, cacheType, cfgStore);
+        validate(ctx.config(), cfg, desc.cacheType(), cfgStore);
 
         if (pluginMgr == null)
             pluginMgr = new CachePluginManager(ctx, cfg);
@@ -1375,7 +1369,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         sharedCtx.jta().registerCache(cfg);
 
         // Skip suggestions for internal caches.
-        if (cacheType.userCache())
+        if (desc.cacheType().userCache())
             suggestOptimizations(cfg, cfgStore != null);
 
         Collection<Object> toPrepare = new ArrayList<>();
@@ -1418,8 +1412,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             ctx,
             sharedCtx,
             cfg,
-            cacheType,
+            desc.cacheType(),
             locStartTopVer,
+            desc.receivedFrom(),
             affNode,
             updatesAllowed,
             memPlc,
@@ -1550,8 +1545,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 ctx,
                 sharedCtx,
                 cfg,
-                cacheType,
+                desc.cacheType(),
                 locStartTopVer,
+                desc.receivedFrom(),
                 affNode,
                 true,
                 memPlc,
@@ -1728,8 +1724,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         prepareCacheStart(
             cacheDesc.cacheConfiguration(),
             nearCfg,
-            cacheDesc.cacheType(),
-            cacheDesc.deploymentId(),
+            cacheDesc,
             exchTopVer,
             cacheDesc.schema()
         );
@@ -1749,8 +1744,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 prepareCacheStart(
                     desc.cacheConfiguration(),
                     t.get2(),
-                    desc.cacheType(),
-                    desc.deploymentId(),
+                    desc,
                     exchTopVer,
                     desc.schema()
                 );
@@ -1778,8 +1772,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     prepareCacheStart(
                         desc.cacheConfiguration(),
                         null,
-                        desc.cacheType(),
-                        desc.deploymentId(),
+                        desc,
                         exchTopVer,
                         desc.schema()
                     );
@@ -1793,8 +1786,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * @param startCfg Start configuration.
      * @param reqNearCfg Near configuration if specified for client cache start request.
-     * @param cacheType Cache type.
-     * @param deploymentId Deployment ID.
+     * @param desc Cache descriptor.
      * @param exchTopVer Current exchange version.
      * @param schema Query schema.
      * @throws IgniteCheckedException If failed.
@@ -1802,8 +1794,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private void prepareCacheStart(
         CacheConfiguration startCfg,
         @Nullable NearCacheConfiguration reqNearCfg,
-        CacheType cacheType,
-        IgniteUuid deploymentId,
+        DynamicCacheDescriptor desc,
         AffinityTopologyVersion exchTopVer,
         @Nullable QuerySchema schema
     ) throws IgniteCheckedException {
@@ -1830,13 +1821,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         GridCacheContext cacheCtx = createCache(ccfg,
             null,
-            cacheType,
+            desc,
             exchTopVer,
             cacheObjCtx,
             affNode,
             true);
 
-        cacheCtx.dynamicDeploymentId(deploymentId);
+        cacheCtx.dynamicDeploymentId(desc.deploymentId());
 
         GridCacheAdapter cache = cacheCtx.cache();
 
@@ -1894,9 +1885,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             sharedCtx.removeCacheContext(ctx);
 
-//            assert req.deploymentId().equals(ctx.dynamicDeploymentId()) : "Different deployment IDs [req=" + req +
-//                ", ctxDepId=" + ctx.dynamicDeploymentId() + ']';
-
             onKernalStop(cache, req.destroy());
 
             stopCache(cache, true, req.destroy());
@@ -1913,7 +1901,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     public void onExchangeDone(
         AffinityTopologyVersion topVer,
-        ExchangeActions exchActions,
+        @Nullable ExchangeActions exchActions,
         Throwable err
     ) {
         for (GridCacheAdapter<?, ?> cache : caches.values()) {
@@ -2531,8 +2519,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                         if (req.close() && desc.cacheConfiguration().getCacheMode() == LOCAL) {
                             req.close(false);
+
                             req.stop(true);
-                            req.destroy(true);
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index f3c3a1b..1de64c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -281,7 +281,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
         long updateSeq = this.updateSeq.incrementAndGet();
 
         // If this is the oldest node.
-        if (oldest.id().equals(loc.id()) || exchFut.isCacheAdded(cacheId, exchId.topologyVersion())) {
+        if (oldest.id().equals(loc.id()) || exchFut.dynamicCacheStarted(cacheId)) {
             if (node2part == null) {
                 node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 f16bb15..0410dda 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 = exchFut.cacheAddedOnExchange(cctx.cacheId(), cctx.receivedFrom());
 
             boolean first = (loc.equals(oldest) && loc.id().equals(exchId.nodeId()) && exchId.isJoined()) || added;
 
@@ -541,7 +541,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 cntrMap.clear();
 
                 // If this is the oldest node.
-                if (oldest != null && (loc.equals(oldest) || exchFut.isCacheAdded(cctx.cacheId(), exchId.topologyVersion()))) {
+                if (oldest != null && (loc.equals(oldest) || exchFut.cacheAddedOnExchange(cctx.cacheId(), cctx.receivedFrom()))) {
                     if (node2part == null) {
                         node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 477d0ce..b4cb3c1 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
@@ -380,23 +380,18 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
     /**
      * @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));
     }
 
     /**
      * @param cacheId Cache ID.
      * @return {@code True} if non-client cache was added during this exchange.
      */
-    public boolean cacheStarted(int cacheId) {
+    public boolean dynamicCacheStarted(int cacheId) {
         return exchActions != null && exchActions.cacheStarted(cacheId);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 3737d69..803beed 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,7 +188,7 @@ public class DiscoveryDataBag {
     }
 
     /**
-     * @return ID on joining node.
+     * @return ID of joining node.
      */
     public UUID joiningNodeId() {
         return joiningNodeId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
index ea298e6..b6114d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
@@ -79,7 +79,7 @@ public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
 
         //Check that nodes are alie
         assertEquals("one", cache1.get(1));
-        assertEquals("tow", cache2.get(2));
+        assertEquals("two", cache2.get(2));
 
         String fullLog = strWr.toString();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
index a7682a1..bf5ba61 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
@@ -26,7 +26,6 @@ import javax.cache.Cache;
 import javax.cache.configuration.Factory;
 import javax.cache.integration.CacheLoaderException;
 import javax.cache.integration.CacheWriterException;
-
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.store.CacheStore;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3762b0dd/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 4f0d9a1..2096179 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
@@ -18,6 +18,7 @@
 package org.apache.ignite.loadtests.hashmap;
 
 import java.util.IdentityHashMap;
+import java.util.UUID;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -80,6 +81,7 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
             defaultCacheConfiguration(),
             CacheType.USER,
             AffinityTopologyVersion.ZERO,
+            UUID.randomUUID(),
             true,
             true,
             null,