You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/02 19:59:08 UTC

[1/5] incubator-ignite git commit: ignite-968 wait for init future before processing message

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-5 4701d6d22 -> 2b052c29c


ignite-968 wait for init future before processing message


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

Branch: refs/heads/ignite-sprint-5
Commit: 4e563ed8985a52722de0b263eb486cd7d945053f
Parents: 2f61522
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 09:52:45 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 09:52:45 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        | 46 ++++++++++----------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4e563ed8/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 145def8..db43c6c 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
@@ -1140,39 +1140,39 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             return;
         }
 
-        ClusterNode curOldest = oldestNode.get();
+        if (log.isDebugEnabled())
+            log.debug("Received full partition map from node [nodeId=" + nodeId + ", msg=" + msg + ']');
 
-        if (!nodeId.equals(curOldest.id())) {
-            if (log.isDebugEnabled())
-                log.debug("Received full partition map from unexpected node [oldest=" + curOldest.id() +
-                    ", unexpectedNodeId=" + nodeId + ']');
+        assert exchId.topologyVersion().equals(msg.topologyVersion());
 
-            ClusterNode snd = cctx.discovery().node(nodeId);
+        initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+            @Override public void apply(IgniteInternalFuture<Boolean> t) {
+                ClusterNode curOldest = oldestNode.get();
 
-            if (snd == null) {
-                if (log.isDebugEnabled())
-                    log.debug("Sender node left grid, will ignore message from unexpected node [nodeId=" + nodeId +
-                        ", exchId=" + msg.exchangeId() + ']');
+                if (!nodeId.equals(curOldest.id())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Received full partition map from unexpected node [oldest=" + curOldest.id() +
+                            ", unexpectedNodeId=" + nodeId + ']');
 
-                return;
-            }
+                    ClusterNode snd = cctx.discovery().node(nodeId);
 
-            // Will process message later if sender node becomes oldest node.
-            if (snd.order() > curOldest.order())
-                fullMsgs.put(nodeId, msg);
+                    if (snd == null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Sender node left grid, will ignore message from unexpected node [nodeId=" + nodeId +
+                                ", exchId=" + msg.exchangeId() + ']');
 
-            return;
-        }
+                        return;
+                    }
 
-        assert msg.exchangeId().equals(exchId);
+                    // Will process message later if sender node becomes oldest node.
+                    if (snd.order() > curOldest.order())
+                        fullMsgs.put(nodeId, msg);
 
-        if (log.isDebugEnabled())
-            log.debug("Received full partition map from node [nodeId=" + nodeId + ", msg=" + msg + ']');
+                    return;
+                }
 
-        assert exchId.topologyVersion().equals(msg.topologyVersion());
+                assert msg.exchangeId().equals(exchId);
 
-        initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-            @Override public void apply(IgniteInternalFuture<Boolean> t) {
                 assert msg.lastVersion() != null;
 
                 cctx.versions().onReceived(nodeId, msg.lastVersion());


[2/5] incubator-ignite git commit: ignite-968 avoid wait for affinity future on metrics creation

Posted by sb...@apache.org.
ignite-968 avoid wait for affinity future on metrics creation


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

Branch: refs/heads/ignite-sprint-5
Commit: 3c1f0caae8266cf65079f345f95063bea5d5e2a8
Parents: 4e563ed
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 10:30:29 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 10:30:29 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/discovery/GridDiscoveryManager.java | 4 +++-
 .../ignite/internal/processors/cache/CacheMetricsImpl.java       | 4 ++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c1f0caa/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 4ef602e..1db0b07 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
@@ -695,7 +695,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 Map<Integer, CacheMetrics> metrics = null;
 
                 for (GridCacheAdapter<?, ?> cache : caches) {
-                    if (cache.context().started() && cache.configuration().isStatisticsEnabled()) {
+                    if (cache.configuration().isStatisticsEnabled() &&
+                        cache.context().started() &&
+                        cache.context().affinity().affinityTopologyVersion().topologyVersion() > 0) {
                         if (metrics == null)
                             metrics = U.newHashMap(caches.size());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c1f0caa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 4e6a447..74ba100 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -225,7 +225,7 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** {@inheritDoc} */
     @Override public long getOffHeapPrimaryEntriesCount() {
         try {
-            return cctx.swap().offheapEntriesCount(true, false, NONE);
+            return cctx.swap().offheapEntriesCount(true, false, cctx.affinity().affinityTopologyVersion());
         }
         catch (IgniteCheckedException e) {
             return 0;
@@ -235,7 +235,7 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** {@inheritDoc} */
     @Override public long getOffHeapBackupEntriesCount() {
         try {
-            return cctx.swap().offheapEntriesCount(false, true, NONE);
+            return cctx.swap().offheapEntriesCount(false, true, cctx.affinity().affinityTopologyVersion());
         }
         catch (IgniteCheckedException e) {
             return 0;


[3/5] incubator-ignite git commit: ignite-968 lock entries after topology version check, otherwise can add entry in evicted partition

Posted by sb...@apache.org.
ignite-968 lock entries after topology version check, otherwise can add entry in evicted partition


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

Branch: refs/heads/ignite-sprint-5
Commit: 3ebb1c03ad8b13714f85d7a464a126f4ca931347
Parents: 3c1f0ca
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 12:46:08 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 12:49:02 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/atomic/GridDhtAtomicCache.java     | 13 ++++++++-----
 1 file changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3ebb1c03/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 69f5501..9ca80f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1024,9 +1024,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         IgniteCacheExpiryPolicy expiry = null;
 
         try {
-            // If batch store update is enabled, we need to lock all entries.
-            // First, need to acquire locks on cache entries, then check filter.
-            List<GridDhtCacheEntry> locked = lockEntries(keys, req.topologyVersion());
+            List<GridDhtCacheEntry> locked = null;
             Collection<IgniteBiTuple<GridDhtCacheEntry, GridCacheVersion>> deleted = null;
 
             try {
@@ -1043,7 +1041,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     }
 
                     // Do not check topology version for CLOCK versioning since
-                    // partition exchange will wait for near update future.
+                    // partition exchange will wait for near update future (if future is on server node).
                     // Also do not check topology version if topology was locked on near node by
                     // external transaction or explicit lock.
                     if ((req.fastMap() && !req.clientRequest()) || req.topologyLocked() ||
@@ -1056,6 +1054,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                             return;
                         }
 
+                        // If batch store update is enabled, we need to lock all entries.
+                        // First, need to acquire locks on cache entries, then check filter.
+                        locked = lockEntries(keys, req.topologyVersion());
+
                         boolean hasNear = ctx.discovery().cacheNearNode(node, name());
 
                         GridCacheVersion ver = req.updateVersion();
@@ -1144,7 +1146,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 e.printStackTrace();
             }
             finally {
-                unlockEntries(locked, req.topologyVersion());
+                if (locked != null)
+                    unlockEntries(locked, req.topologyVersion());
 
                 // Enqueue if necessary after locks release.
                 if (deleted != null) {


[5/5] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-968' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-968' into ignite-sprint-5


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

Branch: refs/heads/ignite-sprint-5
Commit: 2b052c29c2bef2a04b01253cbcb9d35f7e2b6d6d
Parents: 4701d6d f711773
Author: sboikov <se...@inria.fr>
Authored: Tue Jun 2 20:57:38 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Tue Jun 2 20:57:38 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  4 +-
 .../processors/cache/CacheMetricsImpl.java      |  4 +-
 .../dht/atomic/GridDhtAtomicCache.java          | 13 +++---
 .../GridDhtPartitionsExchangeFuture.java        | 46 ++++++++++----------
 4 files changed, 36 insertions(+), 31 deletions(-)
----------------------------------------------------------------------



[4/5] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-968

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-968


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

Branch: refs/heads/ignite-sprint-5
Commit: f7117733048304b02f9565fcf4aeb1fd7614b9c1
Parents: 3ebb1c0 8719b73
Author: sboikov <se...@inria.fr>
Authored: Tue Jun 2 18:50:09 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Tue Jun 2 18:50:09 2015 +0300

----------------------------------------------------------------------
 assembly/dependencies-visor-console.xml                 |  3 +++
 .../cache/eviction/random/RandomEvictionPolicy.java     |  3 ++-
 .../internal/managers/communication/GridIoManager.java  | 12 +++++-------
 .../distributed/dht/preloader/GridDhtPreloader.java     |  9 ++++++---
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java       |  2 +-
 .../random/RandomEvictionPolicyCacheSizeSelfTest.java   |  6 ++++++
 .../testsuites/IgniteCacheQuerySelfTestSuite.java       |  2 +-
 7 files changed, 24 insertions(+), 13 deletions(-)
----------------------------------------------------------------------