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/14 09:26:44 UTC

[32/33] ignite git commit: IGNITE-4126 .NET: Add IgniteConfiguration.SwapSpaceSpi

IGNITE-4126 .NET: Add IgniteConfiguration.SwapSpaceSpi


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

Branch: refs/heads/master
Commit: d69e26dd8d4fd9383a149c93c251911a8dd67528
Parents: c6921a3
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Nov 11 12:22:55 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Nov 11 12:22:55 2016 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       |  61 ++++++++--
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Cache/CacheSwapSpaceTest.cs                 | 120 +++++++++++++++++++
 .../IgniteConfigurationSerializerTest.cs        |  20 +++-
 .../IgniteConfigurationTest.cs                  |  22 +++-
 .../Apache.Ignite.Core.csproj                   |   3 +
 .../Cache/Configuration/CacheConfiguration.cs   |   2 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  15 ++-
 .../IgniteConfigurationSection.xsd              |  37 ++++++
 .../Impl/SwapSpace/SwapSpaceSerializer.cs       |  99 +++++++++++++++
 .../SwapSpace/File/FileSwapSpaceSpi.cs          | 105 ++++++++++++++++
 .../SwapSpace/ISwapSpaceSpi.cs                  |  35 ++++++
 12 files changed, 509 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/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 26374f0..9480dae 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
@@ -51,6 +51,9 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.swapspace.SwapSpaceSpi;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpiMBean;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 
@@ -66,8 +69,14 @@ import java.util.Map;
 /**
  * Configuration utils.
  */
-@SuppressWarnings("unchecked")
+@SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
 public class PlatformConfigurationUtils {
+    /** */
+    private static final byte SWAP_TYP_NONE = 0;
+
+    /** */
+    private static final byte SWAP_TYP_FILE = 1;
+
     /**
      * Write .Net configuration to the stream.
      *
@@ -202,7 +211,7 @@ public class PlatformConfigurationUtils {
      * @param in Stream.
      * @return Eviction policy.
      */
-    public static EvictionPolicy readEvictionPolicy(BinaryRawReader in) {
+    private static EvictionPolicy readEvictionPolicy(BinaryRawReader in) {
         byte plcTyp = in.readByte();
 
         switch (plcTyp) {
@@ -277,7 +286,7 @@ public class PlatformConfigurationUtils {
      * @param cfg NearCacheConfiguration.
      */
     @SuppressWarnings("TypeMayBeWeakened")
-    public static void writeNearConfiguration(BinaryRawWriter out, NearCacheConfiguration cfg) {
+    private static void writeNearConfiguration(BinaryRawWriter out, NearCacheConfiguration cfg) {
         assert cfg != null;
 
         out.writeInt(cfg.getNearStartSize());
@@ -371,7 +380,7 @@ public class PlatformConfigurationUtils {
      * @param in Stream.
      * @return QueryEntity.
      */
-    public static QueryEntity readQueryEntity(BinaryRawReader in) {
+    private static QueryEntity readQueryEntity(BinaryRawReader in) {
         QueryEntity res = new QueryEntity();
 
         res.setKeyType(in.readString());
@@ -422,7 +431,7 @@ public class PlatformConfigurationUtils {
      * @param in Reader.
      * @return Query index.
      */
-    public static QueryIndex readQueryIndex(BinaryRawReader in) {
+    private static QueryIndex readQueryIndex(BinaryRawReader in) {
         QueryIndex res = new QueryIndex();
 
         res.setName(in.readString());
@@ -529,6 +538,27 @@ public class PlatformConfigurationUtils {
 
             cfg.setTransactionConfiguration(tx);
         }
+
+        byte swapType = in.readByte();
+
+        switch (swapType) {
+            case SWAP_TYP_FILE: {
+                FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
+
+                swap.setBaseDirectory(in.readString());
+                swap.setMaximumSparsity(in.readFloat());
+                swap.setMaxWriteQueueSize(in.readInt());
+                swap.setReadStripesNumber(in.readInt());
+                swap.setWriteBufferSize(in.readInt());
+
+                cfg.setSwapSpaceSpi(swap);
+
+                break;
+            }
+
+            default:
+                assert swapType == SWAP_TYP_NONE;
+        }
     }
 
     /**
@@ -537,7 +567,7 @@ public class PlatformConfigurationUtils {
      * @param cfg IgniteConfiguration to update.
      * @param in Reader.
      */
-    public static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfiguration cfg) {
+    private static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfiguration cfg) {
         int len = in.readInt();
 
         if (len == 0)
@@ -569,7 +599,7 @@ public class PlatformConfigurationUtils {
      * @param cfg IgniteConfiguration to update.
      * @param in Reader.
      */
-    public static void readDiscoveryConfiguration(BinaryRawReader in, IgniteConfiguration cfg) {
+    private static void readDiscoveryConfiguration(BinaryRawReader in, IgniteConfiguration cfg) {
         boolean hasConfig = in.readBoolean();
 
         if (!hasConfig)
@@ -917,6 +947,23 @@ public class PlatformConfigurationUtils {
         else
             w.writeBoolean(false);
 
+        SwapSpaceSpi swap = cfg.getSwapSpaceSpi();
+
+        if (swap instanceof FileSwapSpaceSpiMBean) {
+            w.writeByte(SWAP_TYP_FILE);
+
+            FileSwapSpaceSpiMBean fileSwap = (FileSwapSpaceSpiMBean)swap;
+
+            w.writeString(fileSwap.getBaseDirectory());
+            w.writeFloat(fileSwap.getMaximumSparsity());
+            w.writeInt(fileSwap.getMaxWriteQueueSize());
+            w.writeInt(fileSwap.getReadStripesNumber());
+            w.writeInt(fileSwap.getWriteBufferSize());
+        }
+        else {
+            w.writeByte(SWAP_TYP_NONE);
+        }
+
         w.writeString(cfg.getIgniteHome());
 
         w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit());

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/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 34d93d7..a9de399 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
@@ -67,6 +67,7 @@
     <Compile Include="Cache\Affinity\AffinityKeyTest.cs" />
     <Compile Include="Cache\Affinity\AffinityTopologyVersionTest.cs" />
     <Compile Include="Cache\CacheResultTest.cs" />
+    <Compile Include="Cache\CacheSwapSpaceTest.cs" />
     <Compile Include="Cache\Store\CacheStoreAdapterTest.cs" />
     <Compile Include="Collections\MultiValueDictionaryTest.cs" />
     <Compile Include="Collections\ReadOnlyCollectionTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs
new file mode 100644
index 0000000..296fd13
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs
@@ -0,0 +1,120 @@
+\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.IO;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Eviction;
+    using Apache.Ignite.Core.Discovery.Tcp;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.SwapSpace.File;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the swap space.
+    /// </summary>
+    public class CacheSwapSpaceTest
+    {
+        /** */
+        private readonly string _tempDir = IgniteUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            Ignition.StopAll(true);
+
+            Directory.Delete(_tempDir, true);
+        }
+
+        /// <summary>
+        /// Tests that swap space is disabled by default and cache can't have EnableSwap.
+        /// </summary>
+        [Test]
+        public void TestDisabledSwapSpace()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration());
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                // NoopSwapSpaceSpi is used by default.
+                Assert.IsNull(ignite.GetConfiguration().SwapSpaceSpi);
+
+                var ex = Assert.Throws<CacheException>(
+                    () => ignite.CreateCache<int, int>(new CacheConfiguration {EnableSwap = true}));
+
+                Assert.IsTrue(ex.Message.EndsWith("has not swap SPI configured"));
+            }
+        }
+
+        /// <summary>
+        /// Tests the swap space.
+        /// </summary>
+        [Test]
+        public void TestSwapSpace()
+        {
+            const int entrySize = 1024;
+
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                SwapSpaceSpi = new FileSwapSpaceSpi
+                {
+                    BaseDirectory = _tempDir,
+                    WriteBufferSize = 64
+                }
+            };
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                // Create cache with eviction and swap.
+                var cache = ignite.CreateCache<int, byte[]>(new CacheConfiguration("cache")
+                {
+                    EnableSwap = true,
+                    EvictionPolicy = new LruEvictionPolicy
+                    {
+                        MaxSize = 3
+                    },
+                    OffHeapMaxMemory = 5 * entrySize
+                });
+
+                // Populate to trigger eviction.
+                var data = Enumerable.Range(1, entrySize).Select(x => (byte) x).ToArray();
+
+                for (int i = 0; i < 10; i++)
+                    cache[i] = data;
+
+                // Check that swap files exist.
+                var files = Directory.GetFiles(_tempDir, "*.*", SearchOption.AllDirectories);
+                CollectionAssert.IsNotEmpty(files);
+                
+                // Wait for metrics update and check metrics.
+                Thread.Sleep(((TcpDiscoverySpi) ignite.GetConfiguration().DiscoverySpi).HeartbeatFrequency);
+
+                var metrics = cache.GetMetrics();
+
+                Assert.AreEqual(4, metrics.OffHeapEntriesCount);  // Entry takes more space than the value
+                Assert.AreEqual(3, metrics.OverflowSize / entrySize);  // 10 - 3 - 4 = 3
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/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 9fb5bbd..ac3e553 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -45,6 +45,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Log;
+    using Apache.Ignite.Core.SwapSpace.File;
     using Apache.Ignite.Core.Tests.Binary;
     using Apache.Ignite.Core.Transactions;
     using Apache.Ignite.NLog;
@@ -113,6 +114,7 @@ namespace Apache.Ignite.Core.Tests
                             <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
                             <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
                             <logger type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestLogger, Apache.Ignite.Core.Tests' />
+                            <swapSpaceSpi type='FileSwapSpaceSpi' baseDirectory='abcd' maximumSparsity='0.7' maximumWriteQueueSize='25' readStripesNumber='36' writeBufferSize='47' />
                         </igniteConfig>";
             var reader = XmlReader.Create(new StringReader(xml));
 
@@ -199,6 +201,14 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(new TimeSpan(0, 1, 2), comm.IdleConnectionTimeout);
 
             Assert.IsInstanceOf<TestLogger>(cfg.Logger);
+
+            var swap = cfg.SwapSpaceSpi as FileSwapSpaceSpi;
+            Assert.IsNotNull(swap);
+            Assert.AreEqual("abcd", swap.BaseDirectory);
+            Assert.AreEqual(0.7f, swap.MaximumSparsity);
+            Assert.AreEqual(25, swap.MaximumWriteQueueSize);
+            Assert.AreEqual(36, swap.ReadStripesNumber);
+            Assert.AreEqual(47, swap.WriteBufferSize);
         }
 
         /// <summary>
@@ -599,7 +609,15 @@ namespace Apache.Ignite.Core.Tests
                 IsLateAffinityAssignment = false,
                 SpringConfigUrl = "test",
                 Logger = new IgniteNLogLogger(),
-                FailureDetectionTimeout = TimeSpan.FromMinutes(2)
+                FailureDetectionTimeout = TimeSpan.FromMinutes(2),
+                SwapSpaceSpi = new FileSwapSpaceSpi
+                {
+                    MaximumSparsity = 0.1f,
+                    MaximumWriteQueueSize = 55,
+                    WriteBufferSize = 66,
+                    ReadStripesNumber = 77,
+                    BaseDirectory = "test"
+                }
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 30bae47..2e39b9b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -34,6 +34,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Discovery.Tcp.Static;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.SwapSpace.File;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
 
@@ -79,6 +80,7 @@ namespace Apache.Ignite.Core.Tests
             CheckDefaultValueAttributes(new LruEvictionPolicy());
             CheckDefaultValueAttributes(new AtomicConfiguration());
             CheckDefaultValueAttributes(new TransactionConfiguration());
+            CheckDefaultValueAttributes(new FileSwapSpaceSpi());
         }
 
         /// <summary>
@@ -171,6 +173,14 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(com.UnacknowledgedMessagesBufferSize, resCom.UnacknowledgedMessagesBufferSize);
 
                 Assert.AreEqual(cfg.FailureDetectionTimeout, resCfg.FailureDetectionTimeout);
+
+                var swap = (FileSwapSpaceSpi) cfg.SwapSpaceSpi;
+                var resSwap = (FileSwapSpaceSpi) resCfg.SwapSpaceSpi;
+                Assert.AreEqual(swap.MaximumSparsity, resSwap.MaximumSparsity);
+                Assert.AreEqual(swap.BaseDirectory, resSwap.BaseDirectory);
+                Assert.AreEqual(swap.MaximumWriteQueueSize, resSwap.MaximumWriteQueueSize);
+                Assert.AreEqual(swap.ReadStripesNumber, resSwap.ReadStripesNumber);
+                Assert.AreEqual(swap.WriteBufferSize, resSwap.WriteBufferSize);
             }
         }
 
@@ -398,7 +408,7 @@ namespace Apache.Ignite.Core.Tests
         {
             var props = obj.GetType().GetProperties();
 
-            foreach (var prop in props.Where(p => p.Name != "SelectorsCount"))
+            foreach (var prop in props.Where(p => p.Name != "SelectorsCount" && p.Name != "ReadStripesNumber"))
             {
                 var attr = prop.GetCustomAttributes(true).OfType<DefaultValueAttribute>().FirstOrDefault();
                 var propValue = prop.GetValue(obj, null);
@@ -495,7 +505,15 @@ namespace Apache.Ignite.Core.Tests
                     SocketSendBufferSize = 2045,
                     UnacknowledgedMessagesBufferSize = 3450
                 },
-                FailureDetectionTimeout = TimeSpan.FromSeconds(3.5)
+                FailureDetectionTimeout = TimeSpan.FromSeconds(3.5),
+                SwapSpaceSpi = new FileSwapSpaceSpi
+                {
+                    ReadStripesNumber = 64,
+                    MaximumWriteQueueSize = 8,
+                    WriteBufferSize = 9,
+                    BaseDirectory = Path.GetTempPath(),
+                    MaximumSparsity = 11.22f
+                }
             };
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/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 66253f6..64d7388 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -418,6 +418,7 @@
     <Compile Include="Impl\Services\ServiceProxyInvoker.cs" />
     <Compile Include="Impl\Services\ServiceProxySerializer.cs" />
     <Compile Include="Impl\Services\Services.cs" />
+    <Compile Include="Impl\SwapSpace\SwapSpaceSerializer.cs" />
     <Compile Include="Impl\Transactions\Transaction.cs" />
     <Compile Include="Impl\Transactions\TransactionImpl.cs" />
     <Compile Include="Impl\Transactions\TransactionMetricsImpl.cs" />
@@ -471,6 +472,8 @@
     <Compile Include="Services\Package-Info.cs" />
     <Compile Include="Services\ServiceConfiguration.cs" />
     <Compile Include="Services\ServiceInvocationException.cs" />
+    <Compile Include="SwapSpace\File\FileSwapSpaceSpi.cs" />
+    <Compile Include="SwapSpace\ISwapSpaceSpi.cs" />
     <Compile Include="Transactions\ITransaction.cs" />
     <Compile Include="Transactions\ITransactionMetrics.cs" />
     <Compile Include="Transactions\ITransactions.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/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 cb15564..c0c4722 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -507,6 +507,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
 
         /// <summary>
         /// Flag indicating whether Ignite should use swap storage by default.
+        /// <para />
+        /// Enabling this requires configured <see cref="IgniteConfiguration.SwapSpaceSpi"/>.
         /// </summary>
         [DefaultValue(DefaultEnableSwap)]
         public bool EnableSwap { get; set; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 68433f7..e7aa64e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -39,8 +39,10 @@
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.SwapSpace;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Log;
+    using Apache.Ignite.Core.SwapSpace;
     using Apache.Ignite.Core.Transactions;
     using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader;
     using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
@@ -297,10 +299,13 @@
                 writer.WriteInt((int) TransactionConfiguration.DefaultTransactionConcurrency);
                 writer.WriteInt((int) TransactionConfiguration.DefaultTransactionIsolation);
                 writer.WriteLong((long) TransactionConfiguration.DefaultTimeout.TotalMilliseconds);
-                writer.WriteLong((int) TransactionConfiguration.PessimisticTransactionLogLinger.TotalMilliseconds);
+                writer.WriteInt((int) TransactionConfiguration.PessimisticTransactionLogLinger.TotalMilliseconds);
             }
             else
                 writer.WriteBoolean(false);
+
+            // Swap space
+            SwapSpaceSerializer.Write(writer, SwapSpaceSpi);
         }
 
         /// <summary>
@@ -386,6 +391,9 @@
                     PessimisticTransactionLogLinger = TimeSpan.FromMilliseconds(r.ReadInt())
                 };
             }
+
+            // Swap
+            SwapSpaceSpi = SwapSpaceSerializer.Read(r);
         }
 
         /// <summary>
@@ -790,5 +798,10 @@
             get { return _failureDetectionTimeout ?? DefaultFailureDetectionTimeout; }
             set { _failureDetectionTimeout = value; }
         }
+
+        /// <summary>
+        /// Gets or sets the swap space SPI.
+        /// </summary>
+        public ISwapSpaceSpi SwapSpaceSpi { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/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 e9fb991..06541da 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1107,6 +1107,43 @@
                         </xs:attribute>
                     </xs:complexType>
                 </xs:element>
+                <xs:element name="swapSpaceSpi" minOccurs="0">
+                    <xs:annotation>
+                        <xs:documentation>Swap space SPI.</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:attribute name="baseDirectory" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Base directory.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="maximumSparsity" type="xs:decimal">
+                            <xs:annotation>
+                                <xs:documentation>Maximum sparsity. This property defines maximum acceptable wasted file space to whole file size ratio. When this ratio becomes higher than specified number compacting thread starts working.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="maximumWriteQueueSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Maximum size of the write queue in bytes. If there are more values are waiting to be written to disk then specified size, SPI will block on write operation.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="readStripesNumber" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Read stripes number. Defines number of file channels to be used concurrently. Defaults to Environment.ProcessorCount.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="writeBufferSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the write buffer, in bytes. Write to disk occurs only when this buffer is full.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                    </xs:complexType>
+                </xs:element>
             </xs:all>
             <xs:attribute name="gridName" type="xs:string">
                 <xs:annotation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs
new file mode 100644
index 0000000..7fdbaad
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs
@@ -0,0 +1,99 @@
+\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.SwapSpace
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.SwapSpace;
+    using Apache.Ignite.Core.SwapSpace.File;
+
+    /// <summary>
+    /// SwapSpace config serializer.
+    /// </summary>
+    internal static class SwapSpaceSerializer
+    {
+        /// <summary>
+        /// SwapSpace type.
+        /// </summary>
+        private enum Type : byte
+        {
+            None = 0,
+            File = 1
+        }
+
+        /// <summary>
+        /// Writes the configuration to writer.
+        /// </summary>
+        public static void Write(IBinaryRawWriter writer, ISwapSpaceSpi spi)
+        {
+            Debug.Assert(writer != null);
+
+            var fileSwap = spi as FileSwapSpaceSpi;
+
+            if (spi == null)
+            {
+                writer.WriteByte((byte) Type.None);
+            }
+            else if (fileSwap != null)
+            {
+                writer.WriteByte((byte) Type.File);
+
+                writer.WriteString(fileSwap.BaseDirectory);
+                writer.WriteFloat(fileSwap.MaximumSparsity);
+                writer.WriteInt(fileSwap.MaximumWriteQueueSize);
+                writer.WriteInt(fileSwap.ReadStripesNumber);
+                writer.WriteInt(fileSwap.WriteBufferSize);
+
+            }
+            else
+            {
+                throw new InvalidOperationException("Unsupported swap space SPI: " + spi.GetType());
+            }
+        }
+
+        /// <summary>
+        /// Reads the configuration from reader.
+        /// </summary>
+        public static ISwapSpaceSpi Read(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            var type = (Type) reader.ReadByte();
+
+            switch (type)
+            {
+                case Type.None:
+                    return null;
+
+                case Type.File:
+                    return new FileSwapSpaceSpi
+                    {
+                        BaseDirectory = reader.ReadString(),
+                        MaximumSparsity = reader.ReadFloat(),
+                        MaximumWriteQueueSize = reader.ReadInt(),
+                        ReadStripesNumber = reader.ReadInt(),
+                        WriteBufferSize = reader.ReadInt()
+                    };
+
+                default:
+                    throw new ArgumentOutOfRangeException("Invalid Swap Space SPI type: " + type);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs
new file mode 100644
index 0000000..2140275
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs
@@ -0,0 +1,105 @@
+\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.SwapSpace.File
+{
+    using System;
+    using System.ComponentModel;
+
+    /// <summary>
+    /// File-based swap space SPI implementation which holds keys in memory and values on disk.
+    /// It is intended for cases when value is bigger than 100 bytes, otherwise it will not 
+    /// have any positive effect.
+    /// </summary>
+    public class FileSwapSpaceSpi : ISwapSpaceSpi
+    {
+        /// <summary>
+        /// Default value for <see cref="MaximumSparsity"/> property.
+        /// </summary>
+        public const float DefaultMaximumSparsity = 0.5f;
+
+        /// <summary>
+        /// Default value for <see cref="WriteBufferSize"/> property.
+        /// </summary>
+        public const int DefaultWriteBufferSize = 64 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="MaximumWriteQueueSize"/> property.
+        /// </summary>
+        public const int DefaultMaximumWriteQueueSize = 1024 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="ReadStripesNumber"/> property.
+        /// </summary>
+        public static readonly int DefaultReadStripesNumber = Environment.ProcessorCount;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="FileSwapSpaceSpi"/> class.
+        /// </summary>
+        public FileSwapSpaceSpi()
+        {
+            MaximumSparsity = DefaultMaximumSparsity;
+            MaximumWriteQueueSize = DefaultMaximumWriteQueueSize;
+            ReadStripesNumber = DefaultReadStripesNumber;
+            WriteBufferSize = DefaultWriteBufferSize;
+        }
+
+        /// <summary>
+        /// Gets or sets the base directory.
+        /// </summary>
+        public string BaseDirectory { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum sparsity. This property defines maximum acceptable
+        /// wasted file space to whole file size ratio.
+        /// When this ratio becomes higher than specified number compacting thread starts working.
+        /// </summary>
+        /// <value>
+        /// The maximum sparsity. Must be between 0 and 1.
+        /// </value>
+        [DefaultValue(DefaultMaximumSparsity)]
+        public float MaximumSparsity { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum size of the write queue in bytes. If there are more values are waiting
+        /// to be written to disk then specified size, SPI will block on write operation.
+        /// </summary>
+        /// <value>
+        /// The maximum size of the write queue, in bytes.
+        /// </value>
+        [DefaultValue(DefaultMaximumWriteQueueSize)]
+        public int MaximumWriteQueueSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the read stripes number. Defines number of file channels to be used concurrently. 
+        /// Default is <see cref="Environment.ProcessorCount"/>.
+        /// </summary>
+        /// <value>
+        /// Number of read stripes.
+        /// </value>
+        public int ReadStripesNumber { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the write buffer, in bytes. Write to disk occurs only when this buffer is full.
+        /// </summary>
+        /// <value>
+        /// The size of the write buffer, in bytes.
+        /// </value>
+        [DefaultValue(DefaultWriteBufferSize)]
+        public int WriteBufferSize { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs
new file mode 100644
index 0000000..a348b7b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs
@@ -0,0 +1,35 @@
+\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.SwapSpace
+{
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.SwapSpace.File;
+
+    /// <summary>
+    /// Provides a mechanism in grid for storing data on disk. Ignite cache uses swap space to overflow 
+    /// data to disk if it cannot fit in memory. 
+    /// Logically storage is organized into independent 'spaces' in which data is stored.
+    /// <para />
+    /// Only predefined implementation is supported now: <see cref="FileSwapSpaceSpi"/>.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
+    public interface ISwapSpaceSpi
+    {
+        // No-op.
+    }
+}