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/11/13 07:47:36 UTC

ignite git commit: IGNITE-6767 Reset non-valid ner cache entry

Repository: ignite
Updated Branches:
  refs/heads/master 191295d45 -> 85027e7fa


IGNITE-6767 Reset non-valid ner cache entry


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

Branch: refs/heads/master
Commit: 85027e7fa348e9b210d6fa24add6f0b86d6d262f
Parents: 191295d
Author: Tim Onyschak <to...@trustwave.com>
Authored: Mon Nov 13 10:47:30 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Nov 13 10:47:30 2017 +0300

----------------------------------------------------------------------
 .../distributed/near/GridNearCacheEntry.java    |   2 +-
 .../near/GridCacheNearClientHitTest.java        | 154 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   3 +-
 3 files changed, 157 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/85027e7f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index ea52766..baf117b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -379,7 +379,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
                 CacheObject old = this.val;
                 boolean hasVal = hasValueUnlocked();
 
-                if (this.dhtVer == null || this.dhtVer.compareTo(dhtVer) < 0) {
+                if (this.dhtVer == null || this.dhtVer.compareTo(dhtVer) < 0 || !valid(topVer)) {
                     primaryNode(primaryNodeId, topVer);
 
                     update(val, expireTime, ttl, ver, true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/85027e7f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
new file mode 100644
index 0000000..1dd62e4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.near;
+
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CachePeekMode.NEAR;
+
+/**
+ *
+ */
+public class GridCacheNearClientHitTest extends GridCommonAbstractTest {
+    /** Ip finder. */
+    private final static TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private final static String CACHE_NAME = "test-near-cache";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(final String igniteInstanceName) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /**
+     * @param igniteInstanceName Node name.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration getClientConfiguration(final String igniteInstanceName) throws Exception {
+        final IgniteConfiguration cfg = getConfiguration(igniteInstanceName);
+
+        cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Object, Object> cacheConfiguration() {
+        CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>();
+
+        cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+
+        cfg.setCacheMode(CacheMode.PARTITIONED);
+
+        cfg.setBackups(1);
+
+        cfg.setCopyOnRead(false);
+
+        cfg.setName(CACHE_NAME);
+
+        return cfg;
+    }
+
+    /**
+     * @return Near cache configuration.
+     */
+    private NearCacheConfiguration<Object, Object> nearCacheConfiguration() {
+        NearCacheConfiguration<Object, Object> cfg = new NearCacheConfiguration<>();
+
+        cfg.setNearEvictionPolicy(new LruEvictionPolicy<>(25000));
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalPeekAfterPrimaryNodeLeft() throws Exception {
+        try {
+            Ignite crd = startGrid("coordinator", getConfiguration("coordinator"));
+
+            Ignite client = startGrid("client", getClientConfiguration("client"));
+
+            Ignite srvNode = startGrid("server", getConfiguration("server"));
+
+            awaitPartitionMapExchange();
+
+            IgniteCache<Object, Object> cache = srvNode.getOrCreateCache(cacheConfiguration());
+
+            IgniteCache<Object, Object> nearCache = client.createNearCache(CACHE_NAME, nearCacheConfiguration());
+
+            UUID serverNodeId = srvNode.cluster().localNode().id();
+
+            int remoteKey = 0;
+            for (; ; remoteKey++) {
+                if (crd.affinity(CACHE_NAME).mapKeyToNode(remoteKey).id().equals(serverNodeId))
+                    break;
+            }
+
+            cache.put(remoteKey, remoteKey);
+
+            Object value = nearCache.localPeek(remoteKey, NEAR);
+
+            assertNull("The value should not be loaded from a remote node.", value);
+
+            nearCache.get(remoteKey);
+
+            value = nearCache.localPeek(remoteKey, NEAR);
+
+            assertNotNull("The returned value should not be null.", value);
+
+            srvNode.close();
+
+            awaitPartitionMapExchange();
+
+            value = nearCache.localPeek(remoteKey, NEAR);
+
+            assertNull("The value should not be loaded from a remote node.", value);
+
+            value = nearCache.get(remoteKey);
+
+            assertNotNull("The value should be loaded from a remote node.", value);
+
+            value = nearCache.localPeek(remoteKey, NEAR);
+
+            assertNotNull("The returned value should not be null.", value);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/85027e7f/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 6f5b710..5ce213e 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
@@ -87,6 +87,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePa
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEvictionEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearReadersSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearClientHitTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearEvictionEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearJobExecutionSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearMultiGetSelfTest;
@@ -242,7 +243,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheNearPartitionedClearSelfTest.class));
 
         suite.addTest(new TestSuite(GridCacheOffheapUpdateSelfTest.class));
-
+        suite.addTest(new TestSuite(GridCacheNearClientHitTest.class));
         suite.addTest(new TestSuite(GridCacheNearPrimarySyncSelfTest.class));
         suite.addTest(new TestSuite(GridCacheColocatedPrimarySyncSelfTest.class));