You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2015/08/03 17:16:01 UTC

[10/32] incubator-ignite git commit: IGNITE-1165 - Fixed assertion for offheap entry.

IGNITE-1165 - Fixed assertion for offheap entry.


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

Branch: refs/heads/ignite-1142
Commit: f00f71d3a9f059be5625043e032070be88b6ab43
Parents: 6711d2c
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Jul 28 16:02:47 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Jul 28 16:02:47 2015 -0700

----------------------------------------------------------------------
 .../distributed/near/GridNearGetFuture.java     |  20 +--
 .../IgniteCacheNearOffheapGetSelfTest.java      | 131 +++++++++++++++++++
 ...achePartitionedMultiNodeFullApiSelfTest.java |   6 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 4 files changed, 147 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f00f71d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
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 58f6fe5..d109d2b 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
@@ -437,17 +437,19 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
 
                 ClusterNode primary = null;
 
-                if (v == null && allowLocRead) {
+                if (v == null && allowLocRead && cctx.affinityNode()) {
                     GridDhtCacheAdapter<K, V> dht = cache().dht();
 
+                    GridCacheEntryEx dhtEntry = null;
+
                     try {
-                        entry = dht.context().isSwapOrOffheapEnabled() ? dht.entryEx(key) : dht.peekEx(key);
+                        dhtEntry = dht.context().isSwapOrOffheapEnabled() ? dht.entryEx(key) : dht.peekEx(key);
 
                         // If near cache does not have value, then we peek DHT cache.
-                        if (entry != null) {
-                            boolean isNew = entry.isNewLocked() || !entry.valid(topVer);
+                        if (dhtEntry != null) {
+                            boolean isNew = dhtEntry.isNewLocked() || !dhtEntry.valid(topVer);
 
-                            v = entry.innerGet(tx,
+                            v = dhtEntry.innerGet(tx,
                                 /*swap*/true,
                                 /*read-through*/false,
                                 /*fail-fast*/true,
@@ -461,7 +463,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                                 expiryPlc);
 
                             // Entry was not in memory or in swap, so we remove it from cache.
-                            if (v == null && isNew && entry.markObsoleteIfEmpty(ver))
+                            if (v == null && isNew && dhtEntry.markObsoleteIfEmpty(ver))
                                 dht.removeIfObsolete(key);
                         }
 
@@ -483,12 +485,12 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                                 near.metrics0().onRead(false);
                         }
                     }
-                    catch (GridDhtInvalidPartitionException ignored) {
+                    catch (GridDhtInvalidPartitionException | GridCacheEntryRemovedException ignored) {
                         // No-op.
                     }
                     finally {
-                        if (entry != null && (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED))) {
-                            dht.context().evicts().touch(entry, topVer);
+                        if (dhtEntry != null && (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED))) {
+                            dht.context().evicts().touch(dhtEntry, topVer);
 
                             entry = null;
                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f00f71d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java
new file mode 100644
index 0000000..6edf5b6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.eviction.fifo.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheNearOffheapGetSelfTest extends GridCacheAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        if (nearEnabled())
+            grid(gridCount() - 1).getOrCreateCache(new CacheConfiguration(), nearConfiguration());
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        NearCacheConfiguration nearCfg = super.nearConfiguration();
+
+        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(100));
+
+        return nearCfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (getTestGridName(gridCount() - 1).equals(gridName)) {
+            cfg.setClientMode(true);
+
+            cfg.setCacheConfiguration();
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setBackups(1);
+        cfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    @Override
+    protected long getTestTimeout() {
+        return Long.MAX_VALUE;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetFromNear() throws Exception {
+
+        IgniteCache<Object, Object> nearOnly = ignite(gridCount() - 1).cache(null);
+
+        // Start extra node.
+        IgniteEx ignite = startGrid(gridCount());
+
+        try {
+            final int keyCnt = 30;
+
+            for (int i = 0; i < keyCnt; i++)
+                ignite(0).cache(null).put(i, i);
+
+            for (int i = 0; i < keyCnt; i++)
+                assertEquals(i, nearOnly.get(i));
+
+            Collection<Integer> invalidatedKeys = new ArrayList<>();
+
+            Affinity<Object> cacheAff = ignite.affinity(null);
+
+            // Going to stop the last node.
+            for (int i = 0; i < keyCnt; i++) {
+                if (cacheAff.mapKeyToNode(i).equals(ignite.localNode()))
+                    invalidatedKeys.add(i);
+            }
+
+            stopGrid(gridCount());
+
+            for (Integer key : invalidatedKeys)
+                assertEquals(key, nearOnly.get(key));
+        }
+        finally {
+            if (Ignition.state(getTestGridName(gridCount())) == IgniteState.STARTED)
+                stopGrid(gridCount());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f00f71d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 30c9e8a..747de21 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -306,7 +306,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         for (int i = 0; i < gridCount(); i++) {
             IgniteEx ignite = grid(i);
 
-            if (!ignite.configuration().isClientMode()) {
+            if (!Boolean.TRUE.equals(ignite.configuration().isClientMode())) {
                 if (ignite0 == null)
                     ignite0 = ignite;
                 else if (ignite1 == null)
@@ -361,7 +361,9 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         boolean nearEnabled = cache2.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null;
 
         assertEquals(nearEnabled ? 2 : 0, cache2.localSize(NEAR));
-        assertEquals(0, cache2.localSize(CachePeekMode.ALL) - cache2.localSize(NEAR));
+
+        if (cacheMode() != REPLICATED)
+            assertEquals(0, cache2.localSize(CachePeekMode.ALL) - cache2.localSize(NEAR));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f00f71d3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 6a59826..bf760f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -123,6 +123,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheColocatedOptimisticTransactionSelfTest.class));
         suite.addTestSuite(GridCacheAtomicMessageCountSelfTest.class);
         suite.addTest(new TestSuite(GridCacheNearPartitionedClearSelfTest.class));
+        suite.addTest(new TestSuite(IgniteCacheNearOffheapGetSelfTest.class));
 
         suite.addTest(new TestSuite(GridCacheDhtExpiredEntriesPreloadSelfTest.class));
         suite.addTest(new TestSuite(GridCacheNearExpiredEntriesPreloadSelfTest.class));