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:20 UTC

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

Repository: ignite
Updated Branches:
  refs/heads/master ec9a945a1 -> ab08be83c


http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 988fa1f..6ede267 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -759,9 +759,9 @@
                                             <xs:documentation>Value indicating whether statistics gathering is enabled on a cache. These statistics can be retrieved via ICache.GetMetrics().</xs:documentation>
                                         </xs:annotation>
                                     </xs:attribute>
-                                    <xs:attribute name="memoryPolicyName" type="xs:string">
+                                    <xs:attribute name="dataRegionName" type="xs:string">
                                         <xs:annotation>
-                                            <xs:documentation>Name of the MemoryPolicyConfiguration for this cache.</xs:documentation>
+                                            <xs:documentation>Name of the DataRegionConfiguration for this cache.</xs:documentation>
                                         </xs:annotation>
                                     </xs:attribute>
                                     <xs:attribute name="partitionLossPolicy" type="partitionLossPolicy">
@@ -1380,7 +1380,7 @@
                 </xs:element>
                 <xs:element name="persistentStoreConfiguration" minOccurs="0">
                     <xs:annotation>
-                        <xs:documentation>Persistent store configuration.</xs:documentation>
+                        <xs:documentation>Persistent store configuration. Obsolete, use DataStorageConfiguration.</xs:documentation>
                     </xs:annotation>
                     <xs:complexType>
                         <xs:attribute name="persistentStorePath" type="xs:string">
@@ -1490,6 +1490,273 @@
                         </xs:attribute>
                     </xs:complexType>
                 </xs:element>
+                <xs:element name="dataStorageConfiguration" minOccurs="0">
+                    <xs:annotation>
+                        <xs:documentation>Data storage configuration.</xs:documentation>
+                    </xs:annotation>
+                    <xs:complexType>
+                        <xs:all>
+                            <xs:element name="defaultDataRegionConfiguration">
+                                <xs:annotation>
+                                    <xs:documentation>Default data region configuration.</xs:documentation>
+                                </xs:annotation>
+                                <xs:complexType>
+                                    <xs:attribute name="emptyPagesPoolSize" type="xs:int">
+                                        <xs:annotation>
+                                            <xs:documentation>Minimal number of empty pages to be present in reuse lists for this memory policy.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="evictionThreshold" type="xs:double">
+                                        <xs:annotation>
+                                            <xs:documentation>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 (defined by this policy) is occupied.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="name" type="xs:string" use="required">
+                                        <xs:annotation>
+                                            <xs:documentation>Data region name.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="pageEvictionMode" type="dataPageEvictionMode">
+                                        <xs:annotation>
+                                            <xs:documentation>Page eviction mode.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="initialSize" type="xs:long">
+                                        <xs:annotation>
+                                            <xs:documentation>Initial data region size.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="maxSize" type="xs:long">
+                                        <xs:annotation>
+                                            <xs:documentation>Maximum data region size.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="swapPath" type="xs:string">
+                                        <xs:annotation>
+                                            <xs:documentation>Path to the directory for memory-mapped files.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="metricsEnabled" type="xs:boolean">
+                                        <xs:annotation>
+                                            <xs:documentation>Enable memory metrics.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="metricsSubIntervalCount" type="xs:int">
+                                        <xs:annotation>
+                                            <xs:documentation>Number of sub intervals to split RateTimeInterval into.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="metricsRateTimeInterval" type="xs:string">
+                                        <xs:annotation>
+                                            <xs:documentation>Rate time interval for AllocationRate and EvictionRate monitoring.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                    <xs:attribute name="persistenceEnabled" type="xs:boolean">
+                                        <xs:annotation>
+                                            <xs:documentation>Enable disk persistence for this region.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
+                                </xs:complexType>
+                            </xs:element>
+                            <xs:element name="dataRegionConfigurations">
+                                <xs:annotation>
+                                    <xs:documentation>Data region configurations.</xs:documentation>
+                                </xs:annotation>
+                                <xs:complexType>
+                                    <xs:sequence>
+                                        <xs:element minOccurs="1" maxOccurs="unbounded" name="dataRegionConfiguration">
+                                            <xs:annotation>
+                                                <xs:documentation>Data region configuration.</xs:documentation>
+                                            </xs:annotation>
+                                            <xs:complexType>
+                                                <xs:attribute name="emptyPagesPoolSize" type="xs:int">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Minimal number of empty pages to be present in reuse lists for this memory policy.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="evictionThreshold" type="xs:double">
+                                                    <xs:annotation>
+                                                        <xs:documentation>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 (defined by this policy) is occupied.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="name" type="xs:string" use="required">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Data region name.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="pageEvictionMode" type="dataPageEvictionMode">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Page eviction mode.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="initialSize" type="xs:long">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Initial data region size.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="maxSize" type="xs:long">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Maximum data region size.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="swapPath" type="xs:string">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Path to the directory for memory-mapped files.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="metricsEnabled" type="xs:boolean">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Enable memory metrics.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="metricsSubIntervalCount" type="xs:int">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Number of sub intervals to split RateTimeInterval into.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="metricsRateTimeInterval" type="xs:string">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Rate time interval for AllocationRate and EvictionRate monitoring.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="persistenceEnabled" type="xs:boolean">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Enable disk persistence for this region.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                            </xs:complexType>                                            
+                                        </xs:element>
+                                    </xs:sequence>
+                                </xs:complexType>
+                            </xs:element>
+                        </xs:all>
+                        <xs:attribute name="storagePath" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Path where data and indexes will be persisted.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="checkpointFrequency" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Checkpointing frequency which is a minimal interval when the dirty pages will be written to the Persistent Store.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="checkpointPageBufferSize" type="xs:long">
+                            <xs:annotation>
+                                <xs:documentation>Size of the checkpointing page buffer.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="checkpointThreads" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Number of threads for checkpointing.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="lockWaitTime" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Persistent manager file lock wait time.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walHistorySize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Number of checkpoints to store in WAL (Write Ahead Log) history.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walSegments" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Number of WAL (Write Ahead Log) segments to work with.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walSegmentSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the WAL (Write Ahead Log) segment.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walPath" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Path to the directory where WAL (Write Ahead Log) is stored.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walArchivePath" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Path to the directory where WAL (Write Ahead Log) archive is stored.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walMode" type="walMode">
+                            <xs:annotation>
+                                <xs:documentation>WAL (Write Ahead Log) mode.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walThreadLocalBufferSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the TLB (Thread-Local Buffer), in bytes.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walFlushFrequency" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>WAL (Write Ahead Log) flush frequency.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walFsyncDelayNanos" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>WAL (Write Ahead Log) fsync (disk sync) delay, in nanoseconds.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="walRecordIteratorBufferSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the WAL (Write Ahead Log) record iterator buffer, in bytes.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="alwaysWriteFullPages" type="xs:boolean">
+                            <xs:annotation>
+                                <xs:documentation>Whether full pages should always be written.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="metricsEnabled" type="xs:boolean">
+                            <xs:annotation>
+                                <xs:documentation>Enable persistent store metrics.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="metricsSubIntervalCount" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Number of sub intervals to split RateTimeInterval into.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="metricsRateTimeInterval" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Rate time interval.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="checkpointWriteOrder" type="checkpointWriteOrder">
+                            <xs:annotation>
+                                <xs:documentation>Checkpoint page write order on disk.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="writeThrottlingEnabled" type="xs:boolean">
+                            <xs:annotation>
+                                <xs:documentation>Threads that generate dirty pages too fast during ongoing checkpoint will be throttled.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="pageSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the memory page.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="systemRegionInitialSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Initial size of a memory region reserved for system needs.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="systemRegionMaxSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Maximum size of a memory region reserved for system needs.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="concurrencyLevel" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Number of concurrent segments in Ignite internal page mapping tables.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                    </xs:complexType>
+                </xs:element>
                 <xs:element name="pluginConfigurations" minOccurs="0">
                     <xs:annotation>
                         <xs:documentation>Plugin configurations.</xs:documentation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/MemoryMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/MemoryMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/MemoryMetrics.cs
index ae9f85c..9785909 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/MemoryMetrics.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/MemoryMetrics.cs
@@ -24,7 +24,9 @@ namespace Apache.Ignite.Core.Impl.Cache
     /// <summary>
     /// Memory metrics.
     /// </summary>
+#pragma warning disable 618
     internal class MemoryMetrics : IMemoryMetrics
+#pragma warning restore 618
     {
         /// <summary>
         /// Initializes a new instance of the <see cref="MemoryMetrics"/> class.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
index b32d331..cc25a6e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -145,6 +145,15 @@ namespace Apache.Ignite.Core.Impl.Cluster
         /** */
         private const int OpGetServices = 34;
 
+        /** */
+        private const int OpDataRegionMetrics = 35;
+
+        /** */
+        private const int OpDataRegionMetricsByName = 36;
+
+        /** */
+        private const int OpDataStorageMetrics = 37;
+
         /** Initial Ignite instance. */
         private readonly IIgniteInternal _ignite;
         
@@ -597,6 +606,7 @@ namespace Apache.Ignite.Core.Impl.Cluster
         /// <summary>
         /// Gets the memory metrics.
         /// </summary>
+#pragma warning disable 618
         public ICollection<IMemoryMetrics> GetMemoryMetrics()
         {
             return DoInOp(OpMemoryMetrics, stream =>
@@ -624,6 +634,47 @@ namespace Apache.Ignite.Core.Impl.Cluster
             return DoOutInOp(OpMemoryMetricsByName, w => w.WriteString(memoryPolicyName),
                 stream => stream.ReadBool() ? new MemoryMetrics(Marshaller.StartUnmarshal(stream, false)) : null);
         }
+#pragma warning restore 618
+
+        /// <summary>
+        /// Gets the data region metrics.
+        /// </summary>
+        public ICollection<IDataRegionMetrics> GetDataRegionMetrics()
+        {
+            return DoInOp(OpDataRegionMetrics, stream =>
+            {
+                IBinaryRawReader reader = Marshaller.StartUnmarshal(stream, false);
+
+                var cnt = reader.ReadInt();
+
+                var res = new List<IDataRegionMetrics>(cnt);
+
+                for (int i = 0; i < cnt; i++)
+                {
+                    res.Add(new DataRegionMetrics(reader));
+                }
+
+                return res;
+            });
+        }
+
+        /// <summary>
+        /// Gets the data region metrics.
+        /// </summary>
+        public IDataRegionMetrics GetDataRegionMetrics(string memoryPolicyName)
+        {
+            return DoOutInOp(OpDataRegionMetricsByName, w => w.WriteString(memoryPolicyName),
+                stream => stream.ReadBool() ? new DataRegionMetrics(Marshaller.StartUnmarshal(stream, false)) : null);
+        }
+
+        /// <summary>
+        /// Gets the data storage metrics.
+        /// </summary>
+        public IDataStorageMetrics GetDataStorageMetrics()
+        {
+            return DoInOp(OpDataStorageMetrics, stream =>
+                new DataStorageMetrics(Marshaller.StartUnmarshal(stream, false)));
+        }
 
         /// <summary>
         /// Changes Ignite grid state to active or inactive.
@@ -647,11 +698,13 @@ namespace Apache.Ignite.Core.Impl.Cluster
         /// <summary>
         /// Gets the persistent store metrics.
         /// </summary>
+#pragma warning disable 618
         public IPersistentStoreMetrics GetPersistentStoreMetrics()
         {
             return DoInOp(OpGetPersistentStoreMetrics, stream =>
                 new PersistentStoreMetrics(Marshaller.StartUnmarshal(stream, false)));
         }
+#pragma warning restore 618
 
         /// <summary>
         /// Creates new Cluster Group from given native projection.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
index a2f7143..be1a7f1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
@@ -26,6 +26,7 @@ namespace Apache.Ignite.Core.Impl.Common
     using System.Linq;
     using System.Reflection;
     using System.Xml;
+    using System.Xml.Serialization;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Events;
@@ -90,7 +91,7 @@ namespace Apache.Ignite.Core.Impl.Common
                 if (!property.CanWrite && !IsKeyValuePair(property.DeclaringType))
                     return;
 
-                if (IsObsolete(property))
+                if (IsIgnored(property))
                     return;
             }
 
@@ -169,7 +170,7 @@ namespace Apache.Ignite.Core.Impl.Common
             }
 
             // Write attributes
-            foreach (var prop in props.Where(p => IsBasicType(p.PropertyType) && !IsObsolete(p)))
+            foreach (var prop in props.Where(p => IsBasicType(p.PropertyType) && !IsIgnored(p)))
             {
                 var converter = GetConverter(prop, prop.PropertyType);
                 var stringValue = converter.ConvertToInvariantString(prop.GetValue(obj, null));
@@ -557,13 +558,13 @@ namespace Apache.Ignite.Core.Impl.Common
         }
 
         /// <summary>
-        /// Determines whether the specified property is obsolete.
+        /// Determines whether the specified property is marked with XmlIgnore.
         /// </summary>
-        private static bool IsObsolete(PropertyInfo property)
+        private static bool IsIgnored(PropertyInfo property)
         {
             Debug.Assert(property != null);
 
-            return property.GetCustomAttributes(typeof(ObsoleteAttribute), true).Any();
+            return property.GetCustomAttributes(typeof(XmlIgnoreAttribute), true).Any();
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataRegionMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataRegionMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataRegionMetrics.cs
new file mode 100644
index 0000000..7b174a6
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataRegionMetrics.cs
@@ -0,0 +1,61 @@
+/*
+ * 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
+{
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+
+    /// <summary>
+    /// Data region metrics.
+    /// </summary>
+    internal class DataRegionMetrics : IDataRegionMetrics
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataRegionMetrics"/> class.
+        /// </summary>
+        public DataRegionMetrics(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            Name = reader.ReadString();
+            TotalAllocatedPages = reader.ReadLong();
+            AllocationRate = reader.ReadFloat();
+            EvictionRate = reader.ReadFloat();
+            LargeEntriesPagesPercentage = reader.ReadFloat();
+            PageFillFactor = reader.ReadFloat();
+        }
+
+        /** <inheritdoc /> */
+        public string Name { get; private set; }
+
+        /** <inheritdoc /> */
+        public long TotalAllocatedPages { get; private set; }
+
+        /** <inheritdoc /> */
+        public float AllocationRate { get; private set; }
+        
+        /** <inheritdoc /> */
+        public float EvictionRate { get; private set; }
+
+        /** <inheritdoc /> */
+        public float LargeEntriesPagesPercentage { get; private set; }
+
+        /** <inheritdoc /> */
+        public float PageFillFactor { get; private set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStorageMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStorageMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStorageMetrics.cs
new file mode 100644
index 0000000..58b3b37
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStorageMetrics.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.Impl
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary;
+
+    /// <summary>
+    /// Data storage metrics.
+    /// </summary>
+    internal class DataStorageMetrics : IDataStorageMetrics
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataStorageMetrics"/> class.
+        /// </summary>
+        public DataStorageMetrics(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            WalLoggingRate = reader.ReadFloat();
+            WalWritingRate = reader.ReadFloat();
+            WalArchiveSegments = reader.ReadInt();
+            WalFsyncTimeAverage = reader.ReadFloat();
+            LastCheckpointDuration = reader.ReadLongAsTimespan();
+            LastCheckpointLockWaitDuration = reader.ReadLongAsTimespan();
+            LastCheckpointMarkDuration = reader.ReadLongAsTimespan();
+            LastCheckpointPagesWriteDuration = reader.ReadLongAsTimespan();
+            LastCheckpointFsyncDuration = reader.ReadLongAsTimespan();
+            LastCheckpointTotalPagesNumber = reader.ReadLong();
+            LastCheckpointDataPagesNumber = reader.ReadLong();
+            LastCheckpointCopiedOnWritePagesNumber = reader.ReadLong();
+        }
+
+        /** <inheritdoc /> */
+        public float WalLoggingRate { get; private set; }
+
+        /** <inheritdoc /> */
+        public float WalWritingRate { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WalArchiveSegments { get; private set; }
+
+        /** <inheritdoc /> */
+        public float WalFsyncTimeAverage { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointLockWaitDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointMarkDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointPagesWriteDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointFsyncDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public long LastCheckpointTotalPagesNumber { get; private set; }
+
+        /** <inheritdoc /> */
+        public long LastCheckpointDataPagesNumber { get; private set; }
+
+        /** <inheritdoc /> */
+        public long LastCheckpointCopiedOnWritePagesNumber { get; private set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index 1b42462..78b7c74 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -752,6 +752,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+#pragma warning disable 618
         public ICollection<IMemoryMetrics> GetMemoryMetrics()
         {
             return _prj.GetMemoryMetrics();
@@ -764,6 +765,7 @@ namespace Apache.Ignite.Core.Impl
 
             return _prj.GetMemoryMetrics(memoryPolicyName);
         }
+#pragma warning restore 618
 
         /** <inheritdoc /> */
         public void SetActive(bool isActive)
@@ -778,10 +780,30 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+#pragma warning disable 618
         public IPersistentStoreMetrics GetPersistentStoreMetrics()
         {
             return _prj.GetPersistentStoreMetrics();
         }
+#pragma warning restore 618
+
+        /** <inheritdoc /> */
+        public ICollection<IDataRegionMetrics> GetDataRegionMetrics()
+        {
+            return _prj.GetDataRegionMetrics();
+        }
+
+        /** <inheritdoc /> */
+        public IDataRegionMetrics GetDataRegionMetrics(string memoryPolicyName)
+        {
+            return _prj.GetDataRegionMetrics(memoryPolicyName);
+        }
+
+        /** <inheritdoc /> */
+        public IDataStorageMetrics GetDataStorageMetrics()
+        {
+            return _prj.GetDataStorageMetrics();
+        }
 
         /// <summary>
         /// Gets or creates near cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs
index 85a4fdf..7eeabb2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs
@@ -26,7 +26,9 @@ namespace Apache.Ignite.Core.Impl.PersistentStore
     /// <summary>
     /// Persistent store metrics.
     /// </summary>
+#pragma warning disable 618
     internal class PersistentStoreMetrics : IPersistentStoreMetrics
+#pragma warning restore 618
     {
         /// <summary>
         /// Initializes a new instance of the <see cref="PersistentStoreMetrics"/> class.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/CheckpointWriteOrder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/CheckpointWriteOrder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/CheckpointWriteOrder.cs
index ba1153d..7128796 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/CheckpointWriteOrder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/CheckpointWriteOrder.cs
@@ -17,9 +17,12 @@
 
 namespace Apache.Ignite.Core.PersistentStore
 {
+    using System;
+
     /// <summary>
     /// Defines checkpoint pages order on disk.
     /// </summary>
+    [Obsolete("Use Apache.Ignite.Core.Data.CheckpointWriteOrder")]
     public enum CheckpointWriteOrder
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs
index e7e8481..989dbd8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs
@@ -21,7 +21,9 @@ namespace Apache.Ignite.Core.PersistentStore
 
     /// <summary>
     /// Persistent store metrics.
+    /// Obsolete, see <see cref="IDataStorageMetrics"/>.
     /// </summary>
+    [Obsolete("Use IDataStorageMetrics")]
     public interface IPersistentStoreMetrics
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
index 7a2248a..e211126 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
@@ -22,11 +22,15 @@ namespace Apache.Ignite.Core.PersistentStore
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl.Binary;
 
     /// <summary>
     /// Configures Apache Ignite persistent store.
+    /// <para />
+    /// Obsolete, use <see cref="DataStorageConfiguration"/>.
     /// </summary>
+    [Obsolete("Use DataStorageConfiguration.")]
     public class PersistentStoreConfiguration
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/WalMode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/WalMode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/WalMode.cs
index 44d13b8..c937b78 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/WalMode.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/WalMode.cs
@@ -17,9 +17,12 @@
 
 namespace Apache.Ignite.Core.PersistentStore
 {
+    using System;
+
     /// <summary>
     /// Write Ahead Log mode.
     /// </summary>
+    [Obsolete("Use Apache.Ignite.Core.Data.WalMode")]
     public enum WalMode
     {
         /// <summary>


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

Posted by pt...@apache.org.
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)]


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

Posted by pt...@apache.org.
IGNITE-6515 .NET: Enable persistence on per-cache basis

This closes #2891


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

Branch: refs/heads/master
Commit: ab08be83ccc6fe1997ef80c25c3b48d45f410b56
Parents: ec9a945
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Oct 20 14:38:11 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Oct 20 14:38:11 2017 +0300

----------------------------------------------------------------------
 .../platform/cluster/PlatformClusterGroup.java  |  90 ++++
 .../utils/PlatformConfigurationUtils.java       | 174 ++++++-
 .../Apache.Ignite.Core.Tests.csproj             |   5 +-
 .../Cache/CacheConfigurationTest.cs             |   6 +
 .../Cache/DataRegionMetricsTest.cs              | 153 ++++++
 .../Cache/DataStorageMetricsTest.cs             | 107 +++++
 .../Cache/MemoryMetricsTest.cs                  |   1 +
 .../Cache/PersistenceTest.cs                    | 235 ++++++++++
 .../Cache/PersistentStoreTest.cs                | 189 --------
 .../Cache/PersistentStoreTestObsolete.cs        | 190 ++++++++
 .../Config/full-config.xml                      |  18 +
 .../Config/spring-test.xml                      |  18 +-
 .../IgniteConfigurationSerializerTest.cs        | 135 +++++-
 .../IgniteConfigurationTest.cs                  | 227 +++++----
 .../Apache.Ignite.Core.csproj                   |  10 +
 .../Cache/Configuration/CacheConfiguration.cs   |  21 +-
 .../Cache/Configuration/DataPageEvictionMode.cs |   3 +
 .../Cache/Configuration/MemoryConfiguration.cs  |   5 +
 .../Configuration/MemoryPolicyConfiguration.cs  |   3 +
 .../Apache.Ignite.Core/Cache/IMemoryMetrics.cs  |   4 +
 .../Configuration/CheckpointWriteOrder.cs       |  37 ++
 .../Configuration/DataPageEvictionMode.cs       |  59 +++
 .../Configuration/DataRegionConfiguration.cs    | 213 +++++++++
 .../Configuration/DataStorageConfiguration.cs   | 466 +++++++++++++++++++
 .../Apache.Ignite.Core/Configuration/WalMode.cs |  45 ++
 .../Apache.Ignite.Core/IDataRegionMetrics.cs    |  55 +++
 .../Apache.Ignite.Core/IDataStorageMetrics.cs   |  87 ++++
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |  33 ++
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  40 +-
 .../IgniteConfigurationSection.xsd              | 273 ++++++++++-
 .../Impl/Cache/MemoryMetrics.cs                 |   2 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |  53 +++
 .../Common/IgniteConfigurationXmlSerializer.cs  |  11 +-
 .../Impl/DataRegionMetrics.cs                   |  61 +++
 .../Impl/DataStorageMetrics.cs                  |  87 ++++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  22 +
 .../PersistentStore/PersistentStoreMetrics.cs   |   2 +
 .../PersistentStore/CheckpointWriteOrder.cs     |   3 +
 .../PersistentStore/IPersistentStoreMetrics.cs  |   2 +
 .../PersistentStoreConfiguration.cs             |   4 +
 .../PersistentStore/WalMode.cs                  |   3 +
 41 files changed, 2801 insertions(+), 351 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index 7c1c03e..ef382d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -21,6 +21,8 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.UUID;
 
+import org.apache.ignite.DataRegionMetrics;
+import org.apache.ignite.DataStorageMetrics;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCluster;
@@ -142,6 +144,14 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
     private static final int OP_GET_SERVICES = 34;
 
+    /** */
+    private static final int OP_DATA_REGION_METRICS = 35;
+
+    /** */
+    private static final int OP_DATA_REGION_METRICS_BY_NAME = 36;
+
+    /** */
+    private static final int OP_DATA_STORAGE_METRICS = 37;
 
     /** Projection. */
     private final ClusterGroupEx prj;
@@ -187,6 +197,26 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
+            case OP_DATA_STORAGE_METRICS: {
+                DataStorageMetrics metrics = prj.ignite().dataStorageMetrics();
+
+                writeDataStorageMetrics(writer, metrics);
+
+                break;
+            }
+
+            case OP_DATA_REGION_METRICS: {
+                Collection<DataRegionMetrics> metrics = prj.ignite().dataRegionMetrics();
+
+                writer.writeInt(metrics.size());
+
+                for (DataRegionMetrics m : metrics) {
+                    writeDataRegionMetrics(writer, m);
+                }
+
+                break;
+            }
+
             default:
                 super.processOutStream(type, writer);
         }
@@ -287,6 +317,22 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
+            case OP_DATA_REGION_METRICS_BY_NAME: {
+                String name = reader.readString();
+
+                DataRegionMetrics metrics = platformCtx.kernalContext().grid().dataRegionMetrics(name);
+
+                if (metrics != null) {
+                    writer.writeBoolean(true);
+                    writeDataRegionMetrics(writer, metrics);
+                }
+                else {
+                    writer.writeBoolean(false);
+                }
+
+                break;
+            }
+
             default:
                 super.processInStreamOutStream(type, reader, writer);
         }
@@ -479,6 +525,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
      * @param writer Writer.
      * @param metrics Metrics.
      */
+    @SuppressWarnings("deprecation")
     private static void writeMemoryMetrics(BinaryRawWriter writer, MemoryMetrics metrics) {
         assert writer != null;
         assert metrics != null;
@@ -492,11 +539,30 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /**
+     * Writes the data region metrics.
+     *
+     * @param writer Writer.
+     * @param metrics Metrics.
+     */
+    private static void writeDataRegionMetrics(BinaryRawWriter writer, DataRegionMetrics metrics) {
+        assert writer != null;
+        assert metrics != null;
+
+        writer.writeString(metrics.getName());
+        writer.writeLong(metrics.getTotalAllocatedPages());
+        writer.writeFloat(metrics.getAllocationRate());
+        writer.writeFloat(metrics.getEvictionRate());
+        writer.writeFloat(metrics.getLargeEntriesPagesPercentage());
+        writer.writeFloat(metrics.getPagesFillFactor());
+    }
+
+    /**
      * Writes persistent store metrics.
      *
      * @param writer Writer.
      * @param metrics Metrics
      */
+    @SuppressWarnings("deprecation")
     private void writePersistentStoreMetrics(BinaryRawWriter writer, PersistenceMetrics metrics) {
         assert writer != null;
         assert metrics != null;
@@ -514,4 +580,28 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
         writer.writeLong(metrics.getLastCheckpointDataPagesNumber());
         writer.writeLong(metrics.getLastCheckpointCopiedOnWritePagesNumber());
     }
+
+    /**
+     * Writes data storage metrics.
+     *
+     * @param writer Writer.
+     * @param metrics Metrics
+     */
+    private void writeDataStorageMetrics(BinaryRawWriter writer, DataStorageMetrics metrics) {
+        assert writer != null;
+        assert metrics != null;
+
+        writer.writeFloat(metrics.getWalLoggingRate());
+        writer.writeFloat(metrics.getWalWritingRate());
+        writer.writeInt(metrics.getWalArchiveSegments());
+        writer.writeFloat(metrics.getWalFsyncTimeAverage());
+        writer.writeLong(metrics.getLastCheckpointDuration());
+        writer.writeLong(metrics.getLastCheckpointLockWaitDuration());
+        writer.writeLong(metrics.getLastCheckpointMarkDuration());
+        writer.writeLong(metrics.getLastCheckpointPagesWriteDuration());
+        writer.writeLong(metrics.getLastCheckpointFsyncDuration());
+        writer.writeLong(metrics.getLastCheckpointTotalPagesNumber());
+        writer.writeLong(metrics.getLastCheckpointDataPagesNumber());
+        writer.writeLong(metrics.getLastCheckpointCopiedOnWritePagesNumber());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 24f4438..9711e62 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
@@ -57,6 +57,8 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CheckpointWriteOrder;
 import org.apache.ignite.configuration.ClientConnectorConfiguration;
 import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
@@ -184,10 +186,11 @@ public class PlatformConfigurationUtils {
         ccfg.setWriteThrough(in.readBoolean());
         ccfg.setStatisticsEnabled(in.readBoolean());
 
-        String memoryPolicyName = in.readString();
+        String dataRegionName = in.readString();
 
-        if (memoryPolicyName != null)
-            ccfg.setMemoryPolicyName(memoryPolicyName);
+        if (dataRegionName != null)
+            //noinspection deprecation
+            ccfg.setMemoryPolicyName(dataRegionName);
 
         ccfg.setPartitionLossPolicy(PartitionLossPolicy.fromOrdinal((byte)in.readInt()));
         ccfg.setGroupName(in.readString());
@@ -717,6 +720,9 @@ public class PlatformConfigurationUtils {
         if (in.readBoolean())
             cfg.setPersistentStoreConfiguration(readPersistentStoreConfiguration(in));
 
+        if (in.readBoolean())
+            cfg.setDataStorageConfiguration(readDataStorageConfiguration(in));
+
         readPluginConfiguration(cfg, in);
 
         readLocalEventListeners(cfg, in);
@@ -873,6 +879,7 @@ public class PlatformConfigurationUtils {
         writer.writeBoolean(ccfg.isReadThrough());
         writer.writeBoolean(ccfg.isWriteThrough());
         writer.writeBoolean(ccfg.isStatisticsEnabled());
+        //noinspection deprecation
         writer.writeString(ccfg.getMemoryPolicyName());
         writer.writeInt(ccfg.getPartitionLossPolicy().ordinal());
         writer.writeString(ccfg.getGroupName());
@@ -1194,6 +1201,8 @@ public class PlatformConfigurationUtils {
 
         writePersistentStoreConfiguration(w, cfg.getPersistentStoreConfiguration());
 
+        writeDataStorageConfiguration(w, cfg.getDataStorageConfiguration());
+
         w.writeString(cfg.getIgniteHome());
 
         w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit());
@@ -1403,6 +1412,7 @@ public class PlatformConfigurationUtils {
      * @param in Reader
      * @return Config.
      */
+    @SuppressWarnings("deprecation")
     private static MemoryConfiguration readMemoryConfiguration(BinaryRawReader in) {
         MemoryConfiguration res = new MemoryConfiguration();
 
@@ -1446,6 +1456,7 @@ public class PlatformConfigurationUtils {
      * @param w Writer.
      * @param cfg Config.
      */
+    @SuppressWarnings("deprecation")
     private static void writeMemoryConfiguration(BinaryRawWriter w, MemoryConfiguration cfg) {
         if (cfg == null) {
             w.writeBoolean(false);
@@ -1575,6 +1586,7 @@ public class PlatformConfigurationUtils {
      * @param in Reader.
      * @return Config.
      */
+    @SuppressWarnings("deprecation")
     private static PersistentStoreConfiguration readPersistentStoreConfiguration(BinaryRawReader in) {
         return new PersistentStoreConfiguration()
                 .setPersistentStorePath(in.readString())
@@ -1601,10 +1613,64 @@ public class PlatformConfigurationUtils {
     }
 
     /**
+     * Reads the data storage configuration.
+     *
+     * @param in Reader.
+     * @return Config.
+     */
+    private static DataStorageConfiguration readDataStorageConfiguration(BinaryRawReader in) {
+        DataStorageConfiguration res = new DataStorageConfiguration()
+                .setStoragePath(in.readString())
+                .setCheckpointFrequency(in.readLong())
+                .setCheckpointPageBufferSize(in.readLong())
+                .setCheckpointThreads(in.readInt())
+                .setLockWaitTime((int) in.readLong())
+                .setWalHistorySize(in.readInt())
+                .setWalSegments(in.readInt())
+                .setWalSegmentSize(in.readInt())
+                .setWalPath(in.readString())
+                .setWalArchivePath(in.readString())
+                .setWalMode(WALMode.fromOrdinal(in.readInt()))
+                .setWalThreadLocalBufferSize(in.readInt())
+                .setWalFlushFrequency((int) in.readLong())
+                .setWalFsyncDelayNanos(in.readLong())
+                .setWalRecordIteratorBufferSize(in.readInt())
+                .setAlwaysWriteFullPages(in.readBoolean())
+                .setMetricsEnabled(in.readBoolean())
+                .setMetricsSubIntervalCount(in.readInt())
+                .setMetricsRateTimeInterval(in.readLong())
+                .setCheckpointWriteOrder(CheckpointWriteOrder.fromOrdinal(in.readInt()))
+                .setWriteThrottlingEnabled(in.readBoolean())
+                .setSystemRegionInitialSize(in.readLong())
+                .setSystemRegionMaxSize(in.readLong())
+                .setPageSize(in.readInt())
+                .setConcurrencyLevel(in.readInt());
+
+        int cnt = in.readInt();
+
+        if (cnt > 0) {
+            DataRegionConfiguration[] regs = new DataRegionConfiguration[cnt];
+
+            for (int i = 0; i < cnt; i++) {
+                regs[i] = readDataRegionConfiguration(in);
+            }
+
+            res.setDataRegionConfigurations(regs);
+        }
+
+        if (in.readBoolean()) {
+            res.setDefaultDataRegionConfiguration(readDataRegionConfiguration(in));
+        }
+
+        return res;
+    }
+
+    /**
      * Writes the persistent store configuration.
      *
      * @param w Writer.
      */
+    @SuppressWarnings("deprecation")
     private static void writePersistentStoreConfiguration(BinaryRawWriter w, PersistentStoreConfiguration cfg) {
         assert w != null;
 
@@ -1639,6 +1705,108 @@ public class PlatformConfigurationUtils {
     }
 
     /**
+     * Writes the data storage configuration.
+     *
+     * @param w Writer.
+     */
+    private static void writeDataStorageConfiguration(BinaryRawWriter w, DataStorageConfiguration cfg) {
+        assert w != null;
+
+        if (cfg != null) {
+            w.writeBoolean(true);
+
+            w.writeString(cfg.getStoragePath());
+            w.writeLong(cfg.getCheckpointFrequency());
+            w.writeLong(cfg.getCheckpointPageBufferSize());
+            w.writeInt(cfg.getCheckpointThreads());
+            w.writeLong(cfg.getLockWaitTime());
+            w.writeInt(cfg.getWalHistorySize());
+            w.writeInt(cfg.getWalSegments());
+            w.writeInt(cfg.getWalSegmentSize());
+            w.writeString(cfg.getWalPath());
+            w.writeString(cfg.getWalArchivePath());
+            w.writeInt(cfg.getWalMode().ordinal());
+            w.writeInt(cfg.getWalThreadLocalBufferSize());
+            w.writeLong(cfg.getWalFlushFrequency());
+            w.writeLong(cfg.getWalFsyncDelayNanos());
+            w.writeInt(cfg.getWalRecordIteratorBufferSize());
+            w.writeBoolean(cfg.isAlwaysWriteFullPages());
+            w.writeBoolean(cfg.isMetricsEnabled());
+            w.writeInt(cfg.getMetricsSubIntervalCount());
+            w.writeLong(cfg.getMetricsRateTimeInterval());
+            w.writeInt(cfg.getCheckpointWriteOrder().ordinal());
+            w.writeBoolean(cfg.isWriteThrottlingEnabled());
+            w.writeLong(cfg.getSystemRegionInitialSize());
+            w.writeLong(cfg.getSystemRegionMaxSize());
+            w.writeInt(cfg.getPageSize());
+            w.writeInt(cfg.getConcurrencyLevel());
+
+            if (cfg.getDataRegionConfigurations() != null) {
+                w.writeInt(cfg.getDataRegionConfigurations().length);
+
+                for (DataRegionConfiguration d : cfg.getDataRegionConfigurations()) {
+                    writeDataRegionConfiguration(w, d);
+                }
+            } else {
+                w.writeInt(0);
+            }
+
+            if (cfg.getDefaultDataRegionConfiguration() != null) {
+                w.writeBoolean(true);
+                writeDataRegionConfiguration(w, cfg.getDefaultDataRegionConfiguration());
+            } else {
+                w.writeBoolean(false);
+            }
+        } else {
+            w.writeBoolean(false);
+        }
+    }
+
+    /**
+     * Writes the data region configuration.
+     *
+     * @param w Writer.
+     */
+    private static void writeDataRegionConfiguration(BinaryRawWriter w, DataRegionConfiguration cfg) {
+        assert w != null;
+        assert cfg != null;
+
+        w.writeString(cfg.getName());
+        w.writeBoolean(cfg.isPersistenceEnabled());
+        w.writeLong(cfg.getInitialSize());
+        w.writeLong(cfg.getMaxSize());
+        w.writeString(cfg.getSwapPath());
+        w.writeInt(cfg.getPageEvictionMode().ordinal());
+        w.writeDouble(cfg.getEvictionThreshold());
+        w.writeInt(cfg.getEmptyPagesPoolSize());
+        w.writeBoolean(cfg.isMetricsEnabled());
+        w.writeInt(cfg.getMetricsSubIntervalCount());
+        w.writeLong(cfg.getMetricsRateTimeInterval());
+    }
+
+    /**
+     * Reads the data region configuration.
+     *
+     * @param r Reader.
+     */
+    private static DataRegionConfiguration readDataRegionConfiguration(BinaryRawReader r) {
+        assert r != null;
+
+        return new DataRegionConfiguration()
+                .setName(r.readString())
+                .setPersistenceEnabled(r.readBoolean())
+                .setInitialSize(r.readLong())
+                .setMaxSize(r.readLong())
+                .setSwapPath(r.readString())
+                .setPageEvictionMode(DataPageEvictionMode.fromOrdinal(r.readInt()))
+                .setEvictionThreshold(r.readDouble())
+                .setEmptyPagesPoolSize(r.readInt())
+                .setMetricsEnabled(r.readBoolean())
+                .setMetricsSubIntervalCount(r.readInt())
+                .setMetricsRateTimeInterval(r.readLong());
+    }
+
+    /**
      * Reads the plugin configuration.
      *
      * @param cfg Ignite configuration to update.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 7ec75af..1d17757 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
@@ -79,7 +79,10 @@
     <Compile Include="Binary\BinarySelfTestSimpleName.cs" />
     <Compile Include="Binary\EnumsTestOnline.cs" />
     <Compile Include="Binary\Serializable\GenericCollectionsTest.cs" />
-    <Compile Include="Cache\PersistentStoreTest.cs" />
+    <Compile Include="Cache\DataRegionMetricsTest.cs" />
+    <Compile Include="Cache\DataStorageMetricsTest.cs" />
+    <Compile Include="Cache\PersistenceTest.cs" />
+    <Compile Include="Cache\PersistentStoreTestObsolete.cs" />
     <Compile Include="Cache\Query\Linq\CacheLinqTest.CompiledQuery.cs" />
     <Compile Include="Cache\Query\Linq\CacheLinqTest.DateTime.cs" />
     <Compile Include="Cache\Query\Linq\CacheLinqTest.Aggregates.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index ddf669d..4f13172 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -71,6 +71,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                 },
                 IgniteInstanceName = CacheName,
                 BinaryConfiguration = new BinaryConfiguration(typeof(Entity)),
+#pragma warning disable 618
                 MemoryConfiguration = new MemoryConfiguration
                 {
                     MemoryPolicies = new[]
@@ -83,6 +84,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                         }
                     }
                 },
+#pragma warning restore 618
                 SpringConfigUrl = "Config\\cache-default.xml"
             };
 
@@ -297,7 +299,9 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(x.WriteBehindFlushFrequency, y.WriteBehindFlushFrequency);
             Assert.AreEqual(x.WriteBehindFlushSize, y.WriteBehindFlushSize);
             Assert.AreEqual(x.EnableStatistics, y.EnableStatistics);
+#pragma warning disable 618
             Assert.AreEqual(x.MemoryPolicyName, y.MemoryPolicyName);
+#pragma warning restore 618
             Assert.AreEqual(x.PartitionLossPolicy, y.PartitionLossPolicy);
             Assert.AreEqual(x.WriteBehindCoalescing, y.WriteBehindCoalescing);
             Assert.AreEqual(x.GroupName, y.GroupName);
@@ -626,7 +630,9 @@ namespace Apache.Ignite.Core.Tests.Cache
                 },
                 ExpiryPolicyFactory = new ExpiryFactory(),
                 EnableStatistics = true,
+#pragma warning disable 618
                 MemoryPolicyName = "myMemPolicy",
+#pragma warning restore 618
                 PartitionLossPolicy = PartitionLossPolicy.ReadOnlySafe,
                 PluginConfigurations = new[] { new MyPluginConfiguration() },
                 SqlIndexMaxInlineSize = 10000

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataRegionMetricsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataRegionMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataRegionMetricsTest.cs
new file mode 100644
index 0000000..dd1cf53
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataRegionMetricsTest.cs
@@ -0,0 +1,153 @@
+/*
+ * 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.Linq;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Configuration;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Data region metrics test.
+    /// </summary>
+    public class DataRegionMetricsTest
+    {
+        /** */
+        private const string RegionWithMetrics = "regWithMetrics";
+
+        /** */
+        private const string RegionNoMetrics = "regNoMetrics";
+
+        /// <summary>
+        /// Tests the memory metrics.
+        /// </summary>
+        [Test]
+        public void TestMemoryMetrics()
+        {
+            var ignite = StartIgniteWithTwoDataRegions();
+
+            // Verify metrics.
+            var metrics = ignite.GetDataRegionMetrics().OrderBy(x => x.Name).ToArray();
+            Assert.AreEqual(3, metrics.Length);  // two defined plus system.
+
+            var emptyMetrics = metrics[0];
+            Assert.AreEqual(RegionNoMetrics, emptyMetrics.Name);
+            AssertMetricsAreEmpty(emptyMetrics);
+
+            var memMetrics = metrics[1];
+            Assert.AreEqual(RegionWithMetrics, memMetrics.Name);
+            Assert.Greater(memMetrics.AllocationRate, 0);
+            Assert.AreEqual(0, memMetrics.EvictionRate);
+            Assert.AreEqual(0, memMetrics.LargeEntriesPagesPercentage);
+            Assert.Greater(memMetrics.PageFillFactor, 0);
+            Assert.Greater(memMetrics.TotalAllocatedPages, 1000);
+
+            var sysMetrics = metrics[2];
+            Assert.AreEqual("sysMemPlc", sysMetrics.Name);
+            AssertMetricsAreEmpty(sysMetrics);
+
+            // Metrics by name.
+            emptyMetrics = ignite.GetDataRegionMetrics(RegionNoMetrics);
+            Assert.AreEqual(RegionNoMetrics, emptyMetrics.Name);
+            AssertMetricsAreEmpty(emptyMetrics);
+
+            memMetrics = ignite.GetDataRegionMetrics(RegionWithMetrics);
+            Assert.AreEqual(RegionWithMetrics, memMetrics.Name);
+            Assert.Greater(memMetrics.AllocationRate, 0);
+            Assert.AreEqual(0, memMetrics.EvictionRate);
+            Assert.AreEqual(0, memMetrics.LargeEntriesPagesPercentage);
+            Assert.Greater(memMetrics.PageFillFactor, 0);
+            Assert.Greater(memMetrics.TotalAllocatedPages, 1000);
+
+            sysMetrics = ignite.GetDataRegionMetrics("sysMemPlc");
+            Assert.AreEqual("sysMemPlc", sysMetrics.Name);
+            AssertMetricsAreEmpty(sysMetrics);
+
+            // Invalid name.
+            Assert.IsNull(ignite.GetDataRegionMetrics("boo"));
+        }
+
+        /// <summary>
+        /// Asserts that metrics are empty.
+        /// </summary>
+        private static void AssertMetricsAreEmpty(IDataRegionMetrics metrics)
+        {
+            Assert.AreEqual(0, metrics.AllocationRate);
+            Assert.AreEqual(0, metrics.EvictionRate);
+            Assert.AreEqual(0, metrics.LargeEntriesPagesPercentage);
+            Assert.AreEqual(0, metrics.PageFillFactor);
+            Assert.AreEqual(0, metrics.TotalAllocatedPages);
+        }
+
+        /// <summary>
+        /// Starts the ignite with two policies.
+        /// </summary>
+        private static IIgnite StartIgniteWithTwoDataRegions()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                DataStorageConfiguration = new DataStorageConfiguration()
+                {
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration
+                    {
+                        Name = RegionWithMetrics,
+                        MetricsEnabled = true
+                    },
+                    DataRegionConfigurations = new[]
+                    {
+                        new DataRegionConfiguration
+                        {
+                            Name = RegionNoMetrics,
+                            MetricsEnabled = false
+                        }
+                    }
+                }
+            };
+
+            var ignite = Ignition.Start(cfg);
+
+            // Create caches and do some things with them.
+            var cacheNoMetrics = ignite.CreateCache<int, int>(new CacheConfiguration("cacheNoMetrics")
+            {
+                DataRegionName = RegionNoMetrics
+            });
+
+            cacheNoMetrics.Put(1, 1);
+            cacheNoMetrics.Get(1);
+
+            var cacheWithMetrics = ignite.CreateCache<int, int>(new CacheConfiguration("cacheWithMetrics")
+            {
+                DataRegionName = RegionWithMetrics
+            });
+
+            cacheWithMetrics.Put(1, 1);
+            cacheWithMetrics.Get(1);
+
+            return ignite;
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataStorageMetricsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataStorageMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataStorageMetricsTest.cs
new file mode 100644
index 0000000..b24c20b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/DataStorageMetricsTest.cs
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache
+{
+    using System;
+    using System.IO;
+    using System.Linq;
+    using Apache.Ignite.Core.Configuration;
+    using Apache.Ignite.Core.Impl;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests <see cref="IDataStorageMetrics"/>.
+    /// </summary>
+    public class DataStorageMetricsTest
+    {
+        /** Temp dir for WAL. */
+        private readonly string _tempDir = IgniteUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Tests the data storage metrics.
+        /// </summary>
+        [Test]
+        public void TestDataStorageMetrics()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                DataStorageConfiguration = new DataStorageConfiguration
+                {
+                    CheckpointFrequency = TimeSpan.FromSeconds(1),
+                    MetricsEnabled = true,
+                    WalMode = WalMode.LogOnly,
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration
+                    {
+                        PersistenceEnabled = true,
+                        Name = "foobar"
+                    }
+                },
+                WorkDirectory = _tempDir
+            };
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                ignite.SetActive(true);
+
+                var cache = ignite.CreateCache<int, object>("c");
+
+                cache.PutAll(Enumerable.Range(1, 10)
+                    .ToDictionary(x => x, x => (object) new {Name = x.ToString(), Id = x}));
+
+                // Wait for checkpoint and metrics update and verify.
+                IDataStorageMetrics metrics = null;
+
+                Assert.IsTrue(TestUtils.WaitForCondition(() =>
+                {
+                    // ReSharper disable once AccessToDisposedClosure
+                    metrics = ignite.GetDataStorageMetrics();
+
+                    return metrics.LastCheckpointTotalPagesNumber > 0;
+                }, 10000));
+
+                Assert.IsNotNull(metrics);
+
+                Assert.AreEqual(0, metrics.WalArchiveSegments);
+                Assert.AreEqual(0, metrics.WalFsyncTimeAverage);
+
+                Assert.AreEqual(77, metrics.LastCheckpointTotalPagesNumber);
+                Assert.AreEqual(10, metrics.LastCheckpointDataPagesNumber);
+                Assert.AreEqual(0, metrics.LastCheckpointCopiedOnWritePagesNumber);
+                Assert.AreEqual(TimeSpan.Zero, metrics.LastCheckpointLockWaitDuration);
+
+                Assert.Greater(metrics.LastCheckpointPagesWriteDuration, TimeSpan.Zero);
+                Assert.Greater(metrics.LastCheckpointMarkDuration, TimeSpan.Zero);
+                Assert.Greater(metrics.LastCheckpointDuration, TimeSpan.Zero);
+                Assert.Greater(metrics.LastCheckpointFsyncDuration, TimeSpan.Zero);
+
+                Assert.Greater(metrics.LastCheckpointDuration, metrics.LastCheckpointMarkDuration);
+                Assert.Greater(metrics.LastCheckpointDuration, metrics.LastCheckpointPagesWriteDuration);
+                Assert.Greater(metrics.LastCheckpointDuration, metrics.LastCheckpointFsyncDuration);
+            }
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Directory.Delete(_tempDir, true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/MemoryMetricsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/MemoryMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/MemoryMetricsTest.cs
index 1aad823..7ccee94 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/MemoryMetricsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/MemoryMetricsTest.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#pragma warning disable 618
 namespace Apache.Ignite.Core.Tests.Cache
 {
     using System.Linq;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs
new file mode 100644
index 0000000..b2e4d05
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistenceTest.cs
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache
+{
+    using System;
+    using System.IO;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Configuration;
+    using Apache.Ignite.Core.Impl;
+    using NUnit.Framework;
+    using DataPageEvictionMode = Apache.Ignite.Core.Configuration.DataPageEvictionMode;
+
+    /// <summary>
+    /// Tests disk persistence.
+    /// </summary>
+    public class PersistenceTest
+    {
+        /** Temp dir for WAL. */
+        private readonly string _tempDir = IgniteUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+
+            if (Directory.Exists(_tempDir))
+            {
+                Directory.Delete(_tempDir, true);
+            }
+        }
+
+        /// <summary>
+        /// Tests that cache data survives node restart.
+        /// </summary>
+        [Test]
+        public void TestCacheDataSurvivesNodeRestart()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                DataStorageConfiguration = new DataStorageConfiguration
+                {
+                    StoragePath = Path.Combine(_tempDir, "Store"),
+                    WalPath = Path.Combine(_tempDir, "WalStore"),
+                    WalArchivePath = Path.Combine(_tempDir, "WalArchive"),
+                    MetricsEnabled = true,
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration
+                    {
+                        PageEvictionMode = DataPageEvictionMode.Disabled,
+                        Name = DataStorageConfiguration.DefaultDataRegionName,
+                        PersistenceEnabled = true
+                    },
+                    DataRegionConfigurations = new[]
+                    {
+                        new DataRegionConfiguration
+                        {
+                            Name = "volatileRegion",
+                            PersistenceEnabled = false
+                        } 
+                    }
+                }
+            };
+
+            const string cacheName = "persistentCache";
+            const string volatileCacheName = "volatileCache";
+
+            // Start Ignite, put data, stop.
+            using (var ignite = Ignition.Start(cfg))
+            {
+                ignite.SetActive(true);
+
+                // Create cache with default region (persistence enabled), add data.
+                var cache = ignite.CreateCache<int, int>(cacheName);
+                cache[1] = 1;
+
+                // Check some metrics.
+                CheckDataStorageMetrics(ignite);
+
+                // Create cache with non-persistent region.
+                var volatileCache = ignite.CreateCache<int, int>(new CacheConfiguration
+                {
+                    Name = volatileCacheName,
+                    DataRegionName = "volatileRegion"
+                });
+                volatileCache[2] = 2;
+            }
+
+            // Verify directories.
+            Assert.IsTrue(Directory.Exists(cfg.DataStorageConfiguration.StoragePath));
+            Assert.IsTrue(Directory.Exists(cfg.DataStorageConfiguration.WalPath));
+            Assert.IsTrue(Directory.Exists(cfg.DataStorageConfiguration.WalArchivePath));
+
+            // Start Ignite, verify data survival.
+            using (var ignite = Ignition.Start(cfg))
+            {
+                ignite.SetActive(true);
+
+                // Persistent cache already exists and contains data.
+                var cache = ignite.GetCache<int, int>(cacheName);
+                Assert.AreEqual(1, cache[1]);
+
+                // Non-persistent cache does not exist.
+                var ex = Assert.Throws<ArgumentException>(() => ignite.GetCache<int, int>(volatileCacheName));
+                Assert.AreEqual("Cache doesn't exist: volatileCache", ex.Message);
+            }
+
+            // Delete store directory.
+            Directory.Delete(_tempDir, true);
+
+            // Start Ignite, verify data loss.
+            using (var ignite = Ignition.Start(cfg))
+            {
+                ignite.SetActive(true);
+
+                Assert.IsFalse(ignite.GetCacheNames().Contains(cacheName));
+            }
+        }
+
+        /// <summary>
+        /// Checks the data storage metrics.
+        /// </summary>
+        private static void CheckDataStorageMetrics(IIgnite ignite)
+        {
+            // Check metrics.
+            var metrics = ignite.GetDataStorageMetrics();
+            Assert.Greater(metrics.WalLoggingRate, 0);
+            Assert.Greater(metrics.WalWritingRate, 0);
+            Assert.Greater(metrics.WalFsyncTimeAverage, 0);
+        }
+
+        /// <summary>
+        /// Tests the grid activation with persistence (inactive by default).
+        /// </summary>
+        [Test]
+        public void TestGridActivationWithPersistence()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                DataStorageConfiguration = new DataStorageConfiguration
+                {
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration
+                    {
+                        PersistenceEnabled = true,
+                        Name = "foo"
+                    }
+                }
+            };
+
+            // Default config, inactive by default (IsActiveOnStart is ignored when persistence is enabled).
+            using (var ignite = Ignition.Start(cfg))
+            {
+                CheckIsActive(ignite, false);
+
+                ignite.SetActive(true);
+                CheckIsActive(ignite, true);
+
+                ignite.SetActive(false);
+                CheckIsActive(ignite, false);
+            }
+        }
+
+        /// <summary>
+        /// Tests the grid activation without persistence (active by default).
+        /// </summary>
+        [Test]
+        public void TestGridActivationNoPersistence()
+        {
+            var cfg = TestUtils.GetTestConfiguration();
+            Assert.IsTrue(cfg.IsActiveOnStart);
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                CheckIsActive(ignite, true);
+
+                ignite.SetActive(false);
+                CheckIsActive(ignite, false);
+
+                ignite.SetActive(true);
+                CheckIsActive(ignite, true);
+            }
+
+            cfg.IsActiveOnStart = false;
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                CheckIsActive(ignite, false);
+
+                ignite.SetActive(true);
+                CheckIsActive(ignite, true);
+
+                ignite.SetActive(false);
+                CheckIsActive(ignite, false);
+            }
+        }
+
+        /// <summary>
+        /// Checks active state.
+        /// </summary>
+        private static void CheckIsActive(IIgnite ignite, bool isActive)
+        {
+            Assert.AreEqual(isActive, ignite.IsActive());
+
+            if (isActive)
+            {
+                var cache = ignite.GetOrCreateCache<int, int>("default");
+                cache[1] = 1;
+                Assert.AreEqual(1, cache[1]);
+            }
+            else
+            {
+                var ex = Assert.Throws<IgniteException>(() => ignite.GetOrCreateCache<int, int>("default"));
+                Assert.AreEqual("Can not perform the operation because the cluster is inactive.",
+                    ex.Message.Substring(0, 62));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
deleted file mode 100644
index a592859..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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 Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl;
-    using Apache.Ignite.Core.PersistentStore;
-    using NUnit.Framework;
-
-    /// <summary>
-    /// Tests the persistent store.
-    /// </summary>
-    public class PersistentStoreTest
-    {
-        /** Temp dir for WAL. */
-        private readonly string _tempDir = IgniteUtils.GetTempDirectoryName();
-
-        /// <summary>
-        /// Tears down the test.
-        /// </summary>
-        [TearDown]
-        public void TearDown()
-        {
-            Ignition.StopAll(true);
-
-            if (Directory.Exists(_tempDir))
-            {
-                Directory.Delete(_tempDir, true);
-            }
-        }
-
-        /// <summary>
-        /// Tests that cache data survives node restart.
-        /// </summary>
-        [Test]
-        public void TestCacheDataSurvivesNodeRestart()
-        {
-            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
-            {
-                PersistentStoreConfiguration = new PersistentStoreConfiguration
-                {
-                    PersistentStorePath = Path.Combine(_tempDir, "Store"),
-                    WalStorePath = Path.Combine(_tempDir, "WalStore"),
-                    WalArchivePath = Path.Combine(_tempDir, "WalArchive"),
-                    MetricsEnabled = true
-                }
-            };
-
-            const string cacheName = "persistentCache";
-
-            // Start Ignite, put data, stop.
-            using (var ignite = Ignition.Start(cfg))
-            {
-                ignite.SetActive(true);
-
-                var cache = ignite.CreateCache<int, int>(cacheName);
-
-                cache[1] = 1;
-
-                // Check some metrics.
-                var metrics = ignite.GetPersistentStoreMetrics();
-                Assert.Greater(metrics.WalLoggingRate, 0);
-                Assert.Greater(metrics.WalWritingRate, 0);
-                Assert.Greater(metrics.WalFsyncTimeAverage, 0);
-            }
-
-            // Verify directories.
-            Assert.IsTrue(Directory.Exists(cfg.PersistentStoreConfiguration.PersistentStorePath));
-            Assert.IsTrue(Directory.Exists(cfg.PersistentStoreConfiguration.WalStorePath));
-            Assert.IsTrue(Directory.Exists(cfg.PersistentStoreConfiguration.WalArchivePath));
-
-            // Start Ignite, verify data survival.
-            using (var ignite = Ignition.Start(cfg))
-            {
-                ignite.SetActive(true);
-
-                var cache = ignite.GetCache<int, int>(cacheName);
-
-                Assert.AreEqual(1, cache[1]);
-            }
-
-            // Delete store directory.
-            Directory.Delete(_tempDir, true);
-
-            // Start Ignite, verify data loss.
-            using (var ignite = Ignition.Start(cfg))
-            {
-                ignite.SetActive(true);
-
-                Assert.IsFalse(ignite.GetCacheNames().Contains(cacheName));
-            }
-        }
-
-        /// <summary>
-        /// Tests the grid activation with persistence (inactive by default).
-        /// </summary>
-        [Test]
-        public void TestGridActivationWithPersistence()
-        {
-            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
-            {
-                PersistentStoreConfiguration = new PersistentStoreConfiguration()
-            };
-
-            // Default config, inactive by default (IsActiveOnStart is ignored when persistence is enabled).
-            using (var ignite = Ignition.Start(cfg))
-            {
-                CheckIsActive(ignite, false);
-
-                ignite.SetActive(true);
-                CheckIsActive(ignite, true);
-
-                ignite.SetActive(false);
-                CheckIsActive(ignite, false);
-            }
-        }
-
-        /// <summary>
-        /// Tests the grid activation without persistence (active by default).
-        /// </summary>
-        [Test]
-        public void TestGridActivationNoPersistence()
-        {
-            var cfg = TestUtils.GetTestConfiguration();
-            Assert.IsTrue(cfg.IsActiveOnStart);
-
-            using (var ignite = Ignition.Start(cfg))
-            {
-                CheckIsActive(ignite, true);
-
-                ignite.SetActive(false);
-                CheckIsActive(ignite, false);
-
-                ignite.SetActive(true);
-                CheckIsActive(ignite, true);
-            }
-
-            cfg.IsActiveOnStart = false;
-
-            using (var ignite = Ignition.Start(cfg))
-            {
-                CheckIsActive(ignite, false);
-
-                ignite.SetActive(true);
-                CheckIsActive(ignite, true);
-
-                ignite.SetActive(false);
-                CheckIsActive(ignite, false);
-            }
-        }
-
-        /// <summary>
-        /// Checks active state.
-        /// </summary>
-        private static void CheckIsActive(IIgnite ignite, bool isActive)
-        {
-            Assert.AreEqual(isActive, ignite.IsActive());
-
-            if (isActive)
-            {
-                var cache = ignite.GetOrCreateCache<int, int>("default");
-                cache[1] = 1;
-                Assert.AreEqual(1, cache[1]);
-            }
-            else
-            {
-                var ex = Assert.Throws<IgniteException>(() => ignite.GetOrCreateCache<int, int>("default"));
-                Assert.AreEqual("Can not perform the operation because the cluster is inactive.",
-                    ex.Message.Substring(0, 62));
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTestObsolete.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTestObsolete.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTestObsolete.cs
new file mode 100644
index 0000000..a6b9b3b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTestObsolete.cs
@@ -0,0 +1,190 @@
+/*
+ * 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.
+ */
+
+#pragma warning disable 618  // Obsolete.
+namespace Apache.Ignite.Core.Tests.Cache
+{
+    using System.IO;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.PersistentStore;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the persistent store. Uses the obsolete API. See <see cref="PersistenceTest"/> for the actual API.
+    /// </summary>
+    public class PersistentStoreTestObsolete
+    {
+        /** Temp dir for WAL. */
+        private readonly string _tempDir = IgniteUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+
+            if (Directory.Exists(_tempDir))
+            {
+                Directory.Delete(_tempDir, true);
+            }
+        }
+
+        /// <summary>
+        /// Tests that cache data survives node restart.
+        /// </summary>
+        [Test]
+        public void TestCacheDataSurvivesNodeRestart()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                PersistentStoreConfiguration = new PersistentStoreConfiguration
+                {
+                    PersistentStorePath = Path.Combine(_tempDir, "Store"),
+                    WalStorePath = Path.Combine(_tempDir, "WalStore"),
+                    WalArchivePath = Path.Combine(_tempDir, "WalArchive"),
+                    MetricsEnabled = true
+                }
+            };
+
+            const string cacheName = "persistentCache";
+
+            // Start Ignite, put data, stop.
+            using (var ignite = Ignition.Start(cfg))
+            {
+                ignite.SetActive(true);
+
+                var cache = ignite.CreateCache<int, int>(cacheName);
+
+                cache[1] = 1;
+
+                // Check some metrics.
+                var metrics = ignite.GetPersistentStoreMetrics();
+                Assert.Greater(metrics.WalLoggingRate, 0);
+                Assert.Greater(metrics.WalWritingRate, 0);
+                Assert.Greater(metrics.WalFsyncTimeAverage, 0);
+            }
+
+            // Verify directories.
+            Assert.IsTrue(Directory.Exists(cfg.PersistentStoreConfiguration.PersistentStorePath));
+            Assert.IsTrue(Directory.Exists(cfg.PersistentStoreConfiguration.WalStorePath));
+            Assert.IsTrue(Directory.Exists(cfg.PersistentStoreConfiguration.WalArchivePath));
+
+            // Start Ignite, verify data survival.
+            using (var ignite = Ignition.Start(cfg))
+            {
+                ignite.SetActive(true);
+
+                var cache = ignite.GetCache<int, int>(cacheName);
+
+                Assert.AreEqual(1, cache[1]);
+            }
+
+            // Delete store directory.
+            Directory.Delete(_tempDir, true);
+
+            // Start Ignite, verify data loss.
+            using (var ignite = Ignition.Start(cfg))
+            {
+                ignite.SetActive(true);
+
+                Assert.IsFalse(ignite.GetCacheNames().Contains(cacheName));
+            }
+        }
+
+        /// <summary>
+        /// Tests the grid activation with persistence (inactive by default).
+        /// </summary>
+        [Test]
+        public void TestGridActivationWithPersistence()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                PersistentStoreConfiguration = new PersistentStoreConfiguration()
+            };
+
+            // Default config, inactive by default (IsActiveOnStart is ignored when persistence is enabled).
+            using (var ignite = Ignition.Start(cfg))
+            {
+                CheckIsActive(ignite, false);
+
+                ignite.SetActive(true);
+                CheckIsActive(ignite, true);
+
+                ignite.SetActive(false);
+                CheckIsActive(ignite, false);
+            }
+        }
+
+        /// <summary>
+        /// Tests the grid activation without persistence (active by default).
+        /// </summary>
+        [Test]
+        public void TestGridActivationNoPersistence()
+        {
+            var cfg = TestUtils.GetTestConfiguration();
+            Assert.IsTrue(cfg.IsActiveOnStart);
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                CheckIsActive(ignite, true);
+
+                ignite.SetActive(false);
+                CheckIsActive(ignite, false);
+
+                ignite.SetActive(true);
+                CheckIsActive(ignite, true);
+            }
+
+            cfg.IsActiveOnStart = false;
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                CheckIsActive(ignite, false);
+
+                ignite.SetActive(true);
+                CheckIsActive(ignite, true);
+
+                ignite.SetActive(false);
+                CheckIsActive(ignite, false);
+            }
+        }
+
+        /// <summary>
+        /// Checks active state.
+        /// </summary>
+        private static void CheckIsActive(IIgnite ignite, bool isActive)
+        {
+            Assert.AreEqual(isActive, ignite.IsActive());
+
+            if (isActive)
+            {
+                var cache = ignite.GetOrCreateCache<int, int>("default");
+                cache[1] = 1;
+                Assert.AreEqual(1, cache[1]);
+            }
+            else
+            {
+                var ex = Assert.Throws<IgniteException>(() => ignite.GetOrCreateCache<int, int>("default"));
+                Assert.AreEqual("Can not perform the operation because the cluster is inactive.",
+                    ex.Message.Substring(0, 62));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
index 229d42e..1e17752 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
@@ -131,4 +131,22 @@
             <listener type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyEventListener' />
         </localEventListener>
     </localEventListeners>
+    <dataStorageConfiguration alwaysWriteFullPages="false" checkpointFrequency="00:00:01" checkpointPageBufferSize="2"
+                              checkpointThreads="3" concurrencyLevel="4" lockWaitTime="00:00:05" metricsEnabled="true"
+                              pageSize="6" storagePath="cde" metricsRateTimeInterval="00:00:07"
+                              metricsSubIntervalCount="8" systemRegionInitialSize="9" systemRegionMaxSize="10" 
+                              walThreadLocalBufferSize="11"
+                              walArchivePath="abc" walFlushFrequency="00:00:12" walFsyncDelayNanos="13" walHistorySize="14"
+                              walMode="Background" walRecordIteratorBufferSize="15" walSegments="16" walSegmentSize="17"
+                              walPath="wal-store" writeThrottlingEnabled="true">
+        <dataRegionConfigurations>
+            <dataRegionConfiguration emptyPagesPoolSize="1" evictionThreshold="2" initialSize="3" metricsEnabled="true"
+                                     maxSize="4" name="reg2" pageEvictionMode="RandomLru" metricsRateTimeInterval="00:00:01"
+                                     metricsSubIntervalCount="5" swapPath="swap" />
+        </dataRegionConfigurations>
+        <defaultDataRegionConfiguration emptyPagesPoolSize="2" evictionThreshold="3" initialSize="4"
+                                        maxSize="5" metricsEnabled="false" name="reg1" pageEvictionMode="Disabled"
+                                        metricsRateTimeInterval="00:00:03" metricsSubIntervalCount="6"
+                                        swapPath="swap2" />
+    </dataStorageConfiguration>
 </igniteConfiguration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
index 31fa3b3..145fb01 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
@@ -43,22 +43,8 @@
             </bean>
         </property>
 
-        <property name="memoryConfiguration">
-            <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="defaultMemoryPolicyName" value="dfltPlc"/>
-
-                <property name="memoryPolicies">
-                    <list>
-                        <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
-                            <property name="name" value="dfltPlc"/>
-                        </bean>
-                    </list>
-                </property>
-            </bean>
-        </property>
-
-        <property name="persistentStoreConfiguration">
-            <bean class="org.apache.ignite.configuration.PersistentStoreConfiguration"/>
+        <property name="dataStorageConfiguration">
+            <bean class="org.apache.ignite.configuration.DataStorageConfiguration"/>
         </property>
     </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/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 edecccc..72c73e4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -17,6 +17,7 @@
 
 // ReSharper disable UnusedAutoPropertyAccessor.Global
 // ReSharper disable MemberCanBePrivate.Global
+#pragma warning disable 618
 namespace Apache.Ignite.Core.Tests
 {
     using System;
@@ -55,6 +56,9 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Transactions;
     using Apache.Ignite.NLog;
     using NUnit.Framework;
+    using CheckpointWriteOrder = Apache.Ignite.Core.PersistentStore.CheckpointWriteOrder;
+    using DataPageEvictionMode = Apache.Ignite.Core.Cache.Configuration.DataPageEvictionMode;
+    using WalMode = Apache.Ignite.Core.PersistentStore.WalMode;
 
     /// <summary>
     /// Tests <see cref="IgniteConfiguration"/> serialization.
@@ -213,7 +217,6 @@ namespace Apache.Ignite.Core.Tests
 
             Assert.AreEqual(PeerAssemblyLoadingMode.CurrentAppDomain, cfg.PeerAssemblyLoadingMode);
 
-#pragma warning disable 618  // Obsolete
             var sql = cfg.SqlConnectorConfiguration;
             Assert.IsNotNull(sql);
             Assert.AreEqual("bar", sql.Host);
@@ -224,7 +227,6 @@ namespace Apache.Ignite.Core.Tests
             Assert.IsTrue(sql.TcpNoDelay);
             Assert.AreEqual(14, sql.MaxOpenCursorsPerConnection);
             Assert.AreEqual(15, sql.ThreadPoolSize);
-#pragma warning restore 618
 
             var client = cfg.ClientConnectorConfiguration;
             Assert.IsNotNull(client);
@@ -269,6 +271,56 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual("Apache.Ignite.Core.Tests.EventsTestLocalListeners+Listener`1" +
                             "[Apache.Ignite.Core.Events.CacheRebalancingEvent]",
                 rebalListener.Listener.GetType().ToString());
+
+            var ds = cfg.DataStorageConfiguration;
+            Assert.IsFalse(ds.AlwaysWriteFullPages);
+            Assert.AreEqual(TimeSpan.FromSeconds(1), ds.CheckpointFrequency);
+            Assert.AreEqual(2, ds.CheckpointPageBufferSize);
+            Assert.AreEqual(3, ds.CheckpointThreads);
+            Assert.AreEqual(4, ds.ConcurrencyLevel);
+            Assert.AreEqual(TimeSpan.FromSeconds(5), ds.LockWaitTime);
+            Assert.IsTrue(ds.MetricsEnabled);
+            Assert.AreEqual(6, ds.PageSize);
+            Assert.AreEqual("cde", ds.StoragePath);
+            Assert.AreEqual(TimeSpan.FromSeconds(7), ds.MetricsRateTimeInterval);
+            Assert.AreEqual(8, ds.MetricsSubIntervalCount);
+            Assert.AreEqual(9, ds.SystemRegionInitialSize);
+            Assert.AreEqual(10, ds.SystemRegionMaxSize);
+            Assert.AreEqual(11, ds.WalThreadLocalBufferSize);
+            Assert.AreEqual("abc", ds.WalArchivePath);
+            Assert.AreEqual(TimeSpan.FromSeconds(12), ds.WalFlushFrequency);
+            Assert.AreEqual(13, ds.WalFsyncDelayNanos);
+            Assert.AreEqual(14, ds.WalHistorySize);
+            Assert.AreEqual(Core.Configuration.WalMode.Background, ds.WalMode);
+            Assert.AreEqual(15, ds.WalRecordIteratorBufferSize);
+            Assert.AreEqual(16, ds.WalSegments);
+            Assert.AreEqual(17, ds.WalSegmentSize);
+            Assert.AreEqual("wal-store", ds.WalPath);
+            Assert.IsTrue(ds.WriteThrottlingEnabled);
+
+            var dr = ds.DataRegionConfigurations.Single();
+            Assert.AreEqual(1, dr.EmptyPagesPoolSize);
+            Assert.AreEqual(2, dr.EvictionThreshold);
+            Assert.AreEqual(3, dr.InitialSize);
+            Assert.AreEqual(4, dr.MaxSize);
+            Assert.AreEqual("reg2", dr.Name);
+            Assert.AreEqual(Core.Configuration.DataPageEvictionMode.RandomLru, dr.PageEvictionMode);
+            Assert.AreEqual(TimeSpan.FromSeconds(1), dr.MetricsRateTimeInterval);
+            Assert.AreEqual(5, dr.MetricsSubIntervalCount);
+            Assert.AreEqual("swap", dr.SwapPath);
+            Assert.IsTrue(dr.MetricsEnabled);
+
+            dr = ds.DefaultDataRegionConfiguration;
+            Assert.AreEqual(2, dr.EmptyPagesPoolSize);
+            Assert.AreEqual(3, dr.EvictionThreshold);
+            Assert.AreEqual(4, dr.InitialSize);
+            Assert.AreEqual(5, dr.MaxSize);
+            Assert.AreEqual("reg1", dr.Name);
+            Assert.AreEqual(Core.Configuration.DataPageEvictionMode.Disabled, dr.PageEvictionMode);
+            Assert.AreEqual(TimeSpan.FromSeconds(3), dr.MetricsRateTimeInterval);
+            Assert.AreEqual(6, dr.MetricsSubIntervalCount);
+            Assert.AreEqual("swap2", dr.SwapPath);
+            Assert.IsFalse(dr.MetricsEnabled);
         }
 
         /// <summary>
@@ -574,7 +626,7 @@ namespace Apache.Ignite.Core.Tests
                             Serializer = new BinaryReflectiveSerializer()
                         }
                     },
-                    Types = new[] {typeof (string).FullName},
+                    Types = new[] {typeof(string).FullName},
                     IdMapper = new IdMapper(),
                     KeepDeserialized = true,
                     NameMapper = new NameMapper(),
@@ -601,7 +653,7 @@ namespace Apache.Ignite.Core.Tests
                             {
                                 Fields = new[]
                                 {
-                                    new QueryField("field", typeof (int))
+                                    new QueryField("field", typeof(int))
                                     {
                                         IsKeyField = true,
                                         NotNull = true
@@ -619,8 +671,8 @@ namespace Apache.Ignite.Core.Tests
                                 {
                                     new QueryAlias("field.field", "fld")
                                 },
-                                KeyType = typeof (string),
-                                ValueType = typeof (long),
+                                KeyType = typeof(string),
+                                ValueType = typeof(long),
                                 TableName = "table-1",
                                 KeyFieldName = "k",
                                 ValueFieldName = "v"
@@ -645,12 +697,16 @@ namespace Apache.Ignite.Core.Tests
                             NearStartSize = 5,
                             EvictionPolicy = new FifoEvictionPolicy
                             {
-                                BatchSize = 19, MaxMemorySize = 1024, MaxSize = 555
+                                BatchSize = 19,
+                                MaxMemorySize = 1024,
+                                MaxSize = 555
                             }
                         },
                         EvictionPolicy = new LruEvictionPolicy
                         {
-                            BatchSize = 18, MaxMemorySize = 1023, MaxSize = 554
+                            BatchSize = 18,
+                            MaxMemorySize = 1023,
+                            MaxSize = 554
                         },
                         AffinityFunction = new RendezvousAffinityFunction
                         {
@@ -715,7 +771,7 @@ namespace Apache.Ignite.Core.Tests
                 WorkDirectory = @"c:\work",
                 IsDaemon = true,
                 UserAttributes = Enumerable.Range(1, 10).ToDictionary(x => x.ToString(),
-                    x => x%2 == 0 ? (object) x : new FooClass {Bar = x.ToString()}),
+                    x => x % 2 == 0 ? (object) x : new FooClass {Bar = x.ToString()}),
                 AtomicConfiguration = new AtomicConfiguration
                 {
                     CacheMode = CacheMode.Replicated,
@@ -755,7 +811,7 @@ namespace Apache.Ignite.Core.Tests
                 FailureDetectionTimeout = TimeSpan.FromMinutes(2),
                 ClientFailureDetectionTimeout = TimeSpan.FromMinutes(3),
                 LongQueryWarningTimeout = TimeSpan.FromDays(4),
-                PluginConfigurations = new[] {new TestIgnitePluginConfiguration() },
+                PluginConfigurations = new[] {new TestIgnitePluginConfiguration()},
                 EventStorageSpi = new MemoryEventStorageSpi
                 {
                     ExpirationTimeout = TimeSpan.FromMilliseconds(12345),
@@ -838,6 +894,65 @@ namespace Apache.Ignite.Core.Tests
                         EventTypes = new[] {1, 2},
                         Listener = new MyEventListener()
                     }
+                },
+                DataStorageConfiguration = new DataStorageConfiguration
+                {
+                    AlwaysWriteFullPages = true,
+                    CheckpointFrequency = TimeSpan.FromSeconds(25),
+                    CheckpointPageBufferSize = 28 * 1024 * 1024,
+                    CheckpointThreads = 2,
+                    LockWaitTime = TimeSpan.FromSeconds(5),
+                    StoragePath = Path.GetTempPath(),
+                    WalThreadLocalBufferSize = 64 * 1024,
+                    WalArchivePath = Path.GetTempPath(),
+                    WalFlushFrequency = TimeSpan.FromSeconds(3),
+                    WalFsyncDelayNanos = 3,
+                    WalHistorySize = 10,
+                    WalMode = Core.Configuration.WalMode.None,
+                    WalRecordIteratorBufferSize = 32 * 1024 * 1024,
+                    WalSegments = 6,
+                    WalSegmentSize = 5 * 1024 * 1024,
+                    WalPath = Path.GetTempPath(),
+                    MetricsEnabled = true,
+                    MetricsSubIntervalCount = 7,
+                    MetricsRateTimeInterval = TimeSpan.FromSeconds(9),
+                    CheckpointWriteOrder = Core.Configuration.CheckpointWriteOrder.Sequential,
+                    WriteThrottlingEnabled = true,
+                    SystemRegionInitialSize = 64 * 1024 * 1024,
+                    SystemRegionMaxSize = 128 * 1024 * 1024,
+                    ConcurrencyLevel = 1,
+                    PageSize = 5 * 1024,
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration
+                    {
+                        Name = "reg1",
+                        EmptyPagesPoolSize = 50,
+                        EvictionThreshold = 0.8,
+                        InitialSize = 100 * 1024 * 1024,
+                        MaxSize = 150 * 1024 * 1024,
+                        MetricsEnabled = true,
+                        PageEvictionMode = Core.Configuration.DataPageEvictionMode.RandomLru,
+                        PersistenceEnabled = false,
+                        MetricsRateTimeInterval = TimeSpan.FromMinutes(2),
+                        MetricsSubIntervalCount = 6,
+                        SwapPath = Path.GetTempPath()
+                    },
+                    DataRegionConfigurations = new[]
+                    {
+                        new DataRegionConfiguration
+                        {
+                            Name = "reg2",
+                            EmptyPagesPoolSize = 51,
+                            EvictionThreshold = 0.7,
+                            InitialSize = 101 * 1024 * 1024,
+                            MaxSize = 151 * 1024 * 1024,
+                            MetricsEnabled = false,
+                            PageEvictionMode = Core.Configuration.DataPageEvictionMode.RandomLru,
+                            PersistenceEnabled = false,
+                            MetricsRateTimeInterval = TimeSpan.FromMinutes(3),
+                            MetricsSubIntervalCount = 7,
+                            SwapPath = Path.GetTempPath()
+                        }
+                    }
                 }
             };
         }