You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2019/01/11 14:51:53 UTC

[GitHub] asfgit closed pull request #4556: IGNITE-9290: Make remove explicit locks async when node left.

asfgit closed pull request #4556: IGNITE-9290: Make remove explicit locks async when node left.
URL: https://github.com/apache/ignite/pull/4556
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java
index b614728f2594..2a9a0e8b10d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java
@@ -19,7 +19,6 @@
 
 import java.util.Collection;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.jetbrains.annotations.Nullable;
@@ -35,10 +34,9 @@
     public void touch(IgniteTxEntry txEntry, boolean loc);
 
     /**
-     * @param e      Entry for eviction policy notification.
-     * @param topVer Topology version.
+     * @param e Entry for eviction policy notification.
      */
-    public void touch(GridCacheEntryEx e, AffinityTopologyVersion topVer);
+    public void touch(GridCacheEntryEx e);
 
     /**
      * @param entry Entry to attempt to evict.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
index d737c8bfca32..6813fec446d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
@@ -19,7 +19,6 @@
 
 import java.util.Collection;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
@@ -32,11 +31,11 @@
 public class CacheOffheapEvictionManager extends GridCacheManagerAdapter implements CacheEvictionManager {
     /** {@inheritDoc} */
     @Override public void touch(IgniteTxEntry txEntry, boolean loc) {
-        touch(txEntry.cached(), null);
+        touch(txEntry.cached());
     }
 
     /** {@inheritDoc} */
-    @Override public void touch(GridCacheEntryEx e, AffinityTopologyVersion topVer) {
+    @Override public void touch(GridCacheEntryEx e) {
         if (e.detached())
             return;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeDiscoveryEvents.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeDiscoveryEvents.java
index 1d5d01718f71..23df8d4f29e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeDiscoveryEvents.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeDiscoveryEvents.java
@@ -76,11 +76,6 @@
      * @param fut Current exchange future.
      */
     public void processEvents(GridDhtPartitionsExchangeFuture fut) {
-        for (DiscoveryEvent evt : evts) {
-            if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED)
-                fut.sharedContext().mvcc().removeExplicitNodeLocks(evt.eventNode().id(), fut.initialVersion());
-        }
-
         if (hasServerLeft())
             warnNoAffinityNodes(fut.sharedContext());
     }
diff --git 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
index a1c403b3f4b8..4a11ffbddd4e 100644
--- 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
@@ -906,7 +906,7 @@ else if (modes.heap) {
                         continue;
                     }
                     finally {
-                        e.touch(null);
+                        e.touch();
 
                         ctx.shared().database().checkpointReadUnlock();
                     }
@@ -959,7 +959,7 @@ else if (modes.heap) {
                 return e.peek(heap, offheap, AffinityTopologyVersion.NONE, null);
             }
             finally {
-                e.touch(null);
+                e.touch();
             }
         }
 
@@ -2113,7 +2113,7 @@ else if (storeEnabled)
                                         readerArgs);
 
                                     if (res == null)
-                                        entry.touch(topVer);
+                                        entry.touch();
                                 }
                             }
 
@@ -2128,7 +2128,7 @@ else if (storeEnabled)
                                     needVer);
 
                                 if (entry != null && (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED)))
-                                    entry.touch(topVer);
+                                    entry.touch();
 
                                 if (keysSize == 1)
                                     // Safe to return because no locks are required in READ_COMMITTED mode.
@@ -2210,7 +2210,7 @@ else if (storeEnabled)
 
                                                 if (tx0 == null || (!tx0.implicit() &&
                                                     tx0.isolation() == READ_COMMITTED))
-                                                    entry.touch(topVer);
+                                                    entry.touch();
 
                                                 break;
                                             }
@@ -2253,7 +2253,7 @@ else if (storeEnabled)
                                         GridCacheEntryEx entry = peekEx(key);
 
                                         if (entry != null)
-                                            entry.touch(topVer);
+                                            entry.touch();
                                     }
                                 }
 
@@ -2283,7 +2283,7 @@ else if (storeEnabled)
                     for (KeyCacheObject key0 : misses.keySet()) {
                         GridCacheEntryEx entry = peekEx(key0);
                         if (entry != null)
-                            entry.touch(topVer);
+                            entry.touch();
                     }
                 }
 
@@ -2357,7 +2357,7 @@ private void clearReservationsIfNeeded(
                             entry.clearReserveForLoad(e.getValue().version());
 
                         if (needTouch)
-                            entry.touch(topVer);
+                            entry.touch();
                     }
                 }
             }
@@ -3712,7 +3712,7 @@ private void loadEntry(KeyCacheObject key,
                 log.debug("Got removed entry during loadCache (will ignore): " + entry);
         }
         finally {
-            entry.touch(topVer);
+            entry.touch();
         }
 
         CU.unwindEvicts(ctx);
diff --git 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
index 938fd72ea267..35879566980a 100644
--- 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
@@ -204,7 +204,7 @@ protected final GridCacheMapEntry putEntryIfObsoleteOrAbsent(
                         true);
 
                 if (touch)
-                    cur.touch(topVer);
+                    cur.touch();
             }
 
             assert Math.abs(sizeChange) <= 1;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 8cef1763af86..26da38ba019f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -1208,7 +1208,6 @@ public GridCacheUpdateTxResult mvccUpdateRowsWithPreloadInfo(
     /**
      * Touch this entry in its context's eviction manager.
      *
-     * @param topVer Topology version.
      */
-    public void touch(AffinityTopologyVersion topVer);
+    public void touch();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index 5a3453ad8e9c..24c042988369 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -23,7 +23,6 @@
 import org.apache.ignite.cache.eviction.EvictionFilter;
 import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
@@ -200,7 +199,7 @@ private boolean evict0(
     }
 
     /** {@inheritDoc} */
-    @Override public void touch(GridCacheEntryEx e, AffinityTopologyVersion topVer) {
+    @Override public void touch(GridCacheEntryEx e) {
         assert e.context() == cctx : "Entry from another cache context passed to eviction manager: [" +
             "entry=" + e +
             ", cctx=" + cctx +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 881e8688411a..161a65e74c9f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -1069,7 +1069,7 @@ private EntryGetResult entryGetResult(CacheObject val, GridCacheVersion ver, boo
         }
         finally {
             if (touch)
-                touch(cctx.affinity().affinityTopologyVersion());
+                touch();
         }
     }
 
@@ -1244,7 +1244,7 @@ else if (res.resultType() == ResultType.REMOVED_NOT_NULL) {
             if (lockedByCurrentThread()) {
                 unlockEntry();
 
-                cctx.evicts().touch(this, AffinityTopologyVersion.NONE);
+                cctx.evicts().touch(this);
             }
         }
 
@@ -1360,7 +1360,7 @@ else if (res.resultType() == ResultType.LOCKED) {
             if (lockedByCurrentThread()) {
                 unlockEntry();
 
-                cctx.evicts().touch(this, AffinityTopologyVersion.NONE);
+                cctx.evicts().touch(this);
             }
         }
 
@@ -1428,7 +1428,7 @@ else if (res.resultType() == ResultType.LOCKED) {
             if (lockedByCurrentThread()) {
                 unlockEntry();
 
-                cctx.evicts().touch(this, AffinityTopologyVersion.NONE);
+                cctx.evicts().touch(this);
             }
         }
 
@@ -5054,8 +5054,8 @@ private void unlockListenerReadLock() {
     }
 
     /** {@inheritDoc} */
-    @Override public void touch(AffinityTopologyVersion topVer) {
-        context().evicts().touch(this, topVer);
+    @Override public void touch() {
+        context().evicts().touch(this);
     }
 
     /** {@inheritDoc} */
@@ -5255,7 +5255,7 @@ else if (res.resultType() == ResultType.LOCKED) {
                 if (entry.lockedByCurrentThread()) {
                     entry.unlockEntry();
 
-                    cctx.evicts().touch(entry, AffinityTopologyVersion.NONE);
+                    cctx.evicts().touch(entry);
                 }
             }
 
@@ -5360,7 +5360,7 @@ else if (res.resultType() == ResultType.LOCKED) {
                 if (entry.lockedByCurrentThread()) {
                     entry.unlockEntry();
 
-                    cctx.evicts().touch(entry, AffinityTopologyVersion.NONE);
+                    cctx.evicts().touch(entry);
                 }
             }
 
@@ -5610,7 +5610,7 @@ else if (res.resultType() == ResultType.REMOVED_NOT_NULL) {
                 if (entry.lockedByCurrentThread()) {
                     entry.unlockEntry();
 
-                    cctx.evicts().touch(entry, AffinityTopologyVersion.NONE);
+                    cctx.evicts().touch(entry);
                 }
             }
 
@@ -6805,7 +6805,7 @@ private void versionCheck(@Nullable IgniteBiTuple<Object, Exception> invokeRes)
             if (lockedByCurrentThread()) {
                 unlockEntry();
 
-                cctx.evicts().touch(this, AffinityTopologyVersion.NONE);
+                cctx.evicts().touch(this);
             }
         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index bd2b58d6690a..32b06a3f9df6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -260,6 +260,8 @@ else if (log.isDebugEnabled())
             if (log.isDebugEnabled())
                 log.debug("Processing node left [nodeId=" + discoEvt.eventNode().id() + "]");
 
+            removeExplicitNodeLocks(discoEvt.eventNode().id());
+
             for (GridCacheFuture<?> fut : activeFutures())
                 fut.onNodeLeft(discoEvt.eventNode().id());
 
@@ -351,23 +353,25 @@ private IgniteInternalFuture ignoreErrors(IgniteInternalFuture<?> f) {
 
     /**
      * @param leftNodeId Left node ID.
-     * @param topVer Topology version.
      */
-    public void removeExplicitNodeLocks(UUID leftNodeId, AffinityTopologyVersion topVer) {
-        for (GridDistributedCacheEntry entry : locked()) {
-            try {
-                entry.removeExplicitNodeLocks(leftNodeId);
-
-                entry.touch(topVer);
-            }
-            catch (GridCacheEntryRemovedException ignore) {
-                if (log.isDebugEnabled())
-                    log.debug("Attempted to remove node locks from removed entry in mvcc manager " +
-                        "disco callback (will ignore): " + entry);
-            }
-
-            cctx.exchange().exchangerUpdateHeartbeat();
-        }
+    public void removeExplicitNodeLocks(UUID leftNodeId) {
+        cctx.kernalContext().closure().runLocalSafe(
+            new Runnable() {
+                @Override public void run() {
+                    for (GridDistributedCacheEntry entry : locked()) {
+                        try {
+                            entry.removeExplicitNodeLocks(leftNodeId);
+
+                            entry.touch();
+                        }
+                        catch (GridCacheEntryRemovedException ignore) {
+                            if (log.isDebugEnabled())
+                                log.debug("Attempted to remove node locks from removed entry in cache lock manager " +
+                                    "disco callback (will ignore): " + entry);
+                        }
+                    }
+                }
+            }, true);
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index 2166ce5eba93..1a887e24f55a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -71,7 +71,7 @@
                 }
 
                 if (touch)
-                    entry.touch(null);
+                    entry.touch();
             }
         };
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index bf1c35932b8a..0cca2554803c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1790,7 +1790,7 @@ private void process(KeyCacheObject key, CacheObject val, GridCacheVersion ver,
                     }
                     finally {
                         if (entry != null)
-                            entry.touch(topVer);
+                            entry.touch();
 
                         cctx.shared().database().checkpointReadUnlock();
                     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 58b12ed5feb9..dddd39226aa1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -260,7 +260,7 @@ private void processForceKeysRequest0(ClusterNode node, GridDhtForceKeysRequest
                                 res.addInfo(info);
                         }
 
-                        entry.touch(msg.topologyVersion());
+                        entry.touch();
 
                         break;
                     }
@@ -699,7 +699,7 @@ private void loadEntry(KeyCacheObject key,
                 }
                 finally {
                     if (entry != null)
-                        entry.touch(topVer);
+                        entry.touch();
 
                     part.release();
 
@@ -1232,7 +1232,7 @@ private void updateTtl(GridCacheAdapter<K, V> cache,
                 }
                 finally {
                     if (entry != null)
-                        entry.touch(AffinityTopologyVersion.NONE);
+                        entry.touch();
                 }
             }
             catch (IgniteCheckedException e) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index 96d17691f2c7..8c754f21c58f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -440,7 +440,7 @@ private boolean map(KeyCacheObject key, boolean forceKeys) {
                             log.debug("Got removed entry when getting a DHT value: " + e);
                     }
                     finally {
-                        e.touch(topVer);
+                        e.touch();
                     }
                 }
             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
index 8588084fa77e..fa15bd8c6aac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
@@ -388,7 +388,7 @@ private void getAsync() {
                         log.debug("Got removed entry when getting a DHT value: " + e);
                 }
                 finally {
-                    e.touch(topVer);
+                    e.touch();
                 }
             }
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index dcd7be51b4f0..97ef70d0166c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -1647,7 +1647,7 @@ private void clearLocks(UUID nodeId, GridDistributedUnlockRequest req) {
                                     "(added to cancelled locks set): " + req);
                         }
 
-                        entry.touch(ctx.affinity().affinityTopologyVersion());
+                        entry.touch();
 
                         break;
                     }
@@ -1833,7 +1833,7 @@ else if (log.isDebugEnabled())
                     if (created && entry.markObsolete(dhtVer))
                         removeEntry(entry);
 
-                    entry.touch(topVer);
+                    entry.touch();
 
                     break;
                 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 0629754bf985..5c40d915e145 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -551,7 +551,7 @@ private boolean localGet(AffinityTopologyVersion topVer, KeyCacheObject key, int
                                 mvccSnapshot());
                         }
 
-                        entry.touch(topVer);
+                        entry.touch();
 
                         // Entry was not in memory or in swap, so we remove it from cache.
                         if (v == null) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index 0d6948586926..c20c1c8eba49 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -509,7 +509,7 @@ private boolean localGet(AffinityTopologyVersion topVer, KeyCacheObject key, int
                                 mvccSnapshot);
                         }
 
-                        entry.touch(topVer);
+                        entry.touch();
 
                         // Entry was not in memory or in swap, so we remove it from cache.
                         if (v == null) {
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 ab4287e8097f..e25af3c94e75 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
@@ -1594,7 +1594,7 @@ private IgniteInternalFuture removeAllAsync0(
                             }
                             finally {
                                 if (entry != null)
-                                    entry.touch(topVer);
+                                    entry.touch();
                             }
                         }
                     }
@@ -3095,7 +3095,7 @@ private void unlockEntries(List<GridDhtCacheEntry> locked, AffinityTopologyVersi
         for (int i = 0; i < size; i++) {
             GridCacheMapEntry entry = locked.get(i);
             if (entry != null && (skip == null || !skip.contains(entry.key())))
-                entry.touch(topVer);
+                entry.touch();
         }
     }
 
@@ -3375,7 +3375,7 @@ private void processDhtAtomicUpdateRequest(UUID nodeId, GridDhtAtomicAbstractUpd
                         }
                         finally {
                             if (entry != null)
-                                entry.touch(req.topologyVersion());
+                                entry.touch();
                         }
                     }
                 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index a6ac902fba5f..76e11d684a58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -659,7 +659,7 @@ public GridDistributedCacheEntry entryExx(
                             }
                             finally {
                                 if (entry != null)
-                                    entry.touch(topVer);
+                                    entry.touch();
                             }
                         }
                     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index ddbb3b143fd4..23f15cd69c51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -932,7 +932,7 @@ private boolean preloadEntry(
                         cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE,
                         false
                     )) {
-                        cached.touch(topVer); // Start tracking.
+                        cached.touch(); // Start tracking.
 
                         if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_OBJECT_LOADED) && !cached.isInternal())
                             cctx.events().addEvent(cached.partition(), cached.key(), cctx.localNodeId(), null,
@@ -940,7 +940,7 @@ private boolean preloadEntry(
                                 false, null, null, null, true);
                     }
                     else {
-                        cached.touch(topVer); // Start tracking.
+                        cached.touch(); // Start tracking.
 
                         if (log.isTraceEnabled())
                             log.trace("Rebalancing entry is already in cache (will ignore) [key=" + cached.key() +
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 dbc51f72c085..4ffc4be8b4f0 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
@@ -1592,9 +1592,6 @@ private void waitPartitionRelease(boolean distributed, boolean doRollback) throw
 
             // Assign to class variable so it will be included into toString() method.
             this.partReleaseFut = partReleaseFut;
-
-            if (exchId.isLeft())
-                cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
         }
         finally {
             cctx.exchange().exchangerBlockingSectionEnd();
@@ -1768,8 +1765,6 @@ private void onLeft() {
 
             cctx.exchange().exchangerUpdateHeartbeat();
         }
-
-        cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
     }
 
     /**
@@ -4322,8 +4317,6 @@ public void onNodeLeft(final ClusterNode node) {
         if (isDone() || !enterBusy())
             return;
 
-        cctx.mvcc().removeExplicitNodeLocks(node.id(), initialVersion());
-
         try {
             onDiscoveryEvent(new IgniteRunnable() {
                 @Override public void run() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index 998dcf9f3a20..211ae126f6a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -284,7 +284,7 @@ private void processNearAtomicUpdateResponse(
                 }
                 finally {
                     if (entry != null)
-                        entry.touch(topVer);
+                        entry.touch();
                 }
             }
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 40dbf6a969af..ca71e608897f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -455,7 +455,7 @@ public void init(@Nullable AffinityTopologyVersion topVer) {
             }
             finally {
                 if (entry != null && tx == null)
-                    entry.touch(topVer);
+                    entry.touch();
             }
         }
 
@@ -560,7 +560,7 @@ private boolean localDhtGet(
                 if (dhtEntry != null)
                     // Near cache is enabled, so near entry will be enlisted in the transaction.
                     // Always touch DHT entry in this case.
-                    dhtEntry.touch(topVer);
+                    dhtEntry.touch();
             }
         }
     }
@@ -705,7 +705,7 @@ private void releaseEvictions(
                 entry.releaseEviction();
 
                 if (tx == null)
-                    entry.touch(topVer);
+                    entry.touch();
             }
         }
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index 5cf4755b0dcc..b97822319944 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -254,7 +254,7 @@ public void clearLocks(UUID nodeId, GridDhtUnlockRequest req) {
                                         "(added to cancelled locks set): " + req);
                             }
 
-                            entry.touch(topVer);
+                            entry.touch();
                         }
                         else if (log.isDebugEnabled())
                             log.debug("Received unlock request for entry that could not be found: " + req);
@@ -362,7 +362,7 @@ else if (log.isDebugEnabled())
                             );
 
                             if (!req.inTx())
-                                entry.touch(req.topologyVersion());
+                                entry.touch();
                         }
                         else {
                             if (evicted == null)
@@ -595,7 +595,7 @@ else if (log.isDebugEnabled())
                         if (topVer.equals(AffinityTopologyVersion.NONE))
                             topVer = ctx.affinity().affinityTopologyVersion();
 
-                        entry.touch(topVer);
+                        entry.touch();
 
                         break;
                     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 7d509477a12e..b9187534e19c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -1418,7 +1418,7 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx,
                             }
                         }
                         catch (ClusterTopologyCheckedException e) {
-                            entry.touch(topologyVersion());
+                            entry.touch();
 
                             throw e;
                         }
@@ -1476,7 +1476,7 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx,
                         }
 
                         if (readCommitted())
-                            entry.touch(topologyVersion());
+                            entry.touch();
 
                         break; // While.
                     }
@@ -2766,7 +2766,7 @@ private MvccSnapshot mvccReadSnapshot(GridCacheContext cctx) {
                                 }
                             }
                             else
-                                entry.touch(topVer);
+                                entry.touch();
                         }
                     }
                 }
@@ -2955,7 +2955,7 @@ private void onException() {
             GridCacheEntryEx cached0 = txEntry.cached();
 
             if (cached0 != null)
-                cached0.touch(topologyVersion());
+                cached0.touch();
         }
     }
 
@@ -4556,7 +4556,7 @@ public TransactionProxy rollbackOnlyProxy() {
                         GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey, topVer) : txEntry.cached();
 
                         if (readCommitted() || skipVals) {
-                            e.touch(topologyVersion());
+                            e.touch();
 
                             if (visibleVal != null) {
                                 cacheCtx.addResult(map,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
index 481a6cf3d8a1..3bdb44e72aa5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
@@ -182,7 +182,7 @@ GridLocalCacheEntry entryExx(KeyCacheObject key) {
             if (entry != null && ctx.isAll(entry, CU.empty0())) {
                 entry.releaseLocal();
 
-                entry.touch(topVer);
+                entry.touch();
             }
         }
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index dc7e3ec80916..07a6dfc10e47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -512,7 +512,7 @@ public GridLocalAtomicCache(GridCacheContext<K, V> ctx) {
                     }
                     finally {
                         if (entry != null)
-                            entry.touch(ctx.affinity().affinityTopologyVersion());
+                            entry.touch();
                     }
 
                     if (!success && storeEnabled)
@@ -977,7 +977,7 @@ else if (res == null)
                     }
                     finally {
                         if (entry != null)
-                            entry.touch(ctx.affinity().affinityTopologyVersion());
+                            entry.touch();
                     }
                 }
             }
@@ -1508,7 +1508,7 @@ else if (op == UPDATE) {
         AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
 
         for (GridCacheEntryEx entry : locked)
-            entry.touch(topVer);
+            entry.touch();
 
         throw new NullPointerException("Null key.");
     }
@@ -1525,7 +1525,7 @@ private void unlockEntries(Iterable<GridCacheEntryEx> locked) {
         AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
 
         for (GridCacheEntryEx entry : locked)
-            entry.touch(topVer);
+            entry.touch();
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
index c4486202ce29..b14e30bd5fc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
@@ -2448,7 +2448,8 @@ private void cleanup(GridDhtLocalPartition part, KeyCacheObject key, List<MvccLi
                     }
                 } finally {
                     entry.unlockEntry();
-                    cctx.evicts().touch(entry, AffinityTopologyVersion.NONE);
+
+                    cctx.evicts().touch(entry);
 
                     metrics.addCleanupNanoTime(System.nanoTime() - cleanupStartNanoTime);
                     metrics.addCleanupRowsCnt(cleaned);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 8949a2f5dd1a..7f4699fb05c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -3008,7 +3008,7 @@ private void advance() {
 
                                 val = entry.peek(true, true, topVer, expiryPlc);
 
-                                entry.touch(topVer);
+                                entry.touch();
 
                                 break;
                             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 76cb89901b42..5e3a0c825ffc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -2286,7 +2286,7 @@ else if (ttl == CU.TTL_NOT_CHANGED)
                             primary ? GridDrType.DR_LOAD : GridDrType.DR_PRELOAD,
                             false);
 
-                        entry.touch(topVer);
+                        entry.touch();
 
                         CU.unwindEvicts(cctx);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
index cb78ff6b1137..d7bc721a8c5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
@@ -20,7 +20,6 @@
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
@@ -242,7 +241,7 @@ private void processKey(KeyCacheObject key, SchemaIndexCacheVisitorClosure clo)
                     entry.updateIndex(rowFilter, clo);
                 }
                 finally {
-                    entry.touch(AffinityTopologyVersion.NONE);
+                    entry.touch();
                 }
 
                 break;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
index 5c0f3d468a74..bb293f096807 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
@@ -31,7 +31,6 @@
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -164,7 +163,7 @@ private void doTest(boolean cpOnRead, final boolean mutate, int expVal, int expC
 
             CacheObject obj = entry.peekVisibleValue();
 
-            entry.touch(AffinityTopologyVersion.NONE);
+            entry.touch();
 
             int actCnt = cnt.get();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index c26589af1a8f..987648ac3605 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -6531,7 +6531,7 @@ public CheckEntriesTask(Collection<String> keys) {
 
                     size++;
 
-                    e.touch(null);
+                    e.touch();
                 }
             }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 358dfc391ae0..6e640110afd7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -963,7 +963,7 @@ GridCacheMvccCandidate anyOwner() {
     }
 
     /** {@inheritDoc} */
-    @Override public void touch(AffinityTopologyVersion topVer) {
-        context().evicts().touch(this, topVer);
+    @Override public void touch() {
+        context().evicts().touch(this);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
index 30357a1d38e5..a7a2fdc25d17 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
@@ -6378,7 +6378,7 @@ public void testGetEntry() throws Exception {
 
                     size++;
 
-                    e.touch(null);
+                    e.touch();
                 }
             }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 13d83d438925..fbf40afb0bc4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -1733,7 +1733,7 @@ private void checkData(final Map<Integer, Integer> map,
                                     }
                                 }
                                 finally {
-                                    entry.touch(entry.context().affinity().affinityTopologyVersion());
+                                    entry.touch();
                                 }
                             }
                             else
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
index 72fdd1394794..b00429940073 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
@@ -376,7 +376,7 @@ private void dataCheck(IgniteKernal orig, IgniteKernal backup, int key, boolean
             assertTrue("Invalid backup cache entry: " + dhtEntry, dhtEntry.rawGet() == null);
         }
 
-        dhtEntry.touch(null);
+        dhtEntry.touch();
     }
 
     /**


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services