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 2016/12/09 11:27:46 UTC

[24/25] ignite git commit: ignite-3699: CreatedExpiryPolicy doesn't work if entry is loaded from store (partial fix)

ignite-3699: CreatedExpiryPolicy doesn't work if entry is loaded from store (partial fix)


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

Branch: refs/heads/ignite-4371
Commit: 27a81c865f6574f6c287dad6a401171bf0a9c53a
Parents: fa358cc
Author: Amir Akhmedov <am...@gmail.com>
Authored: Fri Dec 9 14:01:10 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 14:01:10 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  7 +++-
 .../processors/cache/GridCacheEntryEx.java      |  4 +-
 .../processors/cache/GridCacheMapEntry.java     | 17 +++++++--
 .../distributed/dht/GridDhtLockFuture.java      | 23 ++++++++++-
 .../dht/atomic/GridDhtAtomicCache.java          |  2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  2 +-
 .../processors/cache/GridCacheTestEntryEx.java  |  3 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java | 40 ++++++++++++++++++++
 8 files changed, 87 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/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 1db3931..88aa4e0 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
@@ -1882,7 +1882,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         @Nullable final UUID subjId,
         final String taskName,
         final boolean deserializeBinary,
-        @Nullable IgniteCacheExpiryPolicy expiry,
+        @Nullable final IgniteCacheExpiryPolicy expiry,
         final boolean skipVals,
         final boolean keepCacheObjects,
         boolean canRemap,
@@ -2017,7 +2017,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                                             GridCacheEntryEx entry = entryEx(key);
 
                                             try {
-                                                GridCacheVersion verSet = entry.versionedValue(cacheVal, ver, null);
+                                                GridCacheVersion verSet = entry.versionedValue(cacheVal,
+                                                    ver,
+                                                    null,
+                                                    expiry);
 
                                                 boolean set = verSet != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/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 d8194fc..9e9b496 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
@@ -725,13 +725,15 @@ public interface GridCacheEntryEx {
      * @param val New value.
      * @param curVer Version to match or {@code null} if match is not required.
      * @param newVer Version to set.
+     * @param loadExpiryPlc Expiry policy if entry is loaded from store.
      * @return Non null version if value was set.
      * @throws IgniteCheckedException If index could not be updated.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     public GridCacheVersion versionedValue(CacheObject val,
         @Nullable GridCacheVersion curVer,
-        @Nullable GridCacheVersion newVer)
+        @Nullable GridCacheVersion newVer,
+        @Nullable IgniteCacheExpiryPolicy loadExpiryPlc)
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/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 31baeda..52b779d 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
@@ -3551,7 +3551,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /** {@inheritDoc} */
     @Override public synchronized GridCacheVersion versionedValue(CacheObject val,
         GridCacheVersion curVer,
-        GridCacheVersion newVer)
+        GridCacheVersion newVer,
+        @Nullable IgniteCacheExpiryPolicy loadExpiryPlc)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
 
         checkObsolete();
@@ -3568,9 +3569,19 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 CacheObject old = rawGetOrUnmarshalUnlocked(false);
 
-                long ttl = ttlExtras();
+                long ttl;
+                long expTime;
 
-                long expTime = CU.toExpireTime(ttl);
+                if (loadExpiryPlc != null) {
+                    IgniteBiTuple<Long, Long> initTtlAndExpireTime = initialTtlAndExpireTime(loadExpiryPlc);
+
+                    ttl = initTtlAndExpireTime.get1();
+                    expTime = initTtlAndExpireTime.get2();
+                }
+                else {
+                    ttl = ttlExtras();
+                    expTime = expireTimeExtras();
+                }
 
                 // Detach value before index update.
                 val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/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 4566044..dd18d7a 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
@@ -28,6 +28,7 @@ import java.util.ListIterator;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
@@ -1059,10 +1060,28 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                             try {
                                 CacheObject val0 = cctx.toCacheObject(val);
 
+                                long ttl = CU.TTL_ETERNAL;
+                                long expireTime = CU.EXPIRE_TIME_ETERNAL;
+
+                                ExpiryPolicy expiry = cctx.expiry();
+
+                                if (expiry != null) {
+                                    ttl = CU.toTtl(expiry.getExpiryForCreation());
+
+                                    if (ttl == CU.TTL_ZERO)
+                                        expireTime = CU.expireTimeInPast();
+                                    else {
+                                        if (ttl == CU.TTL_NOT_CHANGED)
+                                            ttl = CU.TTL_ETERNAL;
+
+                                        expireTime = CU.toExpireTime(ttl);
+                                    }
+                                }
+
                                 entry0.initialValue(val0,
                                     ver,
-                                    0,
-                                    0,
+                                    ttl,
+                                    expireTime,
                                     false,
                                     topVer,
                                     GridDrType.DR_LOAD,

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 0e60ff4..b291bd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -2292,7 +2292,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         try {
                             GridCacheVersion ver = entry.version();
 
-                            entry.versionedValue(ctx.toCacheObject(v), null, ver);
+                            entry.versionedValue(ctx.toCacheObject(v), null, ver, null);
                         }
                         catch (GridCacheEntryRemovedException e) {
                             assert false : "Entry should not get obsolete while holding lock [entry=" + entry +

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/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 393fb1a..ba44655 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
@@ -475,7 +475,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                 GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
 
                                 try {
-                                    GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null);
+                                    GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null, null);
 
                                     boolean set = setVer != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 48621af..e3b4e9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -667,7 +667,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     /** @inheritDoc */
     @Override public GridCacheVersion versionedValue(CacheObject val,
         GridCacheVersion curVer,
-        GridCacheVersion newVer) {
+        GridCacheVersion newVer,
+        IgniteCacheExpiryPolicy loadExpiryPlc) {
         assert false;
 
         return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
index 78c59ac..58e6b02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.expiry;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import javax.cache.configuration.Factory;
 import javax.cache.expiry.Duration;
@@ -174,6 +176,44 @@ public abstract class IgniteCacheExpiryPolicyWithStoreAbstractTest extends Ignit
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testGetReadThrough() throws Exception {
+        IgniteCache<Integer, Integer> cache = jcache(0);
+
+        List<Integer> keys = new ArrayList<>();
+
+        keys.add(primaryKeys(cache, 1, 100_000).get(0));
+        // TODO https://issues.apache.org/jira/browse/IGNITE-3699
+        // TODO: test 'get' inside transactions, 'get' for cache.withAsyncPolicy.
+        //keys.add(backupKeys(cache, 1, 100_000).get(0));
+        //keys.add(nearKeys(cache, 1, 100_000).get(0));
+
+        for (Integer key : keys)
+            storeMap.put(key, 100);
+
+        try {
+            for (Integer key : keys) {
+                Integer res = cache.get(key);
+
+                assertEquals((Integer)100, res);
+
+                checkTtl(key, 500, true);
+
+                assertEquals((Integer)100, res);
+            }
+
+            U.sleep(600);
+
+            for (Integer key : keys)
+                checkExpired(key);
+        }
+        finally {
+            cache.removeAll();
+        }
+    }
+
+    /**
      * @param key Key.
      */
     private void checkExpired(Integer key) {