You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2016/11/15 09:39:39 UTC

ignite git commit: IGNITE-4134 .NET: Add CacheConfiguration.ExpiryPolicyFactory

Repository: ignite
Updated Branches:
  refs/heads/master 0f2a3028d -> 085872ec0


IGNITE-4134 .NET: Add CacheConfiguration.ExpiryPolicyFactory

This closes #1230


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

Branch: refs/heads/master
Commit: 085872ec0541401080002aa7f2e33d87e5bb285b
Parents: 0f2a302
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 15 12:39:31 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Nov 15 12:39:31 2016 +0300

----------------------------------------------------------------------
 .../platform/cache/PlatformCache.java           |  74 +---------
 .../cache/expiry/PlatformExpiryPolicy.java      |  93 +++++++++++++
 .../expiry/PlatformExpiryPolicyFactory.java     |  76 +++++++++++
 .../utils/PlatformConfigurationUtils.java       |  39 ++++++
 .../Cache/CacheAbstractTest.cs                  |  48 +++++--
 .../Cache/CacheConfigurationTest.cs             |  23 +++-
 .../IgniteConfigurationSerializerTest.cs        |  16 +++
 .../Apache.Ignite.Core.csproj                   |   2 +
 .../Cache/Configuration/CacheConfiguration.cs   |  12 ++
 .../IgniteConfigurationSection.xsd              |  12 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  41 +-----
 .../Impl/Cache/Expiry/ExpiryPolicyFactory.cs    |  46 +++++++
 .../Impl/Cache/Expiry/ExpiryPolicySerializer.cs | 134 +++++++++++++++++++
 13 files changed, 493 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index 6f23682..677971c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicy;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryProxy;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformFieldsQueryCursor;
@@ -858,7 +859,7 @@ public class PlatformCache extends PlatformAbstractTarget {
                 long update = reader.readLong();
                 long access = reader.readLong();
 
-                IgniteCache cache0 = rawCache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access));
+                IgniteCache cache0 = rawCache.withExpiryPolicy(new PlatformExpiryPolicy(create, update, access));
 
                 return copy(cache0, keepBinary);
             }
@@ -1417,77 +1418,6 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /**
-     * Interop expiry policy.
-     */
-    private static class InteropExpiryPolicy implements ExpiryPolicy {
-        /** Duration: unchanged. */
-        private static final long DUR_UNCHANGED = -2;
-
-        /** Duration: eternal. */
-        private static final long DUR_ETERNAL = -1;
-
-        /** Duration: zero. */
-        private static final long DUR_ZERO = 0;
-
-        /** Expiry for create. */
-        private final Duration create;
-
-        /** Expiry for update. */
-        private final Duration update;
-
-        /** Expiry for access. */
-        private final Duration access;
-
-        /**
-         * Constructor.
-         *
-         * @param create Expiry for create.
-         * @param update Expiry for update.
-         * @param access Expiry for access.
-         */
-        private InteropExpiryPolicy(long create, long update, long access) {
-            this.create = convert(create);
-            this.update = convert(update);
-            this.access = convert(access);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Duration getExpiryForCreation() {
-            return create;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Duration getExpiryForUpdate() {
-            return update;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Duration getExpiryForAccess() {
-            return access;
-        }
-
-        /**
-         * Convert encoded duration to actual duration.
-         *
-         * @param dur Encoded duration.
-         * @return Actual duration.
-         */
-        private static Duration convert(long dur) {
-            if (dur == DUR_UNCHANGED)
-                return null;
-            else if (dur == DUR_ETERNAL)
-                return Duration.ETERNAL;
-            else if (dur == DUR_ZERO)
-                return Duration.ZERO;
-            else {
-                assert dur > 0;
-
-                return new Duration(TimeUnit.MILLISECONDS, dur);
-            }
-        }
-    }
-
-    /**
      * Listenable around CompletionListener.
      */
     private static class CompletionListenable implements PlatformListenable, CompletionListener {

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicy.java
new file mode 100644
index 0000000..d86d889
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicy.java
@@ -0,0 +1,93 @@
+/*
+ * 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.platform.cache.expiry;
+
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Platform expiry policy.
+ */
+public class PlatformExpiryPolicy implements ExpiryPolicy {
+    /** Duration: unchanged. */
+    private static final long DUR_UNCHANGED = -2;
+
+    /** Duration: eternal. */
+    private static final long DUR_ETERNAL = -1;
+
+    /** Duration: zero. */
+    private static final long DUR_ZERO = 0;
+
+    /** Expiry for create. */
+    private final Duration create;
+
+    /** Expiry for update. */
+    private final Duration update;
+
+    /** Expiry for access. */
+    private final Duration access;
+
+    /**
+     * Constructor.
+     *
+     * @param create Expiry for create.
+     * @param update Expiry for update.
+     * @param access Expiry for access.
+     */
+    public PlatformExpiryPolicy(long create, long update, long access) {
+        this.create = convert(create);
+        this.update = convert(update);
+        this.access = convert(access);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Duration getExpiryForCreation() {
+        return create;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Duration getExpiryForUpdate() {
+        return update;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Duration getExpiryForAccess() {
+        return access;
+    }
+
+    /**
+     * Convert encoded duration to actual duration.
+     *
+     * @param dur Encoded duration.
+     * @return Actual duration.
+     */
+    private static Duration convert(long dur) {
+        if (dur == DUR_UNCHANGED)
+            return null;
+        else if (dur == DUR_ETERNAL)
+            return Duration.ETERNAL;
+        else if (dur == DUR_ZERO)
+            return Duration.ZERO;
+        else {
+            assert dur > 0;
+
+            return new Duration(TimeUnit.MILLISECONDS, dur);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicyFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicyFactory.java
new file mode 100644
index 0000000..13f13f7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/expiry/PlatformExpiryPolicyFactory.java
@@ -0,0 +1,76 @@
+/*
+ * 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.platform.cache.expiry;
+
+import javax.cache.configuration.Factory;
+
+/**
+ * Platform expiry policy factory.
+ */
+public class PlatformExpiryPolicyFactory implements Factory<PlatformExpiryPolicy> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final long create;
+
+    /** */
+    private final long update;
+
+    /** */
+    private final long access;
+
+    /**
+     * Ctor.
+     *
+     * @param create Expiry for create.
+     * @param update Expiry for update.
+     * @param access Expiry for access.
+     */
+    public PlatformExpiryPolicyFactory(long create, long update, long access) {
+        this.create = create;
+        this.update = update;
+        this.access = access;
+    }
+
+    /**
+     * @return Create expiry.
+     */
+    public long getCreate() {
+        return create;
+    }
+
+    /**
+     * @return Update expiry.
+     */
+    public long getUpdate() {
+        return update;
+    }
+
+    /**
+     * @return Access expiry.
+     */
+    public long getAccess() {
+        return access;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformExpiryPolicy create() {
+        return new PlatformExpiryPolicy(create, update, access);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 83d1bf6..83cad82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -42,6 +42,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.binary.*;
 import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction;
+import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory;
 import org.apache.ignite.platform.dotnet.*;
 import org.apache.ignite.spi.communication.CommunicationSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
@@ -57,6 +58,8 @@ import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpiMBean;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.ExpiryPolicy;
 import java.lang.management.ManagementFactory;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -186,11 +189,46 @@ public class PlatformConfigurationUtils {
 
         ccfg.setEvictionPolicy(readEvictionPolicy(in));
         ccfg.setAffinity(readAffinityFunction(in));
+        ccfg.setExpiryPolicyFactory(readExpiryPolicyFactory(in));
 
         return ccfg;
     }
 
     /**
+     * Reads the expiry policy factory.
+     *
+     * @param in Reader.
+     * @return Expiry policy factory.
+     */
+    private static Factory<? extends ExpiryPolicy> readExpiryPolicyFactory(BinaryRawReader in) {
+        if (!in.readBoolean())
+            return null;
+
+        return new PlatformExpiryPolicyFactory(in.readLong(), in.readLong(), in.readLong());
+    }
+
+    /**
+     * Writes the policy factory.
+     *
+     * @param out Writer.
+     */
+    private static void writeExpiryPolicyFactory(BinaryRawWriter out, Factory<? extends ExpiryPolicy> factory) {
+        if (!(factory instanceof PlatformExpiryPolicyFactory)) {
+            out.writeBoolean(false);
+
+            return;
+        }
+
+        out.writeBoolean(true);
+
+        PlatformExpiryPolicyFactory f = (PlatformExpiryPolicyFactory)factory;
+
+        out.writeLong(f.getCreate());
+        out.writeLong(f.getUpdate());
+        out.writeLong(f.getAccess());
+    }
+
+    /**
      * Reads the near config.
      *
      * @param in Stream.
@@ -753,6 +791,7 @@ public class PlatformConfigurationUtils {
 
         writeEvictionPolicy(writer, ccfg.getEvictionPolicy());
         writeAffinityFunction(writer, ccfg.getAffinity());
+        writeExpiryPolicyFactory(writer, ccfg.getExpiryPolicyFactory());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 6910c54..98a2c89 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// ReSharper disable MissingSerializationAttribute
 namespace Apache.Ignite.Core.Tests.Cache
 {
     using System;
@@ -30,6 +31,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Cache.Expiry;
     using Apache.Ignite.Core.Tests.Query;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
@@ -40,14 +42,6 @@ namespace Apache.Ignite.Core.Tests.Cache
     class CacheTestKey
     {
         /// <summary>
-        /// Default constructor.
-        /// </summary>
-        public CacheTestKey()
-        {
-            // No-op.
-        }
-
-        /// <summary>
         /// Constructor.
         /// </summary>
         /// <param name="id">ID.</param>
@@ -872,6 +866,32 @@ namespace Apache.Ignite.Core.Tests.Cache
         [Test]
         public void TestWithExpiryPolicy()
         {
+            TestWithExpiryPolicy((cache, policy) => cache.WithExpiryPolicy(policy), true);
+        }
+
+        /// <summary>
+        /// Expiry policy tests.
+        /// </summary>
+        [Test]
+        public void TestCacheConfigurationExpiryPolicy()
+        {
+            TestWithExpiryPolicy((cache, policy) =>
+            {
+                var cfg = cache.GetConfiguration();
+
+                cfg.Name = string.Format("expiryPolicyCache_{0}_{1}", GetType().Name, policy.GetHashCode());
+                cfg.ExpiryPolicyFactory = new ExpiryPolicyFactory(policy);
+
+                return cache.Ignite.CreateCache<int, int>(cfg);
+            }, false);
+        }
+
+        /// <summary>
+        /// Expiry policy tests.
+        /// </summary>
+        public void TestWithExpiryPolicy(Func<ICache<int, int>, IExpiryPolicy, ICache<int, int>> withPolicyFunc, 
+            bool origCache)
+        {
             ICache<int, int> cache0 = Cache(0);
             
             int key0;
@@ -889,7 +909,8 @@ namespace Apache.Ignite.Core.Tests.Cache
             }
             
             // Test unchanged expiration.
-            ICache<int, int> cache = cache0.WithExpiryPolicy(new ExpiryPolicy(null, null, null));
+            ICache<int, int> cache = withPolicyFunc(cache0, new ExpiryPolicy(null, null, null));
+            cache0 = origCache ? cache0 : cache;
 
             cache.Put(key0, key0);
             cache.Put(key1, key1);
@@ -912,7 +933,8 @@ namespace Apache.Ignite.Core.Tests.Cache
             cache0.RemoveAll(new List<int> { key0, key1 });
 
             // Test eternal expiration.
-            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.MaxValue, TimeSpan.MaxValue, TimeSpan.MaxValue));
+            cache = withPolicyFunc(cache0, new ExpiryPolicy(TimeSpan.MaxValue, TimeSpan.MaxValue, TimeSpan.MaxValue));
+            cache0 = origCache ? cache0 : cache;
 
             cache.Put(key0, key0);
             cache.Put(key1, key1);
@@ -935,8 +957,9 @@ namespace Apache.Ignite.Core.Tests.Cache
             cache0.RemoveAll(new List<int> { key0, key1 });
 
             // Test regular expiration.
-            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.FromMilliseconds(100),
+            cache = withPolicyFunc(cache0, new ExpiryPolicy(TimeSpan.FromMilliseconds(100),
                 TimeSpan.FromMilliseconds(100), TimeSpan.FromMilliseconds(100)));
+            cache0 = origCache ? cache0 : cache;
 
             cache.Put(key0, key0);
             cache.Put(key1, key1);
@@ -3001,7 +3024,10 @@ namespace Apache.Ignite.Core.Tests.Cache
                 Assert.IsInstanceOf<CacheEntryProcessorException>(ex);
 
                 if (string.IsNullOrEmpty(containsText))
+                {
+                    Assert.IsNotNull(ex.InnerException);
                     Assert.AreEqual(AddArgCacheEntryProcessor.ExceptionText, ex.InnerException.Message);
+                }
                 else
                     Assert.IsTrue(ex.ToString().Contains(containsText));
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index 7bd1ba5..b02aae4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -26,6 +26,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Eviction;
+    using Apache.Ignite.Core.Cache.Expiry;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Cache.Affinity;
@@ -251,6 +252,12 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(x.WriteBehindFlushFrequency, y.WriteBehindFlushFrequency);
             Assert.AreEqual(x.WriteBehindFlushSize, y.WriteBehindFlushSize);
 
+            if (x.ExpiryPolicyFactory != null)
+                Assert.AreEqual(x.ExpiryPolicyFactory.CreateInstance().GetType(),
+                    y.ExpiryPolicyFactory.CreateInstance().GetType());
+            else
+                Assert.IsNull(y.ExpiryPolicyFactory);
+
             AssertConfigsAreEqual(x.QueryEntities, y.QueryEntities);
             AssertConfigsAreEqual(x.NearConfiguration, y.NearConfiguration);
             AssertConfigsAreEqual(x.EvictionPolicy, y.EvictionPolicy);
@@ -555,7 +562,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                 {
                     Partitions = 513,
                     ExcludeNeighbors = true
-                }
+                },
+                ExpiryPolicyFactory = new ExpiryFactory()
             };
         }
         /// <summary>
@@ -714,5 +722,18 @@ namespace Apache.Ignite.Core.Tests.Cache
             /// </summary>
             public int Foo { get; set; }
         }
+
+
+        /// <summary>
+        /// Expiry policy factory.
+        /// </summary>
+        private class ExpiryFactory : IFactory<IExpiryPolicy>
+        {
+            /** <inheritdoc /> */
+            public IExpiryPolicy CreateInstance()
+            {
+                return new ExpiryPolicy(null, null, null);
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 898f12a..4584530 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -36,6 +36,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Eviction;
+    using Apache.Ignite.Core.Cache.Expiry;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Communication.Tcp;
@@ -103,6 +104,7 @@ namespace Apache.Ignite.Core.Tests
                                         <evictionPolicy type='FifoEvictionPolicy' batchSize='10' maxSize='20' maxMemorySize='30' />
                                     </nearConfiguration>
                                     <affinityFunction type='RendezvousAffinityFunction' partitions='99' excludeNeighbors='true' />
+                                    <expiryPolicyFactory type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyPolicyFactory, Apache.Ignite.Core.Tests' />
                                 </cacheConfiguration>
                                 <cacheConfiguration name='secondCache' />
                             </cacheConfiguration>
@@ -148,6 +150,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(CacheMode.Replicated, cacheCfg.CacheMode);
             Assert.IsTrue(cacheCfg.ReadThrough);
             Assert.IsTrue(cacheCfg.WriteThrough);
+            Assert.IsInstanceOf<MyPolicyFactory>(cacheCfg.ExpiryPolicyFactory);
 
             var queryEntity = cacheCfg.QueryEntities.Single();
             Assert.AreEqual(typeof(int), queryEntity.KeyType);
@@ -636,6 +639,7 @@ namespace Apache.Ignite.Core.Tests
                             ExcludeNeighbors = true,
                             Partitions = 48
                         },
+                        ExpiryPolicyFactory = new MyPolicyFactory()
                     }
                 },
                 ClientMode = true,
@@ -889,5 +893,17 @@ namespace Apache.Ignite.Core.Tests
                 throw new NotImplementedException();
             }
         }
+
+        /// <summary>
+        /// Test factory.
+        /// </summary>
+        public class MyPolicyFactory : IFactory<IExpiryPolicy>
+        {
+            /** <inheritdoc /> */
+            public IExpiryPolicy CreateInstance()
+            {
+                throw new NotImplementedException();
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 64d7388..f945efe 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -178,6 +178,8 @@
     <Compile Include="Impl\Binary\SerializableSerializer.cs" />
     <Compile Include="Impl\Binary\BinaryWriterExtensions.cs" />
     <Compile Include="Impl\Cache\Affinity\AffinityFunctionBase.cs" />
+    <Compile Include="Impl\Cache\Expiry\ExpiryPolicyFactory.cs" />
+    <Compile Include="Impl\Cache\Expiry\ExpiryPolicySerializer.cs" />
     <Compile Include="Impl\Cache\ICacheLockInternal.cs" />
     <Compile Include="Impl\Common\Platform.cs" />
     <Compile Include="Impl\Binary\UserSerializerProxy.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
index c0c4722..3436cd2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -32,10 +32,12 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using Apache.Ignite.Core.Cache.Affinity.Fair;
     using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
     using Apache.Ignite.Core.Cache.Eviction;
+    using Apache.Ignite.Core.Cache.Expiry;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Cache.Affinity;
+    using Apache.Ignite.Core.Impl.Cache.Expiry;
     using Apache.Ignite.Core.Log;
 
     /// <summary>
@@ -279,6 +281,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
 
             EvictionPolicy = EvictionPolicyBase.Read(reader);
             AffinityFunction = AffinityFunctionSerializer.Read(reader);
+            ExpiryPolicyFactory = ExpiryPolicySerializer.ReadPolicyFactory(reader);
         }
 
         /// <summary>
@@ -352,6 +355,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
 
             EvictionPolicyBase.Write(writer, EvictionPolicy);
             AffinityFunctionSerializer.Write(writer, AffinityFunction);
+            ExpiryPolicySerializer.WritePolicyFactory(writer, ExpiryPolicyFactory);
         }
 
         /// <summary>
@@ -692,5 +696,13 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <see cref="RendezvousAffinityFunction"/>, <see cref="FairAffinityFunction"/>.
         /// </summary>
         public IAffinityFunction AffinityFunction { get; set; }
+
+        /// <summary>
+        /// Gets or sets the factory for <see cref="IExpiryPolicy"/> to be used for all cache operations, 
+        /// unless <see cref="ICache{TK,TV}.WithExpiryPolicy"/> is called.
+        /// <para />
+        /// Default is null, which means no expiration.
+        /// </summary>
+        public IFactory<IExpiryPolicy> ExpiryPolicyFactory { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 06541da..81b2298 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -501,6 +501,18 @@
                                                 </xs:attribute>
                                             </xs:complexType>
                                         </xs:element>
+                                        <xs:element name="expiryPolicyFactory" minOccurs="0">
+                                            <xs:annotation>
+                                                <xs:documentation>Factory for IExpiryPolicy to be used for all cache operations, unless ICache.WithExpiryPolicy is called. Default is null, which means no expiration.</xs:documentation>
+                                            </xs:annotation>
+                                            <xs:complexType>
+                                                <xs:attribute name="type" type="xs:string" use="required">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Assembly-qualified type name.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                            </xs:complexType>
+                                        </xs:element>
                                     </xs:all>
                                     <xs:attribute name="name" type="xs:string">
                                         <xs:annotation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
index 359611d..556b8bb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
@@ -30,6 +30,7 @@ namespace Apache.Ignite.Core.Impl.Cache
     using Apache.Ignite.Core.Cache.Query.Continuous;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Cache.Expiry;
     using Apache.Ignite.Core.Impl.Cache.Query;
     using Apache.Ignite.Core.Impl.Cache.Query.Continuous;
     using Apache.Ignite.Core.Impl.Common;
@@ -41,15 +42,6 @@ namespace Apache.Ignite.Core.Impl.Cache
     [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
     internal class CacheImpl<TK, TV> : PlatformTarget, ICache<TK, TV>, ICacheInternal, ICacheLockInternal
     {
-        /** Duration: unchanged. */
-        private const long DurUnchanged = -2;
-
-        /** Duration: eternal. */
-        private const long DurEternal = -1;
-
-        /** Duration: zero. */
-        private const long DurZero = 0;
-
         /** Ignite instance. */
         private readonly Ignite _ignite;
         
@@ -193,40 +185,11 @@ namespace Apache.Ignite.Core.Impl.Cache
         {
             IgniteArgumentCheck.NotNull(plc, "plc");
 
-            long create = ConvertDuration(plc.GetExpiryForCreate());
-            long update = ConvertDuration(plc.GetExpiryForUpdate());
-            long access = ConvertDuration(plc.GetExpiryForAccess());
-
-            IUnmanagedTarget cache0 = DoOutOpObject((int)CacheOp.WithExpiryPolicy, w =>
-            {
-                w.WriteLong(create);
-                w.WriteLong(update);
-                w.WriteLong(access);
-            });
+            var cache0 = DoOutOpObject((int)CacheOp.WithExpiryPolicy, w => ExpiryPolicySerializer.WritePolicy(w, plc));
 
             return new CacheImpl<TK, TV>(_ignite, cache0, Marshaller, _flagSkipStore, _flagKeepBinary, _flagNoRetries);
         }
 
-        /// <summary>
-        /// Convert TimeSpan to duration recognizable by Java.
-        /// </summary>
-        /// <param name="dur">.Net duration.</param>
-        /// <returns>Java duration in milliseconds.</returns>
-        private static long ConvertDuration(TimeSpan? dur)
-        {
-            if (dur.HasValue)
-            {
-                if (dur.Value == TimeSpan.MaxValue)
-                    return DurEternal;
-
-                long dur0 = (long)dur.Value.TotalMilliseconds;
-
-                return dur0 > 0 ? dur0 : DurZero;
-            }
-            
-            return DurUnchanged;
-        }
-
         /** <inheritDoc /> */
         public bool IsKeepBinary
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicyFactory.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicyFactory.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicyFactory.cs
new file mode 100644
index 0000000..fd241e7
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicyFactory.cs
@@ -0,0 +1,46 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Cache.Expiry
+{
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Factory.
+    /// </summary>
+    internal class ExpiryPolicyFactory : IFactory<IExpiryPolicy>
+    {
+        /** */
+        private readonly IExpiryPolicy _expiryPolicy;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ExpiryPolicyFactory"/> class.
+        /// </summary>
+        /// <param name="expiryPolicy">The expiry policy.</param>
+        public ExpiryPolicyFactory(IExpiryPolicy expiryPolicy)
+        {
+            _expiryPolicy = expiryPolicy;
+        }
+
+        /** <inheritdoc /> */
+        public IExpiryPolicy CreateInstance()
+        {
+            return _expiryPolicy;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/085872ec/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicySerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicySerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicySerializer.cs
new file mode 100644
index 0000000..6f090d7
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Expiry/ExpiryPolicySerializer.cs
@@ -0,0 +1,134 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Cache.Expiry
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Writer for <see cref="IExpiryPolicy"/>.
+    /// </summary>
+    internal static class ExpiryPolicySerializer
+    {
+        /** Duration: unchanged. */
+        private const long DurUnchanged = -2;
+
+        /** Duration: eternal. */
+        private const long DurEternal = -1;
+
+        /** Duration: zero. */
+        private const long DurZero = 0;
+
+        /// <summary>
+        /// Writes the policy.
+        /// </summary>
+        public static void WritePolicy(IBinaryRawWriter writer, IExpiryPolicy plc)
+        {
+            Debug.Assert(plc != null);
+            Debug.Assert(writer != null);
+
+            writer.WriteLong(ConvertDuration(plc.GetExpiryForCreate()));
+            writer.WriteLong(ConvertDuration(plc.GetExpiryForUpdate()));
+            writer.WriteLong(ConvertDuration(plc.GetExpiryForAccess()));
+        }
+
+        /// <summary>
+        /// Reads the policy.
+        /// </summary>
+        public static IExpiryPolicy ReadPolicy(IBinaryRawReader reader)
+        {
+            return new ExpiryPolicy(ConvertDuration(reader.ReadLong()), ConvertDuration(reader.ReadLong()),
+                ConvertDuration(reader.ReadLong()));
+        }
+
+        /// <summary>
+        /// Writes the policy factory.
+        /// </summary>
+        public static void WritePolicyFactory(IBinaryRawWriter writer, IFactory<IExpiryPolicy> factory)
+        {
+            Debug.Assert(writer != null);
+
+            if (factory != null)
+            {
+                writer.WriteBoolean(true);
+                var expiryPolicy = factory.CreateInstance();
+
+                if (expiryPolicy == null)
+                    throw new IgniteException("ExpiryPolicyFactory should return non-null result.");
+
+                WritePolicy(writer, expiryPolicy);
+            }
+            else
+                writer.WriteBoolean(false);
+        }
+
+        /// <summary>
+        /// Reads the expiry policy factory.
+        /// </summary>
+        public static IFactory<IExpiryPolicy> ReadPolicyFactory(IBinaryRawReader reader)
+        {
+            return reader.ReadBoolean() ? new ExpiryPolicyFactory(ReadPolicy(reader)) : null;
+        }
+
+        /// <summary>
+        /// Convert TimeSpan to duration recognizable by Java.
+        /// </summary>
+        /// <param name="dur">.NET duration.</param>
+        /// <returns>Java duration in milliseconds.</returns>
+        private static long ConvertDuration(TimeSpan? dur)
+        {
+            if (dur.HasValue)
+            {
+                if (dur.Value == TimeSpan.MaxValue)
+                    return DurEternal;
+
+                long dur0 = (long) dur.Value.TotalMilliseconds;
+
+                return dur0 > 0 ? dur0 : DurZero;
+            }
+
+            return DurUnchanged;
+        }
+
+        /// <summary>
+        /// Convert duration recognizable by Java to TimeSpan.
+        /// </summary>
+        /// <param name="dur">Java duration.</param>
+        /// <returns>.NET duration.</returns>
+        private static TimeSpan? ConvertDuration(long dur)
+        {
+            switch (dur)
+            {
+                case DurUnchanged:
+                    return null;
+
+                case DurEternal:
+                    return TimeSpan.MaxValue;
+
+                case DurZero:
+                    return TimeSpan.Zero;
+
+                default:
+                    return TimeSpan.FromMilliseconds(dur);
+            }
+        }
+    }
+}