You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2016/02/26 10:22:04 UTC

[03/13] ignite git commit: IGNITE-2707 - Fixed skipStore flag handling - Fixes #508.

IGNITE-2707 - Fixed skipStore flag handling - Fixes #508.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-1186
Commit: 711fe3783cb622ae134df899397505d2813d50ac
Parents: ff80f5d
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Feb 24 18:43:35 2016 -0800
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Feb 24 18:43:35 2016 -0800

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLockFuture.java      | 17 ++++-
 .../transactions/IgniteTxLocalAdapter.java      |  4 ++
 .../cache/GridCacheAbstractFullApiSelfTest.java | 65 ++++++++++++++++++++
 .../cache/GridCacheAbstractSelfTest.java        | 19 ++++++
 .../GridCacheBinaryObjectsAbstractSelfTest.java | 40 ++++++++++++
 5 files changed, 143 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/711fe378/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 07755e0..a33f01f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -1021,8 +1021,21 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             final GridCacheVersion ver = version();
 
             for (GridDhtCacheEntry entry : entries) {
-                if (!entry.hasValue())
-                    loadMap.put(entry.key(), entry);
+                try {
+                    entry.unswap(false);
+
+                    if (!entry.hasValue())
+                        loadMap.put(entry.key(), entry);
+                }
+                catch (GridCacheEntryRemovedException e) {
+                    assert false : "Should not get removed exception while holding lock on entry " +
+                        "[entry=" + entry + ", e=" + e + ']';
+                }
+                catch (IgniteCheckedException e) {
+                    onDone(e);
+
+                    return;
+                }
             }
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/711fe378/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index f35e2e4..1d77da5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -3779,6 +3779,10 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
             old.cached(entry);
             old.filters(filter);
 
+            // Keep old skipStore and keepBinary flags.
+            old.skipStore(skipStore);
+            old.keepBinary(keepBinary);
+
             // Update ttl if specified.
             if (drTtl >= 0L) {
                 assert drExpireTime >= 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/711fe378/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
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 3bd87cf..222a96b 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
@@ -261,6 +261,71 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             info("Grid " + i + ": " + grid(i).localNode().id());
     }
 
+    /**
+     * Checks that skipStore flag gets overriden inside a transaction.
+     *
+     * @throws Exception if failed.
+     */
+    public void testWriteThroughTx() {
+        if(isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-1088");
+
+        String key = "writeThroughKey";
+
+        map.remove(key);
+
+        try (final Transaction transaction = grid(0).transactions().txStart()) {
+            IgniteCache<String, Integer> cache = jcache(0);
+
+            // retrieve market type from the grid
+            Integer old = cache.withSkipStore().get(key);
+
+            assertNull(old);
+
+            // update the grid
+            cache.put(key, 2);
+
+            // finally commit the transaction
+            transaction.commit();
+        }
+
+        assertEquals(2, map.get(key));
+    }
+
+    /**
+     * Checks that skipStore flag gets overriden inside a transaction.
+     *
+     * @throws Exception if failed.
+     */
+    public void testNoReadThroughTx() {
+        if(isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-1088");
+
+        String key = "writeThroughKey";
+
+        IgniteCache<String, Integer> cache = jcache(0);
+
+        resetStore();
+
+        cache.put(key, 1);
+
+        putToStore(key, 2);
+
+        try (final Transaction transaction = grid(0).transactions().txStart()) {
+            Integer old = cache.get(key);
+
+            assertEquals((Integer)1, old);
+
+            // update the grid
+            cache.put(key, 2);
+
+            // finally commit the transaction
+            transaction.commit();
+        }
+
+        assertEquals(0, reads.get());
+    }
+
     /** {@inheritDoc} */
     @Override protected Ignite startGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
         if (cacheCfgMap == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/711fe378/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index b3d1384..b918f28 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -69,6 +69,15 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     /** Store map. */
     protected static final Map<Object, Object> map = new ConcurrentHashMap8<>();
 
+    /** Reads counter. */
+    protected static final AtomicInteger reads = new AtomicInteger();
+
+    /** Writes counter. */
+    protected static final AtomicInteger writes = new AtomicInteger();
+
+    /** Removes counter. */
+    protected static final AtomicInteger removes = new AtomicInteger();
+
     /** VM ip finder for TCP discovery. */
     protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
@@ -187,6 +196,10 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
      */
     protected void resetStore() {
         map.clear();
+
+        reads.set(0);
+        writes.set(0);
+        removes.set(0);
     }
 
     /**
@@ -301,14 +314,20 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
             }
 
             @Override public Object load(Object key) {
+                reads.incrementAndGet();
+
                 return map.get(key);
             }
 
             @Override public void write(javax.cache.Cache.Entry<? extends Object, ? extends Object> e) {
+                writes.incrementAndGet();
+
                 map.put(e.getKey(), e.getValue());
             }
 
             @Override public void delete(Object key) {
+                removes.incrementAndGet();
+
                 map.remove(key);
             }
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/711fe378/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index 271e3b5..36a9450 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import javax.cache.Cache;
 import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -60,6 +61,8 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -732,6 +735,30 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     }
 
     /**
+     * @throws Exception if failed.
+     */
+    public void testKeepBinaryTxOverwrite() throws Exception {
+        if (atomicityMode() != TRANSACTIONAL)
+            return;
+
+        IgniteCache<Integer, TestObject> cache = ignite(0).cache(null);
+
+        cache.put(0, new TestObject(1));
+
+        for (TransactionConcurrency conc : TransactionConcurrency.values()) {
+            for (TransactionIsolation iso : TransactionIsolation.values()) {
+                try (Transaction tx = ignite(0).transactions().txStart(conc, iso)) {
+                    cache.withKeepBinary().get(0);
+
+                    cache.invoke(0, new ObjectEntryProcessor());
+
+                    tx.commit();
+                }
+            }
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testLoadCache() throws Exception {
@@ -936,6 +963,19 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     }
 
     /**
+     * No-op entry processor.
+     */
+    private static class ObjectEntryProcessor implements EntryProcessor<Integer, TestObject, Boolean> {
+        @Override public Boolean process(MutableEntry<Integer, TestObject> entry, Object... args) throws EntryProcessorException {
+            TestObject obj = entry.getValue();
+
+            entry.setValue(new TestObject(obj.val));
+
+            return true;
+        }
+    }
+
+    /**
      *
      */
     private static class TestReferenceObject implements Binarylizable {