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

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

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()
+                        }
+                    }
                 }
             };
         }