You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/24 08:22:09 UTC

[51/65] [abbrv] ignite git commit: IGNITE-5031 .NET: Added Partition Loss APIs

IGNITE-5031 .NET: Added Partition Loss APIs

ICache.GetLostPartitions, ICache.WithPartitionRecover, IIgnite.ResetLostPartitions, CacheConfiguration.PartitionLossPolicy


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

Branch: refs/heads/ignite-5024
Commit: ad7e4a092c9efc9cf7417437b7875dc91eee0f8a
Parents: 457c551
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Apr 21 17:08:52 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Apr 21 17:08:52 2017 +0300

----------------------------------------------------------------------
 .../platform/cache/PlatformCache.java           |  24 +-
 .../platform/cluster/PlatformClusterGroup.java  |  17 ++
 .../utils/PlatformConfigurationUtils.java       |   4 +
 .../ExpiryCacheHolderTest.cs                    |  10 +
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Cache/Affinity/AffinityFunctionTest.cs      |   2 +-
 .../Cache/CacheConfigurationTest.cs             |   3 +
 .../Cache/CacheTestAsyncWrapper.cs              |  16 +-
 .../Cache/PartitionLossTest.cs                  | 260 +++++++++++++++++++
 .../IgniteConfigurationSerializerTest.cs        |   7 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   5 +-
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Cache/Configuration/CacheConfiguration.cs   |  13 +
 .../Cache/Configuration/PartitionLossPolicy.cs  |  68 +++++
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  15 ++
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |  12 +
 .../IgniteConfigurationSection.xsd              |  15 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  48 +++-
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   5 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  27 ++
 .../Impl/Common/DelegateConverter.cs            |   1 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  16 +-
 22 files changed, 549 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 2847813..9a08b2b 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
@@ -67,6 +67,7 @@ import javax.cache.Cache;
 import javax.cache.integration.CompletionListener;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.EntryProcessorResult;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
@@ -198,7 +199,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     public static final int OP_CLEAR_CACHE = 41;
 
     /** */
-    public static final int OP_WITH_ASYNC = 42;
+    public static final int OP_WITH_PARTITION_RECOVER = 42;
 
     /** */
     public static final int OP_REMOVE_ALL2 = 43;
@@ -323,6 +324,9 @@ public class PlatformCache extends PlatformAbstractTarget {
     /** */
     public static final int OP_GLOBAL_METRICS = 83;
 
+    /** */
+    public static final int OP_GET_LOST_PARTITIONS = 84;
+
     /** Underlying JCache in binary mode. */
     private final IgniteCacheProxy cache;
 
@@ -974,6 +978,17 @@ public class PlatformCache extends PlatformAbstractTarget {
 
                 break;
 
+            case OP_GET_LOST_PARTITIONS:
+                Collection<Integer> parts = cache.lostPartitions();
+
+                writer.writeInt(parts.size());
+
+                for (int p : parts) {
+                    writer.writeInt(p);
+                }
+
+                break;
+
             default:
                 super.processOutStream(type, writer);
         }
@@ -982,11 +997,8 @@ public class PlatformCache extends PlatformAbstractTarget {
     /** {@inheritDoc} */
     @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         switch (type) {
-            case OP_WITH_ASYNC: {
-                if (cache.isAsync())
-                    return this;
-
-                return copy(rawCache.withAsync(), keepBinary);
+            case OP_WITH_PARTITION_RECOVER: {
+                return copy(rawCache.withPartitionRecover(), keepBinary);
             }
 
             case OP_WITH_KEEP_BINARY: {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index f49f477..3e14e7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.cluster;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.UUID;
 
@@ -104,6 +105,9 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
     private static final int OP_CACHE_METRICS = 24;
 
+    /** */
+    private static final int OP_RESET_LOST_PARTITIONS = 25;
+
     /** Projection. */
     private final ClusterGroupEx prj;
 
@@ -223,6 +227,19 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
             case OP_PING_NODE:
                 return pingNode(reader.readUuid()) ? TRUE : FALSE;
 
+            case OP_RESET_LOST_PARTITIONS:
+                int cnt = reader.readInt();
+
+                Collection<String> cacheNames = new ArrayList<>(cnt);
+
+                for (int i = 0; i < cnt; i++) {
+                    cacheNames.add(reader.readString());
+                }
+
+                platformCtx.kernalContext().grid().resetLostPartitions(cacheNames);
+
+                return TRUE;
+
             default:
                 return super.processInStreamOutLong(type, reader);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 0fe537f..4186eb9 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
@@ -41,6 +41,7 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
@@ -178,6 +179,8 @@ public class PlatformConfigurationUtils {
         if (memoryPolicyName != null)
             ccfg.setMemoryPolicyName(memoryPolicyName);
 
+        ccfg.setPartitionLossPolicy(PartitionLossPolicy.fromOrdinal((byte)in.readInt()));
+
         Object storeFactory = in.readObjectDetached();
 
         if (storeFactory != null)
@@ -801,6 +804,7 @@ public class PlatformConfigurationUtils {
         writer.writeBoolean(ccfg.isWriteThrough());
         writer.writeBoolean(ccfg.isStatisticsEnabled());
         writer.writeString(ccfg.getMemoryPolicyName());
+        writer.writeInt(ccfg.getPartitionLossPolicy().ordinal());
 
         if (ccfg.getCacheStoreFactory() instanceof PlatformDotNetCacheStoreFactoryNative)
             writer.writeObject(((PlatformDotNetCacheStoreFactoryNative)ccfg.getCacheStoreFactory()).getNativeFactory());

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs
index 9d1a2b5..2fe3309 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs
@@ -498,6 +498,16 @@ namespace Apache.Ignite.AspNet.Tests
             {
                 throw new NotImplementedException();
             }
+
+            public ICache<int, int> WithPartitionRecover()
+            {
+                throw new NotImplementedException();
+            }
+
+            public ICollection<int> GetLostPartitions()
+            {
+                throw new NotImplementedException();
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 232b033..f4f5e59 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -95,6 +95,7 @@
     <Compile Include="Cache\CacheTestKey.cs" />
     <Compile Include="Cache\NonSerializableCacheEntryProcessor.cs" />
     <Compile Include="Cache\NonSerializableException.cs" />
+    <Compile Include="Cache\PartitionLossTest.cs" />
     <Compile Include="Cache\Query\CacheDmlQueriesTest.cs" />
     <Compile Include="Cache\CacheAbstractTransactionalTest.cs" />
     <Compile Include="Cache\Query\CacheDmlQueriesTestSimpleName.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
index 22810da..7c6d779 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
@@ -220,7 +220,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Affinity
             }
 
             // Called on both nodes
-            TestUtils.WaitForCondition(() => RemovedNodes.Count > 0, 3000);
+            TestUtils.WaitForCondition(() => RemovedNodes.Count == 6, 3000);
             Assert.GreaterOrEqual(RemovedNodes.Count, 6);
             Assert.AreEqual(expectedNodeId, RemovedNodes.Distinct().Single());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 25ba43e..67184a6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -215,6 +215,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(CacheConfiguration.DefaultWriteBehindFlushSize, cfg.WriteBehindFlushSize);
             Assert.AreEqual(CacheConfiguration.DefaultWriteBehindFlushThreadCount, cfg.WriteBehindFlushThreadCount);
             Assert.AreEqual(CacheConfiguration.DefaultWriteBehindCoalescing, cfg.WriteBehindCoalescing);
+            Assert.AreEqual(CacheConfiguration.DefaultPartitionLossPolicy, cfg.PartitionLossPolicy);
         }
 
         /// <summary>
@@ -247,6 +248,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(x.WriteBehindFlushSize, y.WriteBehindFlushSize);
             Assert.AreEqual(x.EnableStatistics, y.EnableStatistics);
             Assert.AreEqual(x.MemoryPolicyName, y.MemoryPolicyName);
+            Assert.AreEqual(x.PartitionLossPolicy, y.PartitionLossPolicy);
 
             if (x.ExpiryPolicyFactory != null)
                 Assert.AreEqual(x.ExpiryPolicyFactory.CreateInstance().GetType(),
@@ -562,6 +564,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                 ExpiryPolicyFactory = new ExpiryFactory(),
                 EnableStatistics = true,
                 MemoryPolicyName = "myMemPolicy",
+                PartitionLossPolicy = PartitionLossPolicy.ReadOnlySafe,
                 PluginConfigurations = new[] { new MyPluginConfiguration() }
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs
index 6c8f0d6..5ba2fc9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs
@@ -542,6 +542,18 @@ namespace Apache.Ignite.Core.Tests.Cache
         }
 
         /** <inheritDoc /> */
+        public ICache<TK, TV> WithPartitionRecover()
+        {
+            return _cache.WithPartitionRecover();
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<int> GetLostPartitions()
+        {
+            return _cache.GetLostPartitions();
+        }
+
+        /** <inheritDoc /> */
         public IEnumerator<ICacheEntry<TK, TV>> GetEnumerator()
         {
             return _cache.GetEnumerator();
@@ -565,7 +577,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             }
             catch (AggregateException ex)
             {
-                throw ex.InnerException;
+                throw ex.InnerException ?? ex;
             }
         }
 
@@ -580,7 +592,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             }
             catch (Exception ex)
             {
-                throw ex.InnerException;
+                throw ex.InnerException ?? ex;
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PartitionLossTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PartitionLossTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PartitionLossTest.cs
new file mode 100644
index 0000000..333b3f5
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PartitionLossTest.cs
@@ -0,0 +1,260 @@
+\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.Tests.Cache
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests partition loss management functionality:
+    /// <see cref="PartitionLossPolicy"/>, <see cref="IIgnite.ResetLostPartitions(IEnumerable{string})"/>,
+    /// <see cref="ICache{TK,TV}.GetLostPartitions"/>, <see cref="ICache{TK,TV}.WithPartitionRecover"/>.
+    /// </summary>
+    public class PartitionLossTest
+    {
+        /** */
+        private const string CacheName = "lossTestCache";
+
+        /// <summary>
+        /// Fixture set up.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void FixtureSetUp()
+        {
+            Ignition.Start(TestUtils.GetTestConfiguration());
+        }
+
+        /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Test teardown.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            var ignite = Ignition.GetIgnite();
+
+            ignite.GetCacheNames().ToList().ForEach(ignite.DestroyCache);
+        }
+
+        /// <summary>
+        /// Tests the ReadOnlySafe mode.
+        /// </summary>
+        [Test]
+        public void TestReadOnlySafe()
+        {
+            TestPartitionLoss(PartitionLossPolicy.ReadOnlySafe, false, true);
+        }
+
+        /// <summary>
+        /// Tests the ReadWriteSafe mode.
+        /// </summary>
+        [Test]
+        public void TestReadWriteSafe()
+        {
+            TestPartitionLoss(PartitionLossPolicy.ReadWriteSafe, true, true);
+        }
+
+        /// <summary>
+        /// Tests the ReadOnlyAll mode.
+        /// </summary>
+        [Test]
+        public void TestReadOnlyAll()
+        {
+            TestPartitionLoss(PartitionLossPolicy.ReadOnlyAll, false, false);
+        }
+
+        /// <summary>
+        /// Tests the ReadWriteAll mode.
+        /// </summary>
+        [Test]
+        public void TestReadWriteAll()
+        {
+            TestPartitionLoss(PartitionLossPolicy.ReadWriteAll, true, false);
+        }
+
+        /// <summary>
+        /// Tests the Ignore mode.
+        /// </summary>
+        [Test]
+        public void TestIgnoreLoss()
+        {
+            var ignite = Ignition.GetIgnite();
+
+            var cache = CreateCache(PartitionLossPolicy.Ignore, ignite);
+
+            var lostPart = PrepareTopology();
+
+            Assert.IsEmpty(cache.GetLostPartitions());
+
+            cache[lostPart] = lostPart;
+
+            Assert.AreEqual(lostPart, cache[lostPart]);
+        }
+
+        /// <summary>
+        /// Tests the partition loss.
+        /// </summary>
+        private static void TestPartitionLoss(PartitionLossPolicy policy, bool canWrite, bool safe)
+        {
+            var ignite = Ignition.GetIgnite();
+
+            var cache = CreateCache(policy, ignite);
+
+            // Loose data and verify lost partition.
+            var lostPart = PrepareTopology();
+            var lostParts = cache.GetLostPartitions();
+            Assert.IsTrue(lostParts.Contains(lostPart));
+
+            // Check cache operations.
+            foreach (var part in lostParts)
+            {
+                VerifyCacheOperations(cache, part, canWrite, safe);
+
+                // Check recover cache.
+                var recoverCache = cache.WithPartitionRecover();
+                recoverCache[part] = part;
+                Assert.AreEqual(part, recoverCache[part]);
+            }
+
+            // Reset and verify.
+            ignite.ResetLostPartitions(CacheName);
+            Assert.IsEmpty(cache.GetLostPartitions());
+
+            // Check another ResetLostPartitions overload.
+            PrepareTopology();
+            Assert.IsNotEmpty(cache.GetLostPartitions());
+            ignite.ResetLostPartitions(new List<string> {CacheName, "foo"});
+            Assert.IsEmpty(cache.GetLostPartitions());
+        }
+
+        /// <summary>
+        /// Verifies the cache operations.
+        /// </summary>
+        private static void VerifyCacheOperations(ICache<int, int> cache, int part, bool canWrite, bool safe)
+        {
+            if (safe)
+            {
+                int val;
+                var ex = Assert.Throws<CacheException>(() => cache.TryGet(part, out val));
+                Assert.AreEqual(string.Format(
+                    "class org.apache.ignite.internal.processors.cache.CacheInvalidStateException" +
+                    ": Failed to execute cache operation (all partition owners have left the grid, " +
+                    "partition data has been lost) [cacheName={0}, part={1}," +
+                    " key=UserKeyCacheObjectImpl [part={1}, val={1}, hasValBytes=false]]",
+                    CacheName, part), ex.Message);
+            }
+            else
+            {
+                int val;
+                Assert.IsFalse(cache.TryGet(part, out val));
+            }
+
+            if (canWrite)
+            {
+                if (safe)
+                {
+                    var ex = Assert.Throws<CacheException>(() => cache.Put(part, part));
+                    Assert.AreEqual(string.Format(
+                        "class org.apache.ignite.internal.processors.cache.CacheInvalidStateException: " +
+                        "Failed to execute cache operation (all partition owners have left the grid, " +
+                        "partition data has been lost) [cacheName={0}, part={1}, key={1}]",
+                        CacheName, part), ex.Message);
+                }
+                else
+                {
+                    cache[part] = part;
+                    Assert.AreEqual(part, cache[part]);
+                }
+            }
+            else
+            {
+                var ex = Assert.Throws<CacheException>(() => cache.Put(part, part));
+                Assert.AreEqual(string.Format(
+                    "class org.apache.ignite.IgniteCheckedException: " +
+                    "Failed to write to cache (cache is moved to a read-only state): {0}",
+                    CacheName), ex.Message);
+            }
+        }
+
+        /// <summary>
+        /// Creates the cache.
+        /// </summary>
+        private static ICache<int, int> CreateCache(PartitionLossPolicy policy, IIgnite ignite)
+        {
+            return ignite.CreateCache<int, int>(new CacheConfiguration(CacheName)
+            {
+                CacheMode = CacheMode.Partitioned,
+                Backups = 0,
+                WriteSynchronizationMode = CacheWriteSynchronizationMode.FullSync,
+                PartitionLossPolicy = policy,
+                AffinityFunction = new RendezvousAffinityFunction
+                {
+                    ExcludeNeighbors = false,
+                    Partitions = 32
+                }
+            });
+        }
+
+        /// <summary>
+        /// Prepares the topology: starts a new node and stops it after rebalance to ensure data loss.
+        /// </summary>
+        /// <returns>Lost partition id.</returns>
+        private static int PrepareTopology()
+        {
+            using (var ignite = Ignition.Start(TestUtils.GetTestConfiguration(name: "ignite-2")))
+            {
+                var cache = ignite.GetCache<int, int>(CacheName);
+
+                var affinity = ignite.GetAffinity(CacheName);
+
+                var keys = Enumerable.Range(1, affinity.Partitions).ToArray();
+
+                cache.PutAll(keys.ToDictionary(x => x, x => x));
+
+                cache.Rebalance();
+
+                // Wait for rebalance to complete.
+                var node = ignite.GetCluster().GetLocalNode();
+                Func<int, bool> isPrimary = x => affinity.IsPrimary(node, x);
+
+                while (!keys.Any(isPrimary))
+                {
+                    Thread.Sleep(10);
+                }
+
+                Thread.Sleep(100);  // Some extra wait.
+
+                return keys.First(isPrimary);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 6b94079..31dd887 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -90,7 +90,7 @@ namespace Apache.Ignite.Core.Tests
                                 <iLifecycleHandler type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+LifecycleBean' foo='15' />
                             </lifecycleHandlers>
                             <cacheConfiguration>
-                                <cacheConfiguration cacheMode='Replicated' readThrough='true' writeThrough='true' enableStatistics='true' writeBehindCoalescing='false'>
+                                <cacheConfiguration cacheMode='Replicated' readThrough='true' writeThrough='true' enableStatistics='true' writeBehindCoalescing='false' partitionLossPolicy='ReadWriteAll'>
                                     <queryEntities>    
                                         <queryEntity keyType='System.Int32' valueType='System.String' tableName='myTable'>    
                                             <fields>
@@ -175,6 +175,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.IsInstanceOf<MyPolicyFactory>(cacheCfg.ExpiryPolicyFactory);
             Assert.IsTrue(cacheCfg.EnableStatistics);
             Assert.IsFalse(cacheCfg.WriteBehindCoalescing);
+            Assert.AreEqual(PartitionLossPolicy.ReadWriteAll, cacheCfg.PartitionLossPolicy);
 
             var queryEntity = cacheCfg.QueryEntities.Single();
             Assert.AreEqual(typeof(int), queryEntity.KeyType);
@@ -700,7 +701,9 @@ namespace Apache.Ignite.Core.Tests
                         PluginConfigurations = new[]
                         {
                             new MyPluginConfiguration()
-                        }
+                        },
+                        MemoryPolicyName = "somePolicy",
+                        PartitionLossPolicy = PartitionLossPolicy.ReadOnlyAll
                     }
                 },
                 ClientMode = true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index 8d9a3d2..ed12efd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -336,14 +336,15 @@ namespace Apache.Ignite.Core.Tests
         /// <summary>
         /// Gets the default code-based test configuration.
         /// </summary>
-        public static IgniteConfiguration GetTestConfiguration(bool? jvmDebug = null)
+        public static IgniteConfiguration GetTestConfiguration(bool? jvmDebug = null, string name = null)
         {
             return new IgniteConfiguration
             {
                 DiscoverySpi = GetStaticDiscovery(),
                 Localhost = "127.0.0.1",
                 JvmOptions = TestJavaOptions(jvmDebug),
-                JvmClasspath = CreateTestClasspath()
+                JvmClasspath = CreateTestClasspath(),
+                IgniteInstanceName = name
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 9d1f9fc..7cf79dd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -94,6 +94,7 @@
     <Compile Include="Binary\BinaryBasicNameMapper.cs" />
     <Compile Include="Cache\Configuration\DataPageEvictionMode.cs" />
     <Compile Include="Cache\Configuration\MemoryPolicyConfiguration.cs" />
+    <Compile Include="Cache\Configuration\PartitionLossPolicy.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Events\IEventStorageSpi.cs" />
     <Compile Include="Events\MemoryEventStorageSpi.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 87ee255..dd50c3c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -129,6 +129,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <summary> Default value for <see cref="WriteBehindCoalescing"/>. </summary>
         public const bool DefaultWriteBehindCoalescing = true;
 
+        /// <summary> Default value for <see cref="PartitionLossPolicy"/>. </summary>
+        public const PartitionLossPolicy DefaultPartitionLossPolicy = PartitionLossPolicy.Ignore;
+
         /// <summary>
         /// Gets or sets the cache name.
         /// </summary>
@@ -173,6 +176,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             WriteBehindFlushSize = DefaultWriteBehindFlushSize;
             WriteBehindFlushThreadCount= DefaultWriteBehindFlushThreadCount;
             WriteBehindCoalescing = DefaultWriteBehindCoalescing;
+            PartitionLossPolicy = DefaultPartitionLossPolicy;
         }
 
         /// <summary>
@@ -239,6 +243,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             WriteThrough = reader.ReadBoolean();
             EnableStatistics = reader.ReadBoolean();
             MemoryPolicyName = reader.ReadString();
+            PartitionLossPolicy = (PartitionLossPolicy) reader.ReadInt();
             CacheStoreFactory = reader.ReadObject<IFactory<ICacheStore>>();
 
             var count = reader.ReadInt();
@@ -296,6 +301,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             writer.WriteBoolean(WriteThrough);
             writer.WriteBoolean(EnableStatistics);
             writer.WriteString(MemoryPolicyName);
+            writer.WriteInt((int) PartitionLossPolicy);
             writer.WriteObject(CacheStoreFactory);
 
             if (QueryEntities != null)
@@ -645,5 +651,12 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// </summary>
         [DefaultValue(DefaultWriteBehindCoalescing)]
         public bool WriteBehindCoalescing { get; set; }
+
+        /// <summary>
+        /// Gets or sets the partition loss policy. This policy defines how Ignite will react to
+        /// a situation when all nodes for some partition leave the cluster.
+        /// </summary>
+        [DefaultValue(DefaultPartitionLossPolicy)]
+        public PartitionLossPolicy PartitionLossPolicy { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/PartitionLossPolicy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/PartitionLossPolicy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/PartitionLossPolicy.cs
new file mode 100644
index 0000000..0bea06f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/PartitionLossPolicy.cs
@@ -0,0 +1,68 @@
+\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.Cache.Configuration
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Partition loss policy. Defines how cache will behave in a case when one or more partitions are
+    /// lost because of a node(s) failure.
+    /// <para />
+    /// All *Safe policies prevent a user from interaction with partial data in lost partitions until 
+    /// <see cref="IIgnite.ResetLostPartitions(IEnumerable{string})"/> method is called.
+    /// <para />
+    /// *All policies allow working with partial data in lost partitions.
+    /// <para />
+    /// ReadOnly* and ReadWrite* policies do not automatically change partition state and thus do not change
+    /// rebalancing assignments for such partitions.
+    /// </summary>
+    public enum PartitionLossPolicy
+    {
+        /// <summary>
+        /// All writes to the cache will be failed with an exception, reads will only be allowed for keys in
+        /// non-lost partitions. Reads from lost partitions will be failed with an exception.
+        /// </summary>
+        ReadOnlySafe,
+
+        /// <summary>
+        /// All writes to the cache will be failed with an exception. All reads will proceed as if all partitions
+        /// were in a consistent state. The result of reading from a lost partition is undefined and may be different
+        /// on different nodes in the cluster.
+        /// </summary>
+        ReadOnlyAll,
+
+        /// <summary>
+        /// All reads and writes will be allowed for keys in valid partitions. All reads and writes for keys
+        /// in lost partitions will be failed with an exception.
+        /// </summary>
+        ReadWriteSafe,
+
+        /// <summary>
+        /// All reads and writes will proceed as if all partitions were in a consistent state. The result of
+        /// reading from a lost partition is undefined and may be different on different nodes in the cluster.
+        /// </summary>
+        ReadWriteAll,
+
+        /// <summary>
+        /// If partition is lost, reset it's state and do not clear intermediate data. The result of reading from
+        /// a previously lost and not cleared partition is undefined and may be different on different nodes in the
+        /// cluster.
+        /// </summary>
+        Ignore
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs
index b7cb0d5..8c7fcf9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs
@@ -883,5 +883,20 @@ namespace Apache.Ignite.Core.Cache
         /// </summary>
         /// <returns>Cache with no-retries behavior enabled.</returns>
         ICache<TK, TV> WithNoRetries();
+
+        /// <summary>
+        /// Gets an instance of cache that will be allowed to execute cache operations (read, write)
+        /// regardless of partition loss policy.
+        /// </summary>
+        /// <returns>Cache without partition loss protection.</returns>
+        ICache<TK, TV> WithPartitionRecover();
+
+        /// <summary>
+        /// Gets lost partitions IDs.
+        /// <para />
+        /// See also <see cref="CacheConfiguration.PartitionLossPolicy"/>
+        /// and <see cref="IIgnite.ResetLostPartitions(IEnumerable{string})"/>.
+        /// </summary>
+        ICollection<int> GetLostPartitions();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
index 2996039..573cfb3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
@@ -317,5 +317,17 @@ namespace Apache.Ignite.Core
         /// <exception cref="PluginNotFoundException">When plugin with specified name has not been found.</exception>
         /// <returns>Plugin instance.</returns>
         T GetPlugin<T>(string name) where T : class;
+
+        /// <summary>
+        /// Clears partitions' lost state and moves caches to a normal mode.
+        /// </summary>
+        /// <param name="cacheNames">Names of caches to reset partitions for.</param>
+        void ResetLostPartitions(IEnumerable<string> cacheNames);
+
+        /// <summary>
+        /// Clears partitions' lost state and moves caches to a normal mode.
+        /// </summary>
+        /// <param name="cacheNames">Names of caches to reset partitions for.</param>
+        void ResetLostPartitions(params string[] cacheNames);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 8b07147..67f2715 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -86,6 +86,16 @@
         </xs:restriction>
     </xs:simpleType>
 
+    <xs:simpleType name="partitionLossPolicy" final="restriction">
+        <xs:restriction base="xs:string">
+            <xs:enumeration value="ReadOnlySafe" />
+            <xs:enumeration value="ReadOnlyAll" />
+            <xs:enumeration value="ReadWriteSafe" />
+            <xs:enumeration value="ReadWriteAll" />
+            <xs:enumeration value="Ignore" />
+        </xs:restriction>
+    </xs:simpleType>
+
     <xs:element name="igniteConfiguration">
         <xs:annotation>
             <xs:documentation>Ignite configuration root.</xs:documentation>
@@ -717,6 +727,11 @@
                                             <xs:documentation>Name of the MemoryPolicyConfiguration for this cache.</xs:documentation>
                                         </xs:annotation>
                                     </xs:attribute>
+                                    <xs:attribute name="partitionLossPolicy" type="partitionLossPolicy">
+                                        <xs:annotation>
+                                            <xs:documentation>Partition loss policy defines how Ignite will react to a situation when all nodes for some partition leave the cluster.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
                                 </xs:complexType>
                             </xs:element>
                         </xs:sequence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/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 6009659..749409c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
@@ -58,6 +58,9 @@ namespace Apache.Ignite.Core.Impl.Cache
         /** Flag: no-retries.*/
         private readonly bool _flagNoRetries;
 
+        /** Flag: partition recover.*/
+        private readonly bool _flagPartitionRecover;
+
         /** Transaction manager. */
         private readonly CacheTransactionManager _txManager;
 
@@ -70,8 +73,10 @@ namespace Apache.Ignite.Core.Impl.Cache
         /// <param name="flagSkipStore">Skip store flag.</param>
         /// <param name="flagKeepBinary">Keep binary flag.</param>
         /// <param name="flagNoRetries">No-retries mode flag.</param>
+        /// <param name="flagPartitionRecover">Partition recover mode flag.</param>
         public CacheImpl(Ignite grid, IUnmanagedTarget target, Marshaller marsh,
-            bool flagSkipStore, bool flagKeepBinary, bool flagNoRetries) : base(target, marsh)
+            bool flagSkipStore, bool flagKeepBinary, bool flagNoRetries, bool flagPartitionRecover)
+            : base(target, marsh)
         {
             Debug.Assert(grid != null);
 
@@ -79,6 +84,7 @@ namespace Apache.Ignite.Core.Impl.Cache
             _flagSkipStore = flagSkipStore;
             _flagKeepBinary = flagKeepBinary;
             _flagNoRetries = flagNoRetries;
+            _flagPartitionRecover = flagPartitionRecover;
 
             _txManager = GetConfiguration().AtomicityMode == CacheAtomicityMode.Transactional
                 ? new CacheTransactionManager(grid.GetTransactions())
@@ -167,7 +173,7 @@ namespace Apache.Ignite.Core.Impl.Cache
                 return this;
 
             return new CacheImpl<TK, TV>(_ignite, DoOutOpObject((int) CacheOp.WithSkipStore), Marshaller,
-                true, _flagKeepBinary, true);
+                true, _flagKeepBinary, true, _flagPartitionRecover);
         }
 
         /// <summary>
@@ -191,7 +197,7 @@ namespace Apache.Ignite.Core.Impl.Cache
             }
 
             return new CacheImpl<TK1, TV1>(_ignite, DoOutOpObject((int) CacheOp.WithKeepBinary), Marshaller,
-                _flagSkipStore, true, _flagNoRetries);
+                _flagSkipStore, true, _flagNoRetries, _flagPartitionRecover);
         }
 
         /** <inheritDoc /> */
@@ -201,7 +207,8 @@ namespace Apache.Ignite.Core.Impl.Cache
 
             var cache0 = DoOutOpObject((int)CacheOp.WithExpiryPolicy, w => ExpiryPolicySerializer.WritePolicy(w, plc));
 
-            return new CacheImpl<TK, TV>(_ignite, cache0, Marshaller, _flagSkipStore, _flagKeepBinary, _flagNoRetries);
+            return new CacheImpl<TK, TV>(_ignite, cache0, Marshaller, _flagSkipStore, _flagKeepBinary, 
+                _flagNoRetries, _flagPartitionRecover);
         }
 
         /** <inheritDoc /> */
@@ -1005,7 +1012,38 @@ namespace Apache.Ignite.Core.Impl.Cache
                 return this;
 
             return new CacheImpl<TK, TV>(_ignite, DoOutOpObject((int) CacheOp.WithNoRetries), Marshaller,
-                _flagSkipStore, _flagKeepBinary, true);
+                _flagSkipStore, _flagKeepBinary, true, _flagPartitionRecover);
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithPartitionRecover()
+        {
+            if (_flagPartitionRecover)
+                return this;
+
+            return new CacheImpl<TK, TV>(_ignite, DoOutOpObject((int) CacheOp.WithPartitionRecover), Marshaller,
+                _flagSkipStore, _flagKeepBinary, _flagNoRetries, true);
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<int> GetLostPartitions()
+        {
+            return DoInOp((int) CacheOp.GetLostPartitions, s =>
+            {
+                var cnt = s.ReadInt();
+
+                var res = new List<int>(cnt);
+
+                if (cnt > 0)
+                {
+                    for (var i = 0; i < cnt; i++)
+                    {
+                        res.Add(s.ReadInt());
+                    }
+                }
+
+                return res;
+            });
         }
 
         #region Queries

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs
index 51fef40..b3cbd95 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs
@@ -62,7 +62,7 @@ namespace Apache.Ignite.Core.Impl.Cache
         GetConfig = 39,
         LoadAll = 40,
         ClearCache = 41,
-        WithAsync = 42,
+        WithPartitionRecover = 42,
         RemoveAll2 = 43,
         WithKeepBinary = 44,
         WithExpiryPolicy = 45,
@@ -103,6 +103,7 @@ namespace Apache.Ignite.Core.Impl.Cache
         InvokeAllAsync = 80,
         PutIfAbsentAsync = 81,
         Extension = 82,
-        GlobalMetrics = 83
+        GlobalMetrics = 83,
+        GetLostPartitions = 84
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
index 641b6b4..7e97852 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -115,6 +115,9 @@ namespace Apache.Ignite.Core.Impl.Cluster
         /** */
         private const int OpCacheMetrics = 24;
         
+        /** */
+        private const int OpResetLostPartitions = 25;
+        
         /** Initial Ignite instance. */
         private readonly Ignite _ignite;
         
@@ -511,6 +514,30 @@ namespace Apache.Ignite.Core.Impl.Cluster
         }
 
         /// <summary>
+        /// Resets the lost partitions.
+        /// </summary>
+        public void ResetLostPartitions(IEnumerable<string> cacheNames)
+        {
+            IgniteArgumentCheck.NotNull(cacheNames, "cacheNames");
+
+            DoOutOp(OpResetLostPartitions, w =>
+            {
+                var pos = w.Stream.Position;
+
+                var count = 0;
+                w.WriteInt(count);  // Reserve space.
+
+                foreach (var cacheName in cacheNames)
+                {
+                    w.WriteString(cacheName);
+                    count++;
+                }
+
+                w.Stream.WriteInt(pos, count);
+            });
+        }
+
+        /// <summary>
         /// Gets the cache metrics within this cluster group.
         /// </summary>
         /// <param name="cacheName">Name of the cache.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
index 0407b62..4eebbf9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
@@ -504,6 +504,7 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <param name="type">The type.</param>
         /// <param name="types">The argument types.</param>
         /// <returns>Constructor info.</returns>
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public static ConstructorInfo GetConstructorExact(Type type, Type[] types)
         {
             Debug.Assert(type != null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad7e4a09/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index 9cd1aa5..a795459 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -484,7 +484,7 @@ namespace Apache.Ignite.Core.Impl
         /// </returns>
         public ICache<TK, TV> Cache<TK, TV>(IUnmanagedTarget nativeCache, bool keepBinary = false)
         {
-            return new CacheImpl<TK, TV>(this, nativeCache, _marsh, false, keepBinary, false);
+            return new CacheImpl<TK, TV>(this, nativeCache, _marsh, false, keepBinary, false, false);
         }
 
         /** <inheritdoc /> */
@@ -703,6 +703,20 @@ namespace Apache.Ignite.Core.Impl
             return PluginProcessor.GetProvider(name).GetPlugin<T>();
         }
 
+        /** <inheritdoc /> */
+        public void ResetLostPartitions(IEnumerable<string> cacheNames)
+        {
+            IgniteArgumentCheck.NotNull(cacheNames, "cacheNames");
+
+            _prj.ResetLostPartitions(cacheNames);
+        }
+
+        /** <inheritdoc /> */
+        public void ResetLostPartitions(params string[] cacheNames)
+        {
+            ResetLostPartitions((IEnumerable<string>) cacheNames);
+        }
+
         /// <summary>
         /// Gets or creates near cache.
         /// </summary>