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 2017/10/20 11:38:21 UTC

[2/3] ignite git commit: IGNITE-6515 .NET: Enable persistence on per-cache basis

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 73636d1..c8c06b2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -41,6 +41,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Tests.Plugin;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
+    using WalMode = Apache.Ignite.Core.PersistentStore.WalMode;
 
     /// <summary>
     /// Tests code-based configuration.
@@ -64,6 +65,8 @@ namespace Apache.Ignite.Core.Tests
         {
             CheckDefaultProperties(new IgniteConfiguration());
             CheckDefaultProperties(new PersistentStoreConfiguration());
+            CheckDefaultProperties(new DataStorageConfiguration());
+            CheckDefaultProperties(new DataRegionConfiguration());
             CheckDefaultProperties(new ClientConnectorConfiguration());
             CheckDefaultProperties(new SqlConnectorConfiguration());
         }
@@ -94,6 +97,8 @@ namespace Apache.Ignite.Core.Tests
             CheckDefaultValueAttributes(new PersistentStoreConfiguration());
             CheckDefaultValueAttributes(new IgniteClientConfiguration());
             CheckDefaultValueAttributes(new QueryIndex());
+            CheckDefaultValueAttributes(new DataStorageConfiguration());
+            CheckDefaultValueAttributes(new DataRegionConfiguration());
         }
 
         /// <summary>
@@ -219,33 +224,6 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(eventCfg.ExpirationTimeout, resEventCfg.ExpirationTimeout);
                 Assert.AreEqual(eventCfg.MaxEventCount, resEventCfg.MaxEventCount);
 
-                var memCfg = cfg.MemoryConfiguration;
-                var resMemCfg = resCfg.MemoryConfiguration;
-                Assert.IsNotNull(memCfg);
-                Assert.IsNotNull(resMemCfg);
-                Assert.AreEqual(memCfg.PageSize, resMemCfg.PageSize);
-                Assert.AreEqual(memCfg.ConcurrencyLevel, resMemCfg.ConcurrencyLevel);
-                Assert.AreEqual(memCfg.DefaultMemoryPolicyName, resMemCfg.DefaultMemoryPolicyName);
-                Assert.AreEqual(memCfg.SystemCacheInitialSize, resMemCfg.SystemCacheInitialSize);
-                Assert.AreEqual(memCfg.SystemCacheMaxSize, resMemCfg.SystemCacheMaxSize);
-                Assert.IsNotNull(memCfg.MemoryPolicies);
-                Assert.IsNotNull(resMemCfg.MemoryPolicies);
-                Assert.AreEqual(2, memCfg.MemoryPolicies.Count);
-                Assert.AreEqual(2, resMemCfg.MemoryPolicies.Count);
-
-                for (var i = 0; i < memCfg.MemoryPolicies.Count; i++)
-                {
-                    var plc = memCfg.MemoryPolicies.Skip(i).First();
-                    var resPlc = resMemCfg.MemoryPolicies.Skip(i).First();
-
-                    Assert.AreEqual(plc.PageEvictionMode, resPlc.PageEvictionMode);
-                    Assert.AreEqual(plc.MaxSize, resPlc.MaxSize);
-                    Assert.AreEqual(plc.EmptyPagesPoolSize, resPlc.EmptyPagesPoolSize);
-                    Assert.AreEqual(plc.EvictionThreshold, resPlc.EvictionThreshold);
-                    Assert.AreEqual(plc.Name, resPlc.Name);
-                    Assert.AreEqual(plc.SwapFilePath, resPlc.SwapFilePath);
-                }
-
                 var sql = cfg.SqlConnectorConfiguration;
                 var resSql = resCfg.SqlConnectorConfiguration;
 
@@ -258,30 +236,7 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(sql.TcpNoDelay, resSql.TcpNoDelay);
                 Assert.AreEqual(sql.ThreadPoolSize, resSql.ThreadPoolSize);
 
-                var pers = cfg.PersistentStoreConfiguration;
-                var resPers = resCfg.PersistentStoreConfiguration;
-
-                Assert.AreEqual(pers.AlwaysWriteFullPages, resPers.AlwaysWriteFullPages);
-                Assert.AreEqual(pers.CheckpointingFrequency, resPers.CheckpointingFrequency);
-                Assert.AreEqual(pers.CheckpointingPageBufferSize, resPers.CheckpointingPageBufferSize);
-                Assert.AreEqual(pers.CheckpointingThreads, resPers.CheckpointingThreads);
-                Assert.AreEqual(pers.LockWaitTime, resPers.LockWaitTime);
-                Assert.AreEqual(pers.PersistentStorePath, resPers.PersistentStorePath);
-                Assert.AreEqual(pers.TlbSize, resPers.TlbSize);
-                Assert.AreEqual(pers.WalArchivePath, resPers.WalArchivePath);
-                Assert.AreEqual(pers.WalFlushFrequency, resPers.WalFlushFrequency);
-                Assert.AreEqual(pers.WalFsyncDelayNanos, resPers.WalFsyncDelayNanos);
-                Assert.AreEqual(pers.WalHistorySize, resPers.WalHistorySize);
-                Assert.AreEqual(pers.WalMode, resPers.WalMode);
-                Assert.AreEqual(pers.WalRecordIteratorBufferSize, resPers.WalRecordIteratorBufferSize);
-                Assert.AreEqual(pers.WalSegments, resPers.WalSegments);
-                Assert.AreEqual(pers.WalSegmentSize, resPers.WalSegmentSize);
-                Assert.AreEqual(pers.WalStorePath, resPers.WalStorePath);
-                Assert.AreEqual(pers.MetricsEnabled, resPers.MetricsEnabled);
-                Assert.AreEqual(pers.RateTimeInterval, resPers.RateTimeInterval);
-                Assert.AreEqual(pers.SubIntervals, resPers.SubIntervals);
-                Assert.AreEqual(pers.CheckpointWriteOrder, resPers.CheckpointWriteOrder);
-                Assert.AreEqual(pers.WriteThrottlingEnabled, resPers.WriteThrottlingEnabled);
+                TestUtils.AssertReflectionEqual(cfg.DataStorageConfiguration, resCfg.DataStorageConfiguration);
             }
         }
 
@@ -311,24 +266,9 @@ namespace Apache.Ignite.Core.Tests
                 Assert.IsNotNull(disco);
                 Assert.AreEqual(TimeSpan.FromMilliseconds(300), disco.SocketTimeout);
 
-                // Check memory configuration defaults.
-                var mem = resCfg.MemoryConfiguration;
-
-                Assert.IsNotNull(mem);
-                Assert.AreEqual("dfltPlc", mem.DefaultMemoryPolicyName);
-                Assert.AreEqual(MemoryConfiguration.DefaultSystemCacheInitialSize, mem.SystemCacheInitialSize);
-                Assert.AreEqual(MemoryConfiguration.DefaultSystemCacheMaxSize, mem.SystemCacheMaxSize);
-
-                var plc = mem.MemoryPolicies.Single();
-                Assert.AreEqual("dfltPlc", plc.Name);
-                Assert.AreEqual(MemoryPolicyConfiguration.DefaultEmptyPagesPoolSize, plc.EmptyPagesPoolSize);
-                Assert.AreEqual(MemoryPolicyConfiguration.DefaultEvictionThreshold, plc.EvictionThreshold);
-                Assert.AreEqual(MemoryPolicyConfiguration.DefaultMaxSize, plc.MaxSize);
-                Assert.AreEqual(MemoryPolicyConfiguration.DefaultSubIntervals, plc.SubIntervals);
-                Assert.AreEqual(MemoryPolicyConfiguration.DefaultRateTimeInterval, plc.RateTimeInterval);
-
-                // Check PersistentStoreConfiguration defaults.
-                CheckDefaultProperties(resCfg.PersistentStoreConfiguration);
+                // DataStorage defaults.
+                CheckDefaultProperties(resCfg.DataStorageConfiguration);
+                CheckDefaultProperties(resCfg.DataStorageConfiguration.DefaultDataRegionConfiguration);
 
                 // Connector defaults.
                 CheckDefaultProperties(resCfg.ClientConnectorConfiguration);
@@ -598,6 +538,54 @@ namespace Apache.Ignite.Core.Tests
         /// Checks the default properties.
         /// </summary>
         /// <param name="cfg">Config.</param>
+        private static void CheckDefaultProperties(DataStorageConfiguration cfg)
+        {
+            Assert.AreEqual(DataStorageConfiguration.DefaultTlbSize, cfg.WalThreadLocalBufferSize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultCheckpointFrequency, cfg.CheckpointFrequency);
+            Assert.AreEqual(DataStorageConfiguration.DefaultCheckpointThreads, cfg.CheckpointThreads);
+            Assert.AreEqual(default(long), cfg.CheckpointPageBufferSize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultLockWaitTime, cfg.LockWaitTime);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalFlushFrequency, cfg.WalFlushFrequency);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalFsyncDelayNanos, cfg.WalFsyncDelayNanos);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalHistorySize, cfg.WalHistorySize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalRecordIteratorBufferSize,
+                cfg.WalRecordIteratorBufferSize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalSegmentSize, cfg.WalSegmentSize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalSegments, cfg.WalSegments);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalMode, cfg.WalMode);
+            Assert.IsFalse(cfg.MetricsEnabled);
+            Assert.AreEqual(DataStorageConfiguration.DefaultMetricsSubIntervalCount, cfg.MetricsSubIntervalCount);
+            Assert.AreEqual(DataStorageConfiguration.DefaultMetricsRateTimeInterval, cfg.MetricsRateTimeInterval);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalPath, cfg.WalPath);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalArchivePath, cfg.WalArchivePath);
+            Assert.AreEqual(DataStorageConfiguration.DefaultCheckpointWriteOrder, cfg.CheckpointWriteOrder);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWriteThrottlingEnabled, cfg.WriteThrottlingEnabled);
+
+            Assert.AreEqual(DataStorageConfiguration.DefaultSystemRegionInitialSize, cfg.SystemRegionInitialSize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultSystemRegionMaxSize, cfg.SystemRegionMaxSize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultPageSize, cfg.PageSize);
+            Assert.AreEqual(DataStorageConfiguration.DefaultConcurrencyLevel, cfg.ConcurrencyLevel);
+        }
+
+        /// <summary>
+        /// Checks the default properties.
+        /// </summary>
+        /// <param name="cfg">Config.</param>
+        private static void CheckDefaultProperties(DataRegionConfiguration cfg)
+        {
+            Assert.AreEqual(DataRegionConfiguration.DefaultEmptyPagesPoolSize, cfg.EmptyPagesPoolSize);
+            Assert.AreEqual(DataRegionConfiguration.DefaultEvictionThreshold, cfg.EvictionThreshold);
+            Assert.AreEqual(DataRegionConfiguration.DefaultInitialSize, cfg.InitialSize);
+            Assert.AreEqual(DataRegionConfiguration.DefaultMaxSize, cfg.MaxSize);
+            Assert.AreEqual(DataRegionConfiguration.DefaultPersistenceEnabled, cfg.PersistenceEnabled);
+            Assert.AreEqual(DataRegionConfiguration.DefaultMetricsRateTimeInterval, cfg.MetricsRateTimeInterval);
+            Assert.AreEqual(DataRegionConfiguration.DefaultMetricsSubIntervalCount, cfg.MetricsSubIntervalCount);
+        }
+
+        /// <summary>
+        /// Checks the default properties.
+        /// </summary>
+        /// <param name="cfg">Config.</param>
         private static void CheckDefaultProperties(ClientConnectorConfiguration cfg)
         {
             Assert.AreEqual(ClientConnectorConfiguration.DefaultPort, cfg.Port);
@@ -636,8 +624,7 @@ namespace Apache.Ignite.Core.Tests
 
             foreach (var prop in props.Where(p => p.Name != "SelectorsCount" && p.Name != "ReadStripesNumber" &&
                                                   !p.Name.Contains("ThreadPoolSize") &&
-                                                  !(p.Name == "MaxSize" &&
-                                                    p.DeclaringType == typeof(MemoryPolicyConfiguration))))
+                                                  p.Name != "MaxSize"))
             {
                 var attr = prop.GetCustomAttributes(true).OfType<DefaultValueAttribute>().FirstOrDefault();
                 var propValue = prop.GetValue(obj, null);
@@ -645,7 +632,7 @@ namespace Apache.Ignite.Core.Tests
                 if (attr != null)
                     Assert.AreEqual(attr.Value, propValue, string.Format("{0}.{1}", obj.GetType(), prop.Name));
                 else if (prop.PropertyType.IsValueType)
-                    Assert.AreEqual(Activator.CreateInstance(prop.PropertyType), propValue);
+                    Assert.AreEqual(Activator.CreateInstance(prop.PropertyType), propValue, prop.Name);
                 else
                     Assert.IsNull(propValue);
             }
@@ -757,39 +744,6 @@ namespace Apache.Ignite.Core.Tests
                     ExpirationTimeout = TimeSpan.FromSeconds(5),
                     MaxEventCount = 10
                 },
-                MemoryConfiguration = new MemoryConfiguration
-                {
-                    ConcurrencyLevel = 3,
-                    DefaultMemoryPolicyName = "myDefaultPlc",
-                    PageSize = 2048,
-                    SystemCacheInitialSize = 13 * 1024 * 1024,
-                    SystemCacheMaxSize = 15 * 1024 * 1024,
-                    MemoryPolicies = new[]
-                    {
-                        new MemoryPolicyConfiguration
-                        {
-                            Name = "myDefaultPlc",
-                            PageEvictionMode = DataPageEvictionMode.Disabled,
-                            InitialSize = 340 * 1024 * 1024,
-                            MaxSize = 345 * 1024 * 1024,
-                            EvictionThreshold = 0.88,
-                            EmptyPagesPoolSize = 77,
-                            SwapFilePath = "myPath1",
-                            RateTimeInterval = TimeSpan.FromSeconds(35),
-                            SubIntervals = 7
-                        },
-                        new MemoryPolicyConfiguration
-                        {
-                            Name = "customPlc",
-                            PageEvictionMode = DataPageEvictionMode.Disabled,
-                            MaxSize = 456 * 1024 * 1024,
-                            EvictionThreshold = 0.77,
-                            EmptyPagesPoolSize = 66,
-                            SwapFilePath = "somePath2",
-                            MetricsEnabled = true
-                        }
-                    }
-                },
                 PublicThreadPoolSize = 3,
                 StripedThreadPoolSize = 5,
                 ServiceThreadPoolSize = 6,
@@ -810,31 +764,66 @@ namespace Apache.Ignite.Core.Tests
                     TcpNoDelay = false,
                     SocketSendBufferSize = 4096
                 },
-                PersistentStoreConfiguration = new PersistentStoreConfiguration
+                ConsistentId = new MyConsistentId {Data = "abc"},
+                DataStorageConfiguration = new DataStorageConfiguration
                 {
                     AlwaysWriteFullPages = true,
-                    CheckpointingFrequency = TimeSpan.FromSeconds(25),
-                    CheckpointingPageBufferSize = 28 * 1024 * 1024,
-                    CheckpointingThreads = 2,
+                    CheckpointFrequency = TimeSpan.FromSeconds(25),
+                    CheckpointPageBufferSize = 28 * 1024 * 1024,
+                    CheckpointThreads = 2,
                     LockWaitTime = TimeSpan.FromSeconds(5),
-                    PersistentStorePath = Path.GetTempPath(),
-                    TlbSize = 64 * 1024,
+                    StoragePath = Path.GetTempPath(),
+                    WalThreadLocalBufferSize = 64 * 1024,
                     WalArchivePath = Path.GetTempPath(),
                     WalFlushFrequency = TimeSpan.FromSeconds(3),
                     WalFsyncDelayNanos = 3,
                     WalHistorySize = 10,
-                    WalMode = WalMode.LogOnly,
+                    WalMode = Configuration.WalMode.LogOnly,
                     WalRecordIteratorBufferSize = 32 * 1024 * 1024,
                     WalSegments = 6,
                     WalSegmentSize = 5 * 1024 * 1024,
-                    WalStorePath = Path.GetTempPath(),
+                    WalPath = Path.GetTempPath(),
                     MetricsEnabled = true,
-                    SubIntervals = 7,
-                    RateTimeInterval = TimeSpan.FromSeconds(9),
-                    CheckpointWriteOrder = CheckpointWriteOrder.Random,
-                    WriteThrottlingEnabled = true
-                },
-                ConsistentId = new MyConsistentId {Data = "abc"}
+                    MetricsSubIntervalCount = 7,
+                    MetricsRateTimeInterval = TimeSpan.FromSeconds(9),
+                    CheckpointWriteOrder = Configuration.CheckpointWriteOrder.Random,
+                    WriteThrottlingEnabled = true,
+                    SystemRegionInitialSize = 64 * 1024 * 1024,
+                    SystemRegionMaxSize = 128 * 1024 * 1024,
+                    ConcurrencyLevel = 1,
+                    PageSize = 8 * 1024,
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration
+                    {
+                        Name = "reg1",
+                        EmptyPagesPoolSize = 50,
+                        EvictionThreshold = 0.8,
+                        InitialSize = 100 * 1024 * 1024,
+                        MaxSize = 150 * 1024 * 1024,
+                        MetricsEnabled = true,
+                        PageEvictionMode = Configuration.DataPageEvictionMode.Random2Lru,
+                        PersistenceEnabled = false,
+                        MetricsRateTimeInterval = TimeSpan.FromMinutes(2),
+                        MetricsSubIntervalCount = 6,
+                        SwapPath = IgniteUtils.GetTempDirectoryName()
+                    },
+                    DataRegionConfigurations = new[]
+                    {
+                        new DataRegionConfiguration
+                        {
+                            Name = "reg2",
+                            EmptyPagesPoolSize = 51,
+                            EvictionThreshold = 0.7,
+                            InitialSize = 101 * 1024 * 1024,
+                            MaxSize = 151 * 1024 * 1024,
+                            MetricsEnabled = false,
+                            PageEvictionMode = Configuration.DataPageEvictionMode.RandomLru,
+                            PersistenceEnabled = false,
+                            MetricsRateTimeInterval = TimeSpan.FromMinutes(3),
+                            MetricsSubIntervalCount = 7,
+                            SwapPath = IgniteUtils.GetTempDirectoryName()
+                        }
+                    }
+                }
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 446208a..20a54d0 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,10 @@
     <Compile Include="Binary\BinaryBasicNameMapper.cs" />
     <Compile Include="Binary\TimestampAttribute.cs" />
     <Compile Include="Cache\Configuration\DataPageEvictionMode.cs" />
+    <Compile Include="Configuration\CheckpointWriteOrder.cs" />
+    <Compile Include="Configuration\DataPageEvictionMode.cs" />
+    <Compile Include="Configuration\DataRegionConfiguration.cs" />
+    <Compile Include="Configuration\DataStorageConfiguration.cs" />
     <Compile Include="Cache\Configuration\MemoryPolicyConfiguration.cs" />
     <Compile Include="Cache\Configuration\PartitionLossPolicy.cs" />
     <Compile Include="Cache\IMemoryMetrics.cs" />
@@ -105,6 +109,9 @@
     <Compile Include="Configuration\Package-Info.cs" />
     <Compile Include="Configuration\ClientConnectorConfiguration.cs" />
     <Compile Include="Datastream\DataStreamerDefaults.cs" />
+    <Compile Include="IDataRegionMetrics.cs" />
+    <Compile Include="IDataStorageMetrics.cs" />
+    <Compile Include="Configuration\WalMode.cs" />
     <Compile Include="Impl\Binary\BinaryTypeId.cs" />
     <Compile Include="Impl\Client\Cache\CacheFlags.cs" />
     <Compile Include="Impl\Client\Cache\Query\ClientQueryCursor.cs" />
@@ -113,6 +120,7 @@
     <Compile Include="Impl\Binary\IBinaryProcessor.cs" />
     <Compile Include="Impl\Client\ClientStatus.cs" />
     <Compile Include="Events\LocalEventListener.cs" />
+    <Compile Include="Impl\DataStorageMetrics.cs" />
     <Compile Include="Impl\IIgniteInternal.cs" />
     <Compile Include="Impl\Client\Cache\CacheClient.cs" />
     <Compile Include="Impl\Client\ClientOp.cs" />
@@ -120,6 +128,7 @@
     <Compile Include="Impl\Client\ClientSocket.cs" />
     <Compile Include="Impl\Client\IgniteClient.cs" />
     <Compile Include="Impl\IPlatformTargetInternal.cs" />
+    <Compile Include="Impl\DataRegionMetrics.cs" />
     <Compile Include="Impl\PersistentStore\PersistentStoreMetrics.cs" />
     <Compile Include="Impl\PlatformDisposableTargetAdapter.cs" />
     <Compile Include="Impl\PlatformJniTarget.cs" />
@@ -603,6 +612,7 @@
     </None>
   </ItemGroup>
   <ItemGroup>
+    <Folder Include="Data\" />
     <Folder Include="Impl\Common\JavaObjects\" />
   </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 c6b81f0..e7252b2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using System.Linq;
+    using System.Xml.Serialization;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Affinity;
     using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
@@ -34,6 +35,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using Apache.Ignite.Core.Cache.Expiry;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Cache.Affinity;
@@ -286,7 +288,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             ReadThrough = reader.ReadBoolean();
             WriteThrough = reader.ReadBoolean();
             EnableStatistics = reader.ReadBoolean();
-            MemoryPolicyName = reader.ReadString();
+            DataRegionName = reader.ReadString();
             PartitionLossPolicy = (PartitionLossPolicy) reader.ReadInt();
             GroupName = reader.ReadString();
             CacheStoreFactory = reader.ReadObject<IFactory<ICacheStore>>();
@@ -366,7 +368,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             writer.WriteBoolean(ReadThrough);
             writer.WriteBoolean(WriteThrough);
             writer.WriteBoolean(EnableStatistics);
-            writer.WriteString(MemoryPolicyName);
+            writer.WriteString(DataRegionName);
             writer.WriteInt((int) PartitionLossPolicy);
             writer.WriteString(GroupName);
             writer.WriteObject(CacheStoreFactory);
@@ -747,7 +749,18 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// Gets or sets the name of the <see cref="MemoryPolicyConfiguration"/> for this cache.
         /// See <see cref="IgniteConfiguration.MemoryConfiguration"/>.
         /// </summary>
-        public string MemoryPolicyName { get; set; }
+        [Obsolete("Use DataRegionName.")]
+        [XmlIgnore]
+        public string MemoryPolicyName
+        {
+            get { return DataRegionName; }
+            set { DataRegionName = value; }
+        }
+
+        /// <summary>
+        /// Gets or sets the name of the data region, see <see cref="DataRegionConfiguration"/>.
+        /// </summary>
+        public string DataRegionName { get; set; }
 
         /// <summary>
         /// Gets or sets write coalescing flag for write-behind cache store operations.
@@ -770,7 +783,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <para />
         /// Since underlying cache is shared, the following configuration properties should be the same within group:
         /// <see cref="AffinityFunction"/>, <see cref="CacheMode"/>, <see cref="PartitionLossPolicy"/>,
-        /// <see cref="MemoryPolicyName"/>
+        /// <see cref="DataRegionName"/>
         /// <para />
         /// Grouping caches reduces overall overhead, since internal data structures are shared.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
index a6263d7..57e60d9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
@@ -17,11 +17,14 @@
 
 namespace Apache.Ignite.Core.Cache.Configuration
 {
+    using System;
+
     /// <summary>
     /// Memory page eviction mode.
     /// Only data pages, that store key-value entries, are eligible for eviction.
     /// The other types of pages, like index or system pages, are not evictable.
     /// </summary>
+    [Obsolete("Use Apache.Ignite.Core.Configuration.DataPageEvictionMode")]
     public enum DataPageEvictionMode
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
index 3be6012..12d0002 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Cache.Configuration
 {
+    using System;
     using System.Collections.Generic;
     using System.ComponentModel;
     using System.Diagnostics;
@@ -24,6 +25,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using System.Linq;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Configuration;
 
     /// <summary>
     /// A page memory configuration for an Apache Ignite node. The page memory is a manageable off-heap based
@@ -42,7 +44,10 @@ namespace Apache.Ignite.Core.Cache.Configuration
     /// eviction policies, swapping options, etc. Once you define a new memory region you can bind
     /// particular Ignite caches to it. <para />
     /// To learn more about memory policies refer to <see cref="MemoryPolicyConfiguration" /> documentation.
+    /// <para />
+    /// Obsolete, use <see cref="DataStorageConfiguration"/>.
     /// </summary>
+    [Obsolete("Use DataStorageConfiguration.")]
     public class MemoryConfiguration
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
index 16d8dcc..e204ee7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
@@ -21,12 +21,15 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using System.ComponentModel;
     using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
 
     /// <summary>
     /// Defines page memory policy configuration. See <see cref="MemoryConfiguration.MemoryPolicies"/>.
+    /// Obsolete, use <see cref="DataRegionConfiguration"/>.
     /// </summary>
+    [Obsolete("Use DataRegionConfiguration.")]
     public class MemoryPolicyConfiguration
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IMemoryMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IMemoryMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IMemoryMetrics.cs
index 0298c1f..ff8d64e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IMemoryMetrics.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IMemoryMetrics.cs
@@ -17,9 +17,13 @@
 
 namespace Apache.Ignite.Core.Cache
 {
+    using System;
+
     /// <summary>
     /// Memory usage metrics.
+    /// Obsolete, use <see cref="IDataRegionMetrics"/>.
     /// </summary>
+    [Obsolete("See IDataRegionMetrics.")]
     public interface IMemoryMetrics
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/CheckpointWriteOrder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/CheckpointWriteOrder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/CheckpointWriteOrder.cs
new file mode 100644
index 0000000..5243f4a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/CheckpointWriteOrder.cs
@@ -0,0 +1,37 @@
+/*
+ * 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.Configuration
+{
+    /// <summary>
+    /// Defines checkpoint pages order on disk.
+    /// </summary>
+    public enum CheckpointWriteOrder
+    {
+        /// <summary>
+        /// Pages are written in order provided by checkpoint pages collection iterator
+        /// (which is basically a hashtable).
+        /// </summary>
+        Random,
+
+        /// <summary>
+        /// All checkpoint pages are collected into single list and sorted by page index.
+        /// Provides almost sequential disk writes, which can be much faster on some SSD models.
+        /// </summary>
+        Sequential
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataPageEvictionMode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataPageEvictionMode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataPageEvictionMode.cs
new file mode 100644
index 0000000..ec835bb
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataPageEvictionMode.cs
@@ -0,0 +1,59 @@
+/*
+ * 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.Configuration
+{
+    /// <summary>
+    /// Memory page eviction mode.
+    /// Only data pages, that store key-value entries, are eligible for eviction.
+    /// The other types of pages, like index or system pages, are not evictable.
+    /// </summary>
+    public enum DataPageEvictionMode
+    {
+        /// <summary>
+        /// Eviction is disabled.
+        /// </summary>
+        Disabled,
+
+        /// <summary>
+        /// Random-LRU algorithm.
+        /// <para />
+        /// Once a memory region defined by a memory policy is configured, an off-heap array is allocated to track
+        /// last usage timestamp for every individual data page. The size of the array equals to
+        /// <see cref="DataRegionConfiguration.MaxSize"/> / <see cref="DataStorageConfiguration.PageSize"/>.
+        /// <para />
+        /// When a data page is accessed, its timestamp gets updated in the tracking array. The page index in the
+        /// tracking array equals to pageAddress / <see cref="DataRegionConfiguration.MaxSize"/>.
+        /// <para />
+        /// When some pages need to be evicted, the algorithm randomly chooses 5 indexes from the tracking array and
+        /// evicts a page with the latest timestamp. If some of the indexes point to non-data pages
+        /// (index or system pages) then the algorithm picks other pages.
+        /// </summary>
+        RandomLru,
+
+        /// <summary>
+        /// Activates Random-2-LRU algorithm which is a scan resistant version of Random-LRU.
+        /// <para />
+        /// This algorithm differs from Random-LRU only in a way that two latest access timestamps are stored for every
+        /// data page. At the eviction time, a minimum between two latest timestamps is taken for further
+        /// comparison with minimums of other pages that might be evicted. LRU-2 outperforms LRU by
+        /// resolving "one-hit wonder" problem - if a data page is accessed rarely, but accidentally accessed once,
+        /// its protected from eviction for a long time.
+        /// </summary>
+        Random2Lru
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs
new file mode 100644
index 0000000..5c4240e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs
@@ -0,0 +1,213 @@
+/*
+ * 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.Configuration
+{
+    using System;
+    using System.ComponentModel;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Binary;
+
+    /// <summary>
+    /// Defines custom data region configuration for Apache Ignite page memory
+    /// (see <see cref="DataStorageConfiguration"/>). 
+    /// <para />
+    /// For each configured data region Apache Ignite instantiates respective memory regions with different
+    /// parameters like maximum size, eviction policy, swapping options, etc.
+    /// An Apache Ignite cache can be mapped to a particular region using
+    /// <see cref="CacheConfiguration.DataRegionName"/> method.
+    /// </summary>
+    public class DataRegionConfiguration
+    {
+        /// <summary>
+        /// Default value for <see cref="PersistenceEnabled"/>.
+        /// </summary>
+        public const bool DefaultPersistenceEnabled = false;
+
+        /// <summary>
+        /// The default eviction threshold.
+        /// </summary>
+        public const double DefaultEvictionThreshold = 0.9;
+
+        /// <summary>
+        /// The default empty pages pool size.
+        /// </summary>
+        public const int DefaultEmptyPagesPoolSize = 100;
+
+        /// <summary>
+        /// The default initial size.
+        /// </summary>
+        public const long DefaultInitialSize = 256 * 1024 * 1024;
+
+        /// <summary>
+        /// The default maximum size, equals to 20% of total RAM.
+        /// </summary>
+        public static readonly long DefaultMaxSize = (long)((long)NativeMethods.GetTotalPhysicalMemory() * 0.2);
+
+        /// <summary>
+        /// The default sub intervals.
+        /// </summary>
+        [SuppressMessage("Microsoft.Naming", "CA1702:CompoundWordsShouldBeCasedCorrectly",
+            Justification = "Consistency with Java config")]
+        public const int DefaultMetricsSubIntervalCount = 5;
+
+        /// <summary>
+        /// The default rate time interval.
+        /// </summary>
+        public static readonly TimeSpan DefaultMetricsRateTimeInterval = TimeSpan.FromSeconds(60);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataRegionConfiguration"/> class.
+        /// </summary>
+        public DataRegionConfiguration()
+        {
+            PersistenceEnabled = DefaultPersistenceEnabled;
+            EvictionThreshold = DefaultEvictionThreshold;
+            EmptyPagesPoolSize = DefaultEmptyPagesPoolSize;
+            InitialSize = DefaultInitialSize;
+            MaxSize = DefaultMaxSize;
+            MetricsSubIntervalCount = DefaultMetricsSubIntervalCount;
+            MetricsRateTimeInterval = DefaultMetricsRateTimeInterval;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataRegionConfiguration"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        internal DataRegionConfiguration(IBinaryRawReader reader)
+        {
+            Name = reader.ReadString();
+            PersistenceEnabled = reader.ReadBoolean();
+            InitialSize = reader.ReadLong();
+            MaxSize = reader.ReadLong();
+            SwapPath = reader.ReadString();
+            PageEvictionMode = (DataPageEvictionMode)reader.ReadInt();
+            EvictionThreshold = reader.ReadDouble();
+            EmptyPagesPoolSize = reader.ReadInt();
+            MetricsEnabled = reader.ReadBoolean();
+            MetricsSubIntervalCount = reader.ReadInt();
+            MetricsRateTimeInterval = reader.ReadLongAsTimespan();
+        }
+
+        /// <summary>
+        /// Writes this instance to a writer.
+        /// </summary>
+        internal void Write(IBinaryRawWriter writer)
+        {
+            writer.WriteString(Name);
+            writer.WriteBoolean(PersistenceEnabled);
+            writer.WriteLong(InitialSize);
+            writer.WriteLong(MaxSize);
+            writer.WriteString(SwapPath);
+            writer.WriteInt((int)PageEvictionMode);
+            writer.WriteDouble(EvictionThreshold);
+            writer.WriteInt(EmptyPagesPoolSize);
+            writer.WriteBoolean(MetricsEnabled);
+            writer.WriteInt(MetricsSubIntervalCount);
+            writer.WriteTimeSpanAsLong(MetricsRateTimeInterval);
+        }
+
+        /// <summary>
+        /// Gets or sets the data region name.
+        /// </summary>
+        public string Name { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether disk persistence is enabled for this region.
+        /// Default is <see cref="DefaultPersistenceEnabled"/>.
+        /// </summary>
+        [DefaultValue(DefaultPersistenceEnabled)]
+        public bool PersistenceEnabled { get; set; }
+
+        /// <summary>
+        /// Gets or sets initial memory region size.
+        /// When the used memory size exceeds this value, new chunks of memory will be allocated.
+        /// </summary>
+        [DefaultValue(DefaultInitialSize)]
+        public long InitialSize { get; set; }
+
+        /// <summary>
+        /// Sets maximum memory region size. The total size should not be less
+        /// than 10 MB due to internal data structures overhead.
+        /// </summary>
+        public long MaxSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the the path to the directory for memory-mapped files.
+        /// <para />
+        /// Null for no swap.
+        /// </summary>
+        public string SwapPath { get; set; }
+
+        /// <summary>
+        /// Gets or sets the page eviction mode. If <see cref="DataPageEvictionMode.Disabled"/> is used (default)
+        /// then an out of memory exception will be thrown if the memory region usage 
+        /// goes beyond <see cref="MaxSize"/>.
+        /// </summary>
+        public DataPageEvictionMode PageEvictionMode { get; set; }
+
+        /// <summary>
+        /// Gets or sets the threshold for memory pages eviction initiation. For instance, if the threshold is 0.9
+        /// it means that the page memory will start the eviction only after 90% of the memory region is occupied.
+        /// </summary>
+        [DefaultValue(DefaultEvictionThreshold)]
+        public double EvictionThreshold { get; set; }
+
+        /// <summary>
+        /// Gets or sets the minimal number of empty pages to be present in reuse lists for this data region.
+        /// This parameter ensures that Ignite will be able to successfully evict old data entries when the size of
+        /// (key, value) pair is slightly larger than page size / 2.
+        /// Increase this parameter if cache can contain very big entries (total size of pages in this pool
+        /// should be enough to contain largest cache entry).
+        /// </summary>
+        [DefaultValue(DefaultEmptyPagesPoolSize)]
+        public int EmptyPagesPoolSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether memory metrics should be enabled.
+        /// <para />
+        /// Metrics can be retrieved with <see cref="IIgnite.GetDataRegionMetrics()"/> method.
+        /// </summary>
+        public bool MetricsEnabled { get; set; }
+
+        /// <summary>
+        /// Gets or sets the rate time interval for <see cref="IDataRegionMetrics.AllocationRate"/>
+        /// and <see cref="IDataRegionMetrics.EvictionRate"/> monitoring purposes.
+        /// <para />
+        /// For instance, after setting the interval to 60 seconds, subsequent calls
+        /// to <see cref="IDataRegionMetrics.AllocationRate"/> will return average allocation
+        /// rate (pages per second) for the last minute.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "00:01:00")]
+        public TimeSpan MetricsRateTimeInterval { get; set; }
+
+        /// <summary>
+        /// Gets or sets the number of sub intervals to split <see cref="MetricsRateTimeInterval"/> into to calculate 
+        /// <see cref="IDataRegionMetrics.AllocationRate"/> and <see cref="IDataRegionMetrics.EvictionRate"/>.
+        /// <para />
+        /// Bigger value results in more accurate metrics.
+        /// </summary>
+        [DefaultValue(DefaultMetricsSubIntervalCount)]
+        [SuppressMessage("Microsoft.Naming", "CA1702:CompoundWordsShouldBeCasedCorrectly",
+            Justification = "Consistency with Java config")]
+        public int MetricsSubIntervalCount { get; set; }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
new file mode 100644
index 0000000..17b4ada
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
@@ -0,0 +1,466 @@
+/*
+ * 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.Configuration
+{
+    using System;
+    using System.Collections.Generic;
+    using System.ComponentModel;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Binary;
+
+    /// <summary>
+    /// Data storage configuration for Ignite page memory.
+    /// <para />
+    /// The page memory is a manageable off-heap based memory architecture that divides all expandable data
+    /// regions into pages of fixed size. An individual page can store one or many cache key-value entries
+    /// that allows reusing the memory in the most efficient way and avoid memory fragmentation issues.
+    /// <para />
+    /// By default, the page memory allocates a single expandable data region. All the caches that will be
+    /// configured in an application will be mapped to this data region by default, thus, all the cache data
+    /// will reside in that data region.
+    /// </summary>
+    public class DataStorageConfiguration
+    {
+        /// <summary>
+        /// Default value for <see cref="CheckpointThreads"/>.
+        /// </summary>
+        public const int DefaultCheckpointThreads = 4;
+
+        /// <summary>
+        /// Default name is assigned to default data region if no user-defined
+        /// <see cref="DefaultDataRegionConfiguration"/> is specified.
+        /// </summary>
+        public const string DefaultDataRegionName = "default";
+
+        /// <summary>
+        /// Default value for <see cref="CheckpointFrequency"/>.
+        /// </summary>
+        public static readonly TimeSpan DefaultCheckpointFrequency = TimeSpan.FromSeconds(180);
+
+        /// <summary>
+        /// Default value for <see cref="LockWaitTime"/>.
+        /// </summary>
+        public static readonly TimeSpan DefaultLockWaitTime = TimeSpan.FromSeconds(10);
+
+        /// <summary>
+        /// Default value for <see cref="WalHistorySize"/>.
+        /// </summary>
+        public const int DefaultWalHistorySize = 20;
+
+        /// <summary>
+        /// Default value for <see cref="WalSegments"/>.
+        /// </summary>
+        public const int DefaultWalSegments = 10;
+
+        /// <summary>
+        /// Default value for <see cref="WalSegmentSize"/>.
+        /// </summary>
+        public const int DefaultWalSegmentSize = 64 * 1024 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="WalThreadLocalBufferSize"/>.
+        /// </summary>
+        public const int DefaultTlbSize = 128 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="WalFlushFrequency"/>.
+        /// </summary>
+        public static readonly TimeSpan DefaultWalFlushFrequency = TimeSpan.FromSeconds(2);
+
+        /// <summary>
+        /// Default value for <see cref="WalRecordIteratorBufferSize"/>.
+        /// </summary>
+        public const int DefaultWalRecordIteratorBufferSize = 64 * 1024 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="WalFsyncDelayNanos"/>.
+        /// </summary>
+        public const long DefaultWalFsyncDelayNanos = 1000;
+
+        /// <summary>
+        /// The default sub intervals.
+        /// </summary>
+        [SuppressMessage("Microsoft.Naming", "CA1702:CompoundWordsShouldBeCasedCorrectly",
+            Justification = "Consistency with Java config")]
+        public const int DefaultMetricsSubIntervalCount = 5;
+
+        /// <summary>
+        /// The default rate time interval.
+        /// </summary>
+        public static readonly TimeSpan DefaultMetricsRateTimeInterval = TimeSpan.FromSeconds(60);
+
+        /// <summary>
+        /// Default value for <see cref="WalPath"/>.
+        /// </summary>
+        public const string DefaultWalPath = "db/wal";
+
+        /// <summary>
+        /// Default value for <see cref="WalArchivePath"/>.
+        /// </summary>
+        public const string DefaultWalArchivePath = "db/wal/archive";
+
+        /// <summary>
+        /// Default value for <see cref="WalMode"/>.
+        /// </summary>
+        public const WalMode DefaultWalMode = WalMode.Default;
+
+        /// <summary>
+        /// Default value for <see cref="CheckpointWriteOrder"/>.
+        /// </summary>
+        public const CheckpointWriteOrder DefaultCheckpointWriteOrder = CheckpointWriteOrder.Sequential;
+
+        /// <summary>
+        /// Default value for <see cref="WriteThrottlingEnabled"/>.
+        /// </summary>
+        public const bool DefaultWriteThrottlingEnabled = false;
+
+        /// <summary>
+        /// Default size of a memory chunk reserved for system cache initially.
+        /// </summary>
+        public const long DefaultSystemRegionInitialSize = 40 * 1024 * 1024;
+
+        /// <summary>
+        /// Default max size of a memory chunk for the system cache.
+        /// </summary>
+        public const long DefaultSystemRegionMaxSize = 100 * 1024 * 1024;
+
+        /// <summary>
+        /// The default page size.
+        /// </summary>
+        public const int DefaultPageSize = 4 * 1024;
+
+        /// <summary>
+        /// The default concurrency level.
+        /// </summary>
+        public const int DefaultConcurrencyLevel = 0;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataStorageConfiguration"/> class.
+        /// </summary>
+        public DataStorageConfiguration()
+        {
+            CheckpointThreads = DefaultCheckpointThreads;
+            CheckpointFrequency = DefaultCheckpointFrequency;
+            LockWaitTime = DefaultLockWaitTime;
+            WalHistorySize = DefaultWalHistorySize;
+            WalSegments = DefaultWalSegments;
+            WalSegmentSize = DefaultWalSegmentSize;
+            WalThreadLocalBufferSize = DefaultTlbSize;
+            WalFlushFrequency = DefaultWalFlushFrequency;
+            WalRecordIteratorBufferSize = DefaultWalRecordIteratorBufferSize;
+            WalFsyncDelayNanos = DefaultWalFsyncDelayNanos;
+            MetricsRateTimeInterval = DefaultMetricsRateTimeInterval;
+            MetricsSubIntervalCount = DefaultMetricsSubIntervalCount;
+            WalArchivePath = DefaultWalArchivePath;
+            WalPath = DefaultWalPath;
+            CheckpointWriteOrder = DefaultCheckpointWriteOrder;
+            WriteThrottlingEnabled = DefaultWriteThrottlingEnabled;
+            SystemRegionInitialSize = DefaultSystemRegionInitialSize;
+            SystemRegionMaxSize = DefaultSystemRegionMaxSize;
+            PageSize = DefaultPageSize;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataStorageConfiguration"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        internal DataStorageConfiguration(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            StoragePath = reader.ReadString();
+            CheckpointFrequency = reader.ReadLongAsTimespan();
+            CheckpointPageBufferSize = reader.ReadLong();
+            CheckpointThreads = reader.ReadInt();
+            LockWaitTime = reader.ReadLongAsTimespan();
+            WalHistorySize = reader.ReadInt();
+            WalSegments = reader.ReadInt();
+            WalSegmentSize = reader.ReadInt();
+            WalPath = reader.ReadString();
+            WalArchivePath = reader.ReadString();
+            WalMode = (WalMode)reader.ReadInt();
+            WalThreadLocalBufferSize = reader.ReadInt();
+            WalFlushFrequency = reader.ReadLongAsTimespan();
+            WalFsyncDelayNanos = reader.ReadLong();
+            WalRecordIteratorBufferSize = reader.ReadInt();
+            AlwaysWriteFullPages = reader.ReadBoolean();
+            MetricsEnabled = reader.ReadBoolean();
+            MetricsSubIntervalCount = reader.ReadInt();
+            MetricsRateTimeInterval = reader.ReadLongAsTimespan();
+            CheckpointWriteOrder = (CheckpointWriteOrder)reader.ReadInt();
+            WriteThrottlingEnabled = reader.ReadBoolean();
+
+            SystemRegionInitialSize = reader.ReadLong();
+            SystemRegionMaxSize = reader.ReadLong();
+            PageSize = reader.ReadInt();
+            ConcurrencyLevel = reader.ReadInt();
+
+            var count = reader.ReadInt();
+
+            if (count > 0)
+            {
+                DataRegionConfigurations = Enumerable.Range(0, count)
+                    .Select(x => new DataRegionConfiguration(reader))
+                    .ToArray();
+            }
+
+            if (reader.ReadBoolean())
+            {
+                DefaultDataRegionConfiguration = new DataRegionConfiguration(reader);
+            }
+        }
+
+        /// <summary>
+        /// Writes this instance to the specified writer.
+        /// </summary>
+        /// <param name="writer">The writer.</param>
+        internal void Write(IBinaryRawWriter writer)
+        {
+            Debug.Assert(writer != null);
+
+            writer.WriteString(StoragePath);
+            writer.WriteTimeSpanAsLong(CheckpointFrequency);
+            writer.WriteLong(CheckpointPageBufferSize);
+            writer.WriteInt(CheckpointThreads);
+            writer.WriteTimeSpanAsLong(LockWaitTime);
+            writer.WriteInt(WalHistorySize);
+            writer.WriteInt(WalSegments);
+            writer.WriteInt(WalSegmentSize);
+            writer.WriteString(WalPath);
+            writer.WriteString(WalArchivePath);
+            writer.WriteInt((int)WalMode);
+            writer.WriteInt(WalThreadLocalBufferSize);
+            writer.WriteTimeSpanAsLong(WalFlushFrequency);
+            writer.WriteLong(WalFsyncDelayNanos);
+            writer.WriteInt(WalRecordIteratorBufferSize);
+            writer.WriteBoolean(AlwaysWriteFullPages);
+            writer.WriteBoolean(MetricsEnabled);
+            writer.WriteInt(MetricsSubIntervalCount);
+            writer.WriteTimeSpanAsLong(MetricsRateTimeInterval);
+            writer.WriteInt((int)CheckpointWriteOrder);
+            writer.WriteBoolean(WriteThrottlingEnabled);
+
+            writer.WriteLong(SystemRegionInitialSize);
+            writer.WriteLong(SystemRegionMaxSize);
+            writer.WriteInt(PageSize);
+            writer.WriteInt(ConcurrencyLevel);
+
+            if (DataRegionConfigurations != null)
+            {
+                writer.WriteInt(DataRegionConfigurations.Count);
+
+                foreach (var region in DataRegionConfigurations)
+                {
+                    if (region == null)
+                    {
+                        throw new IgniteException(
+                            "DataStorageConfiguration.DataRegionConfigurations must not contain null items.");
+                    }
+
+                    region.Write(writer);
+                }
+            }
+            else
+            {
+                writer.WriteInt(0);
+            }
+
+            if (DefaultDataRegionConfiguration != null)
+            {
+                writer.WriteBoolean(true);
+                DefaultDataRegionConfiguration.Write(writer);
+            }
+            else
+            {
+                writer.WriteBoolean(false);
+            }
+        }
+
+        /// <summary>
+        /// Gets or sets the path where data and indexes will be persisted.
+        /// </summary>
+        public string StoragePath { get; set; }
+
+        /// <summary>
+        /// Gets or sets the checkpointing frequency which is a minimal interval when the dirty pages will be written
+        /// to the Persistent Store.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "00:03:00")]
+        public TimeSpan CheckpointFrequency { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the checkpointing page buffer.
+        /// <para />
+        /// Default is <c>0</c>: Ignite will choose buffer size automatically.
+        /// </summary>
+        public long CheckpointPageBufferSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the number of threads for checkpointing.
+        /// </summary>
+        [DefaultValue(DefaultCheckpointThreads)]
+        public int CheckpointThreads { get; set; }
+
+        /// <summary>
+        /// Gets or sets the persistent manager file lock wait time.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "00:00:10")]
+        public TimeSpan LockWaitTime { get; set; }
+
+        /// <summary>
+        /// Gets or sets the number of checkpoints to store in WAL (Write Ahead Log) history.
+        /// </summary>
+        [DefaultValue(DefaultWalHistorySize)]
+        public int WalHistorySize { get; set; }
+
+        /// <summary>
+        /// Gets or sets a number of WAL (Write Ahead Log) segments to work with.
+        /// For performance reasons, the whole WAL is split into files of fixed length called segments.
+        /// </summary>
+        [DefaultValue(DefaultWalSegments)]
+        public int WalSegments { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the WAL (Write Ahead Log) segment.
+        /// For performance reasons, the whole WAL is split into files of fixed length called segments.
+        /// </summary>
+        [DefaultValue(DefaultWalSegmentSize)]
+        public int WalSegmentSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the path to the directory where WAL (Write Ahead Log) is stored.
+        /// </summary>
+        [DefaultValue(DefaultWalPath)]
+        public string WalPath { get; set; }
+
+        /// <summary>
+        /// Gets or sets the path to the directory where WAL (Write Ahead Log) archive is stored.
+        /// Every WAL segment will be fully copied to this directory before it can be reused for WAL purposes.
+        /// </summary>
+        [DefaultValue(DefaultWalArchivePath)]
+        public string WalArchivePath { get; set; }
+
+        /// <summary>
+        /// Gets or sets the WAL (Write Ahead Log) mode.
+        /// </summary>
+        [DefaultValue(DefaultWalMode)]
+        public WalMode WalMode { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the TLB (Thread-Local Buffer), in bytes.
+        /// </summary>
+        [DefaultValue(DefaultTlbSize)]
+        public int WalThreadLocalBufferSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the WAL (Write Ahead Log) flush frequency.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "00:00:02")]
+        public TimeSpan WalFlushFrequency { get; set; }
+
+        /// <summary>
+        /// Gets or sets the WAL (Write Ahead Log) fsync (disk sync) delay, in nanoseconds
+        /// </summary>
+        [DefaultValue(DefaultWalFsyncDelayNanos)]
+        public long WalFsyncDelayNanos { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the WAL (Write Ahead Log) record iterator buffer, in bytes.
+        /// </summary>
+        [DefaultValue(DefaultWalRecordIteratorBufferSize)]
+        public int WalRecordIteratorBufferSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether full pages should always be written.
+        /// </summary>
+        public bool AlwaysWriteFullPages { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether to enable data storage metrics.
+        /// See <see cref="IIgnite.GetDataStorageMetrics"/>.
+        /// </summary>
+        public bool MetricsEnabled { get; set; }
+
+        /// <summary>
+        /// Gets or sets the length of the time interval for rate-based metrics.
+        /// This interval defines a window over which hits will be tracked.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "00:01:00")]
+        public TimeSpan MetricsRateTimeInterval { get; set; }
+
+        /// <summary>
+        /// Number of sub-intervals to split the <see cref="MetricsRateTimeInterval"/> into to track the update history.
+        /// </summary>
+        [DefaultValue(DefaultMetricsSubIntervalCount)]
+        [SuppressMessage("Microsoft.Naming", "CA1702:CompoundWordsShouldBeCasedCorrectly",
+            Justification = "Consistency with Java config")]
+        public int MetricsSubIntervalCount { get; set; }
+
+        /// <summary>
+        /// Gets or sets the checkpoint page write order on disk.
+        /// </summary>
+        [DefaultValue(DefaultCheckpointWriteOrder)]
+        public CheckpointWriteOrder CheckpointWriteOrder { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether threads that generate dirty
+        /// pages too fast during ongoing checkpoint will be throttled.
+        /// </summary>
+        [DefaultValue(DefaultWriteThrottlingEnabled)]
+        public bool WriteThrottlingEnabled { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of a memory chunk reserved for system needs.
+        /// </summary>
+        [DefaultValue(DefaultSystemRegionInitialSize)]
+        public long SystemRegionInitialSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum memory region size reserved for system needs.
+        /// </summary>
+        [DefaultValue(DefaultSystemRegionMaxSize)]
+        public long SystemRegionMaxSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the memory page.
+        /// </summary>
+        [DefaultValue(DefaultPageSize)]
+        public int PageSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the number of concurrent segments in Ignite internal page mapping tables.
+        /// </summary>
+        [DefaultValue(DefaultConcurrencyLevel)]
+        public int ConcurrencyLevel { get; set; }
+
+        /// <summary>
+        /// Gets or sets the data region configurations.
+        /// </summary>
+        [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")]
+        public ICollection<DataRegionConfiguration> DataRegionConfigurations { get; set; }
+
+        /// <summary>
+        /// Gets or sets the default region configuration.
+        /// </summary>
+        public DataRegionConfiguration DefaultDataRegionConfiguration { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/WalMode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/WalMode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/WalMode.cs
new file mode 100644
index 0000000..d6e4532
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/WalMode.cs
@@ -0,0 +1,45 @@
+/*
+ * 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.Configuration
+{
+    /// <summary>
+    /// Write Ahead Log mode.
+    /// </summary>
+    public enum WalMode
+    {
+        /// <summary>
+        /// Default mode: full-sync disk writes. These writes survive power loss scenarios.
+        /// </summary>
+        Default,
+
+        /// <summary>
+        /// Log only mode: flushes application buffers. These writes survive process crash.
+        /// </summary>
+        LogOnly,
+
+        /// <summary>
+        /// Background mode. Does not force application buffer flush. Data may be lost in case of process crash.
+        /// </summary>
+        Background,
+
+        /// <summary>
+        /// WAL disabled.
+        /// </summary>
+        None
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/IDataRegionMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IDataRegionMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IDataRegionMetrics.cs
new file mode 100644
index 0000000..0cb6192
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IDataRegionMetrics.cs
@@ -0,0 +1,55 @@
+/*
+ * 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
+{
+    /// <summary>
+    /// Memory usage metrics.
+    /// </summary>
+    public interface IDataRegionMetrics
+    {
+        /// <summary>
+        /// Gets the memory policy name.
+        /// </summary>
+        string Name { get; }
+
+        /// <summary>
+        /// Gets the count of allocated pages.
+        /// </summary>
+        long TotalAllocatedPages { get; }
+
+        /// <summary>
+        /// Gets the allocation rate, in pages per second.
+        /// </summary>
+        float AllocationRate { get; }
+
+        /// <summary>
+        /// Gets the eviction rate, in pages per second.
+        /// </summary>
+        float EvictionRate { get; }
+
+        /// <summary>
+        /// Gets the percentage of pages fully occupied by entries that are larger than page.
+        /// </summary>
+        float LargeEntriesPagesPercentage { get; }
+
+        /// <summary>
+        /// Gets the page fill factor: free space to overall size ratio across all pages.
+        /// </summary>
+        float PageFillFactor { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/IDataStorageMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IDataStorageMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IDataStorageMetrics.cs
new file mode 100644
index 0000000..6f3562d
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IDataStorageMetrics.cs
@@ -0,0 +1,87 @@
+/*
+ * 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
+{
+    using System;
+
+    /// <summary>
+    /// Persistent store metrics.
+    /// </summary>
+    public interface IDataStorageMetrics
+    {
+        /// <summary>
+        /// Gets the average number of WAL records per second written during the last time interval. 
+        /// </summary>
+        float WalLoggingRate { get; }
+
+        /// <summary>
+        /// Gets the average number of bytes per second written during the last time interval.
+        /// </summary>
+        float WalWritingRate { get; }
+
+        /// <summary>
+        /// Gets the current number of WAL segments in the WAL archive.
+        /// </summary>
+        int WalArchiveSegments { get; }
+
+        /// <summary>
+        /// Gets the average WAL fsync duration in microseconds over the last time interval.
+        /// </summary>
+        float WalFsyncTimeAverage { get; }
+
+        /// <summary>
+        /// Gets the duration of the last checkpoint.
+        /// </summary>
+        TimeSpan LastCheckpointDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of last checkpoint lock wait.
+        /// </summary>
+        TimeSpan LastCheckpointLockWaitDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of last checkpoint mark phase.
+        /// </summary>
+        TimeSpan LastCheckpointMarkDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of last checkpoint pages write phase.
+        /// </summary>
+        TimeSpan LastCheckpointPagesWriteDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of the sync phase of the last checkpoint.
+        /// </summary>
+        TimeSpan LastCheckpointFsyncDuration { get; }
+
+        /// <summary>
+        /// Gets the total number of pages written during the last checkpoint.
+        /// </summary>
+        long LastCheckpointTotalPagesNumber { get; }
+
+        /// <summary>
+        /// Gets the number of data pages written during the last checkpoint.
+        /// </summary>
+        long LastCheckpointDataPagesNumber { get; }
+
+        /// <summary>
+        /// Gets the number of pages copied to a temporary checkpoint buffer during the last checkpoint.
+        /// </summary>
+        long LastCheckpointCopiedOnWritePagesNumber { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 9548aca..f61da06 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
@@ -26,6 +26,7 @@ namespace Apache.Ignite.Core
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Datastream;
     using Apache.Ignite.Core.DataStructures;
     using Apache.Ignite.Core.Events;
@@ -336,7 +337,10 @@ namespace Apache.Ignite.Core
         /// Gets a collection of memory metrics, one for each <see cref="MemoryConfiguration.MemoryPolicies"/>.
         /// <para />
         /// Memory metrics should be enabled with <see cref="MemoryPolicyConfiguration.MetricsEnabled"/>.
+        /// <para />
+        /// Obsolete, use <see cref="GetDataRegionMetrics()"/>.
         /// </summary>
+        [Obsolete("Use GetDataRegionMetrics.")]
         ICollection<IMemoryMetrics> GetMemoryMetrics();
 
         /// <summary>
@@ -344,8 +348,11 @@ namespace Apache.Ignite.Core
         /// <para />
         /// To get metrics for the default memory region,
         /// use <see cref="MemoryConfiguration.DefaultMemoryPolicyName"/>.
+        /// <para />
+        /// Obsolete, use <see cref="GetDataRegionMetrics(string)"/>.
         /// </summary>
         /// <param name="memoryPolicyName">Name of the memory policy.</param>
+        [Obsolete("Use GetDataRegionMetrics.")]
         IMemoryMetrics GetMemoryMetrics(string memoryPolicyName);
 
         /// <summary>
@@ -367,6 +374,32 @@ namespace Apache.Ignite.Core
         /// To enable metrics set <see cref="PersistentStoreConfiguration.MetricsEnabled"/> property
         /// in <see cref="IgniteConfiguration.PersistentStoreConfiguration"/>.
         /// </summary>
+        [Obsolete("Use GetDataStorageMetrics.")]
         IPersistentStoreMetrics GetPersistentStoreMetrics();
+
+        /// <summary>
+        /// Gets a collection of memory metrics, one for each 
+        /// <see cref="DataStorageConfiguration.DataRegionConfigurations"/>.
+        /// <para />
+        /// Metrics should be enabled with <see cref="DataStorageConfiguration.MetricsEnabled"/>.
+        /// </summary>
+        ICollection<IDataRegionMetrics> GetDataRegionMetrics();
+
+        /// <summary>
+        /// Gets the memory metrics for the specified data region.
+        /// <para />
+        /// To get metrics for the default memory region,
+        /// use <see cref="DataStorageConfiguration.DefaultDataRegionName"/>.
+        /// </summary>
+        /// <param name="dataRegionName">Name of the data region.</param>
+        IDataRegionMetrics GetDataRegionMetrics(string dataRegionName);
+
+        /// <summary>
+        /// Gets the persistent store metrics.
+        /// <para />
+        /// To enable metrics set <see cref="DataStorageConfiguration.MetricsEnabled"/> property
+        /// in <see cref="IgniteConfiguration.DataStorageConfiguration"/>.
+        /// </summary>
+        IDataStorageMetrics GetDataStorageMetrics();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 b0fe0df..a6ff324 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -26,6 +26,7 @@ namespace Apache.Ignite.Core
     using System.Linq;
     using System.Text;
     using System.Xml;
+    using System.Xml.Serialization;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Configuration;
@@ -439,6 +440,7 @@ namespace Apache.Ignite.Core
                 memEventStorage.Write(writer);
             }
 
+#pragma warning disable 618  // Obsolete
             if (MemoryConfiguration != null)
             {
                 writer.WriteBoolean(true);
@@ -448,6 +450,7 @@ namespace Apache.Ignite.Core
             {
                 writer.WriteBoolean(false);
             }
+#pragma warning restore 618
 
             // SQL connector.
 #pragma warning disable 618  // Obsolete
@@ -476,6 +479,7 @@ namespace Apache.Ignite.Core
             writer.WriteBoolean(ClientConnectorConfigurationEnabled);
 
             // Persistence.
+#pragma warning disable 618  // Obsolete
             if (PersistentStoreConfiguration != null)
             {
                 writer.WriteBoolean(true);
@@ -485,6 +489,18 @@ namespace Apache.Ignite.Core
             {
                 writer.WriteBoolean(false);
             }
+#pragma warning restore 618
+
+            // Data storage.
+            if (DataStorageConfiguration != null)
+            {
+                writer.WriteBoolean(true);
+                DataStorageConfiguration.Write(writer);
+            }
+            else
+            {
+                writer.WriteBoolean(false);
+            }
 
             // Plugins (should be last).
             if (PluginConfigurations != null)
@@ -675,7 +691,9 @@ namespace Apache.Ignite.Core
 
             if (r.ReadBoolean())
             {
+#pragma warning disable 618  // Obsolete
                 MemoryConfiguration = new MemoryConfiguration(r);
+#pragma warning restore 618  // Obsolete
             }
 
             // SQL.
@@ -697,7 +715,15 @@ namespace Apache.Ignite.Core
             // Persistence.
             if (r.ReadBoolean())
             {
+#pragma warning disable 618 // Obsolete
                 PersistentStoreConfiguration = new PersistentStoreConfiguration(r);
+#pragma warning restore 618
+            }
+
+            // Data storage.
+            if (r.ReadBoolean())
+            {
+                DataStorageConfiguration = new DataStorageConfiguration(r);
             }
         }
 
@@ -793,6 +819,7 @@ namespace Apache.Ignite.Core
         /// This property is used to when there are multiple Ignite nodes in one process to distinguish them.
         /// </summary>
         [Obsolete("Use IgniteInstanceName instead.")]
+        [XmlIgnore]
         public string GridName
         {
             get { return IgniteInstanceName; }
@@ -1243,10 +1270,18 @@ namespace Apache.Ignite.Core
         /// <summary>
         /// Gets or sets the page memory configuration.
         /// <see cref="MemoryConfiguration"/> for more details.
+        /// <para />
+        /// Obsolete, use <see cref="DataStorageConfiguration"/>.
         /// </summary>
+        [Obsolete("Use DataStorageConfiguration.")]
         public MemoryConfiguration MemoryConfiguration { get; set; }
 
         /// <summary>
+        /// Gets or sets the data storage configuration.
+        /// </summary>
+        public DataStorageConfiguration DataStorageConfiguration { get; set; }
+
+        /// <summary>
         /// Gets or sets a value indicating how user assemblies should be loaded on remote nodes.
         /// <para />
         /// For example, when executing <see cref="ICompute.Call{TRes}(IComputeFunc{TRes})"/>,
@@ -1374,14 +1409,17 @@ namespace Apache.Ignite.Core
 
         /// <summary>
         /// Gets or sets the persistent store configuration.
+        /// <para />
+        /// Obsolete, use <see cref="DataStorageConfiguration"/>.
         /// </summary>
+        [Obsolete("Use DataStorageConfiguration.")]
         public PersistentStoreConfiguration PersistentStoreConfiguration { get; set; }
 
         /// <summary>
         /// Gets or sets a value indicating whether grid should be active on start.
         /// See also <see cref="IIgnite.IsActive"/> and <see cref="IIgnite.SetActive"/>.
         /// <para />
-        /// This property is ignored when <see cref="PersistentStoreConfiguration"/> is present:
+        /// This property is ignored when <see cref="DataStorageConfiguration"/> is present:
         /// cluster is always inactive on start when Ignite Persistence is enabled.
         /// </summary>
         [DefaultValue(DefaultIsActiveOnStart)]