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/07/07 08:07:24 UTC

[12/18] ignite git commit: IGNITE-5528 - IS_EVICT_DISABLED flag is not cleared on cache store exception. This closes #2183.

IGNITE-5528 - IS_EVICT_DISABLED flag is not cleared on cache store exception. This closes #2183.


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

Branch: refs/heads/ignite-gg-12306-1
Commit: 4a169dc5625f6f9794c17bfcf5bc1a318c91a996
Parents: 9675061
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jun 20 07:59:09 2017 +0300
Committer: Andrey V. Mashenkov <an...@gmail.com>
Committed: Thu Jul 6 14:37:59 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  62 ++++---
 .../processors/cache/GridCacheEntryEx.java      |   3 +-
 .../processors/cache/GridCacheMapEntry.java     |   2 +-
 ...titionEvictionDuringReadThroughSelfTest.java | 160 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite5.java       |   3 +
 5 files changed, 205 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4a169dc5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
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 5937a48..9213be3 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
@@ -2086,35 +2086,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                                     }
                                 });
 
-                                if (loaded.size() != loadKeys.size()) {
-                                    boolean needTouch =
-                                        tx0 == null || (!tx0.implicit() && tx0.isolation() == READ_COMMITTED);
-
-                                    for (Map.Entry<KeyCacheObject, EntryGetResult> e : loadKeys.entrySet()) {
-                                        if (loaded.contains(e.getKey()))
-                                            continue;
-
-                                        if (needTouch || e.getValue().reserved()) {
-                                            GridCacheEntryEx entry = peekEx(e.getKey());
-
-                                            if (entry != null) {
-                                                if (e.getValue().reserved())
-                                                    entry.clearReserveForLoad(e.getValue().version());
-
-                                                if (needTouch)
-                                                    ctx.evicts().touch(entry, topVer);
-                                            }
-                                        }
-                                    }
-                                }
+                                clearReservationsIfNeeded(topVer, loadKeys, loaded, tx0);
 
                                 return map;
                             }
                         }), true),
                         new C2<Map<K, V>, Exception, IgniteInternalFuture<Map<K, V>>>() {
                             @Override public IgniteInternalFuture<Map<K, V>> apply(Map<K, V> map, Exception e) {
-                                if (e != null)
+                                if (e != null) {
+                                    clearReservationsIfNeeded(topVer, loadKeys, loaded, tx0);
+
                                     return new GridFinishedFuture<>(e);
+                                }
 
                                 if (tx0 == null || (!tx0.implicit() && tx0.isolation() == READ_COMMITTED)) {
                                     Collection<KeyCacheObject> notFound = new HashSet<>(loadKeys.keySet());
@@ -2181,6 +2164,41 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
     }
 
+    /**
+     * @param topVer Affinity topology version for which load was performed.
+     * @param loadKeys Keys to load.
+     * @param loaded Actually loaded keys.
+     * @param tx0 Transaction within which the load was run, if any.
+     */
+    private void clearReservationsIfNeeded(
+        AffinityTopologyVersion topVer,
+        Map<KeyCacheObject, EntryGetResult> loadKeys,
+        Collection<KeyCacheObject> loaded,
+        IgniteTxLocalAdapter tx0
+    ) {
+        if (loaded.size() != loadKeys.size()) {
+            boolean needTouch =
+                tx0 == null || (!tx0.implicit() && tx0.isolation() == READ_COMMITTED);
+
+            for (Map.Entry<KeyCacheObject, EntryGetResult> e : loadKeys.entrySet()) {
+                if (loaded.contains(e.getKey()))
+                    continue;
+
+                if (needTouch || e.getValue().reserved()) {
+                    GridCacheEntryEx entry = peekEx(e.getKey());
+
+                    if (entry != null) {
+                        if (e.getValue().reserved())
+                            entry.clearReserveForLoad(e.getValue().version());
+
+                        if (needTouch)
+                            ctx.evicts().touch(entry, topVer);
+                    }
+                }
+            }
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public final V getAndPut(K key, V val) throws IgniteCheckedException {
         return getAndPut(key, val, null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a169dc5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
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 4e52680..5aabd30 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
@@ -332,9 +332,8 @@ public interface GridCacheEntryEx {
 
     /**
      * @param ver Expected entry version.
-     * @throws IgniteCheckedException If failed.
      */
-    public void clearReserveForLoad(GridCacheVersion ver) throws IgniteCheckedException;
+    public void clearReserveForLoad(GridCacheVersion ver);
 
     /**
      * Reloads entry from underlying storage.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a169dc5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
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 eea8935..673945a 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
@@ -2699,7 +2699,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized void clearReserveForLoad(GridCacheVersion ver) throws IgniteCheckedException {
+    @Override public synchronized void clearReserveForLoad(GridCacheVersion ver) {
         if (obsoleteVersionExtras() != null)
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a169dc5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.java
new file mode 100644
index 0000000..d5351f7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.java
@@ -0,0 +1,160 @@
+/*
+ * 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 java.util.LinkedHashSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.CreatedExpiryPolicy;
+import javax.cache.expiry.Duration;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class GridCachePartitionEvictionDuringReadThroughSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<Integer, Integer> ccfg =
+            new CacheConfiguration<Integer, Integer>()
+                .setName("config")
+                .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+                .setBackups(0) // No need for backup, just load from the store if needed
+                .setCacheStoreFactory(new CacheStoreFactory())
+                .setEvictionPolicy(new LruEvictionPolicy(100))
+                .setNearConfiguration(new NearCacheConfiguration<Integer, Integer>()
+                .setNearEvictionPolicy(new LruEvictionPolicy<Integer, Integer>()));
+
+        ccfg.setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.MINUTES, 1)))
+            .setReadThrough(true)
+            .setWriteThrough(false);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testPartitionRent() throws Exception {
+        startGrid(0);
+
+        final AtomicBoolean done = new AtomicBoolean();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Integer>() {
+            @Override
+            public Integer call() throws Exception {
+                LinkedHashSet<Integer> set = new LinkedHashSet<>();
+
+                set.add(1);
+                set.add(2);
+                set.add(3);
+                set.add(4);
+                set.add(5);
+
+                while (!done.get()) {
+                    try {
+                        grid(0).<Integer, Integer>cache("config").getAll(set);
+                    }
+                    catch (Throwable ignore) {
+                        // No-op.
+                    }
+                }
+
+                return null;
+            }
+        }, 4, "loader");
+
+        IgniteInternalFuture<Void> startFut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                for (int i = 1; i < 5; i++) {
+                    startGrid(i);
+
+                    awaitPartitionMapExchange();
+                }
+
+                return null;
+            }
+        });
+
+        startFut.get();
+
+        done.set(true);
+
+        fut.get();
+    }
+
+    /**
+     *
+     */
+    private static class CacheStoreFactory implements Factory<CacheStore<Integer, Integer>> {
+        /** {@inheritDoc} */
+        @Override public CacheStore<Integer, Integer> create() {
+            return new HangingCacheStore();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class HangingCacheStore extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private CountDownLatch releaseLatch = new CountDownLatch(1);
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            if (key == 3)
+                throw new CacheLoaderException();
+
+            return key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry) throws CacheWriterException {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a169dc5/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
index 1ab516a..1395b95 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheStoreCollectionTes
 import org.apache.ignite.internal.processors.cache.PartitionsExchangeOnDiscoveryHistoryOverflowTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest;
+import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionEvictionDuringReadThroughSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheGroupsPartitionLossPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCachePartitionLossPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest;
@@ -92,6 +93,8 @@ public class IgniteCacheTestSuite5 extends TestSuite {
 
         suite.addTestSuite(GridCachePartitionExchangeManagerHistSizeTest.class);
 
+        suite.addTestSuite(GridCachePartitionEvictionDuringReadThroughSelfTest.class);
+
         return suite;
     }
 }