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/01/09 08:22:11 UTC

ignite git commit: Re-throw NodeStoppingException from GridDhtLocalPartition.tryEvict to avoid infinite loop in GridDhtPreloader.evictPartitionAsync.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3477 92dca1e80 -> 0e8cb0098


Re-throw NodeStoppingException from GridDhtLocalPartition.tryEvict to avoid infinite loop in GridDhtPreloader.evictPartitionAsync.


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

Branch: refs/heads/ignite-3477
Commit: 0e8cb00985c458faa24bbb60a379b491bd080042
Parents: 92dca1e
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jan 9 11:22:15 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jan 9 11:22:15 2017 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  | 40 +++++++++++++++++---
 .../near/NoneRebalanceModeSelfTest.java         |  4 +-
 2 files changed, 36 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e8cb009/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index d57c012..6bddb6b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -33,6 +33,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
@@ -374,7 +375,12 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
         map.removeEntry(entry);
 
         // Attempt to evict.
-        tryEvict();
+        try {
+            tryEvict();
+        }
+        catch (NodeStoppingException ignore) {
+            // No-op.
+        }
     }
 
     /**
@@ -498,7 +504,12 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                 if ((reservations & 0xFFFF) == 0 && shouldBeRenting)
                     rent(true);
 
-                tryEvict();
+                try {
+                    tryEvict();
+                }
+                catch (NodeStoppingException ignore) {
+                    // No-op.
+                }
 
                 break;
             }
@@ -750,9 +761,9 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     }
 
     /**
-     *
+     * @throws NodeStoppingException If node is stopping.
      */
-    public void tryEvict() {
+    public void tryEvict() throws NodeStoppingException {
         long reservations = state.get();
 
         int ord = (int)(reservations >> 32);
@@ -795,7 +806,12 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      *
      */
     void onUnlock() {
-        tryEvict();
+        try {
+            tryEvict();
+        }
+        catch (NodeStoppingException ignore) {
+            // No-op.
+        }
     }
 
     /**
@@ -849,7 +865,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     /**
      * Clears values for this partition.
      */
-    public void clearAll() {
+    public void clearAll() throws NodeStoppingException {
         GridCacheVersion clearVer = cctx.versions().next();
 
         boolean rec = cctx.events().isRecordable(EVT_CACHE_REBALANCE_OBJECT_UNLOADED);
@@ -894,6 +910,12 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
                 break; // Partition is already concurrently cleared and evicted.
             }
+            catch (NodeStoppingException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to clear cache entry for evicted partition: " + cached.partition());
+
+                throw e;
+            }
             catch (IgniteCheckedException e) {
                 U.error(log, "Failed to clear cache entry for evicted partition: " + cached, e);
             }
@@ -943,6 +965,12 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                     }
                 }
             }
+            catch (NodeStoppingException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to get iterator for evicted partition: " + id);
+
+                throw e;
+            }
             catch (IgniteCheckedException e) {
                 U.error(log, "Failed to get iterator for evicted partition: " + id, e);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e8cb009/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
index 17defd6..b25d4c4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalP
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
-import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING;
 
 /**
  * Test none rebalance mode.
@@ -61,7 +61,7 @@ public class NoneRebalanceModeSelfTest extends GridCommonAbstractTest {
         GridNearTransactionalCache cache = (GridNearTransactionalCache)((IgniteKernal)grid(0)).internalCache(null);
 
         for (GridDhtLocalPartition part : cache.dht().topology().localPartitions())
-            assertEquals(MOVING, part.state());
+            assertEquals(OWNING, part.state());
 
         grid(0).cache(null).removeAll();
     }