You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/10/25 09:40:51 UTC

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab08be83/modules/platforms/dotnet/Apache.Ignite.Core/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>