You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/27 16:55:37 UTC

[1/3] ignite git commit: IGNITE-5072 - Updated memory metrics to comply with other metrics

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 f5fe301e1 -> 11c23b628


http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 32c6675..0254c4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLongArray;
 import java.util.concurrent.locks.Lock;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
@@ -629,7 +630,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         Map<Long,Long> map = new HashMap<>();
 
-        int loops = reuseList == null ? 100_000 : 300_000;
+        int loops = reuseList == null ? 20_000 : 60_000;
 
         for (int i = 0 ; i < loops; i++) {
             final Long x = (long)BPlusTree.randomInt(CNT);
@@ -1232,7 +1233,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         final Map<Long,Long> map = new ConcurrentHashMap8<>();
 
-        final int loops = reuseList == null ? 100_000 : 200_000;
+        final int loops = reuseList == null ? 20_000 : 60_000;
 
         final GridStripedLock lock = new GridStripedLock(256);
 
@@ -1272,7 +1273,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
                             tree.invoke(x, null, new IgniteTree.InvokeClosure<Long>() {
                                 IgniteTree.OperationType opType;
 
-                                @Override public void call(@Nullable Long row) throws IgniteCheckedException {
+                                @Override public void call(@Nullable Long row) {
                                     opType = PUT;
 
                                     if (row != null)
@@ -1294,7 +1295,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
                             tree.invoke(x, null, new IgniteTree.InvokeClosure<Long>() {
                                 IgniteTree.OperationType opType;
 
-                                @Override public void call(@Nullable Long row) throws IgniteCheckedException {
+                                @Override public void call(@Nullable Long row) {
                                     if (row != null) {
                                         assertEquals(x, row);
                                         opType = REMOVE;
@@ -1685,8 +1686,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             Long row = PageUtils.getLong(pageAddr, offset(idx));
 
             checkNotRemoved(row);
@@ -1699,17 +1699,14 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
      * @return Page memory.
      */
     protected PageMemory createPageMemory() throws Exception {
-        long[] sizes = new long[CPUS];
-
-        for (int i = 0; i < sizes.length; i++)
-            sizes[i] = 1024 * MB / CPUS;
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
 
         PageMemory pageMem = new PageMemoryNoStoreImpl(log,
-            new UnsafeMemoryProvider(sizes),
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
-            null,
-            new MemoryMetricsImpl(null), true);
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg), true);
 
         pageMem.start();
 
@@ -1754,8 +1751,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             return PageUtils.getLong(pageAddr, offset(idx));
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 1cede9b..5f61bd6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -145,7 +146,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param pageSize Page size.
-     * @throws Exception
+     * @throws Exception If failed.
      */
     protected void checkInsertDeleteMultiThreaded(final int pageSize) throws Exception {
         final FreeList list = createFreeList(pageSize);
@@ -175,7 +176,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
             @Override public Object call() throws Exception {
                 Random rnd = ThreadLocalRandom.current();
 
-                for (int i = 0; i < 1_000_000; i++) {
+                for (int i = 0; i < 200_000; i++) {
                     boolean grow0 = grow.get();
 
                     if (grow0) {
@@ -313,18 +314,13 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
     /**
      * @return Page memory.
      */
-    protected PageMemory createPageMemory(int pageSize) throws Exception {
-        long[] sizes = new long[CPUS];
-
-        for (int i = 0; i < sizes.length; i++)
-            sizes[i] = 1024 * MB / CPUS;
-
+    protected PageMemory createPageMemory(int pageSize, MemoryPolicyConfiguration plcCfg) throws Exception {
         PageMemory pageMem = new PageMemoryNoStoreImpl(log,
-            new UnsafeMemoryProvider(sizes),
+            new UnsafeMemoryProvider(log),
             null,
             pageSize,
-            null,
-            new MemoryMetricsImpl(null),
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg),
             true);
 
         pageMem.start();
@@ -338,13 +334,15 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     protected FreeList createFreeList(int pageSize) throws Exception {
-        pageMem = createPageMemory(pageSize);
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
+
+        pageMem = createPageMemory(pageSize, plcCfg);
 
         long metaPageId = pageMem.allocatePage(1, 1, PageIdAllocator.FLAG_DATA);
 
-        MemoryMetricsImpl metrics = new MemoryMetricsImpl(null);
+        MemoryMetricsImpl metrics = new MemoryMetricsImpl(plcCfg);
 
-        MemoryPolicy memPlc = new MemoryPolicy(pageMem, null, metrics, new NoOpPageEvictionTracker());
+        MemoryPolicy memPlc = new MemoryPolicy(pageMem, plcCfg, metrics, new NoOpPageEvictionTracker());
 
         return new FreeListImpl(1, "freelist", metrics, memPlc, null, null, metaPageId, true);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
index a2732e8..3b3e1de 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
@@ -44,7 +44,7 @@ public class IgniteDbDynamicCacheSelfTest extends GridCommonAbstractTest {
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
         plc.setName("dfltPlc");
-        plc.setSize(200 * 1024 * 1024);
+        plc.setMaxSize(200 * 1024 * 1024);
 
         dbCfg.setDefaultMemoryPolicyName("dfltPlc");
         dbCfg.setMemoryPolicies(plc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java
index 5347a23..cb5700f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java
@@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.database;
 
 import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.MemoryMetrics;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -41,7 +42,9 @@ public class MemoryMetricsSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        memMetrics = new MemoryMetricsImpl(null);
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration();
+
+        memMetrics = new MemoryMetricsImpl(plcCfg);
 
         memMetrics.enableMetrics();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
index 61c8ad9..af0b849 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.database;
 
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
@@ -154,13 +155,17 @@ public class MetadataStorageSelfTest extends GridCommonAbstractTest {
      * @return Page memory instance.
      */
     protected PageMemory memory(boolean clean) throws Exception {
-        long[] sizes = new long[10];
-
-        for (int i = 0; i < sizes.length; i++)
-            sizes[i] = 1024 * 1024;
-
-        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath, clean, sizes);
-
-        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE, null, new MemoryMetricsImpl(null), true);
+        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath);
+
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(30 * 1024 * 1024);
+
+        return new PageMemoryNoStoreImpl(
+            log,
+            provider,
+            null,
+            PAGE_SIZE,
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg),
+            true);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
index fbe0872..456971a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
@@ -397,7 +397,7 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
                 String memPlcName = "igfsDataMemPlc";
 
                 cfg.setMemoryConfiguration(new MemoryConfiguration().setMemoryPolicies(
-                    new MemoryPolicyConfiguration().setSize(maxSize).setName(memPlcName)));
+                    new MemoryPolicyConfiguration().setMaxSize(maxSize).setName(memPlcName)));
 
                 FileSystemConfiguration igfsCfg = cfg.getFileSystemConfiguration()[0];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
index da013bd..b99424d 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
@@ -235,11 +235,6 @@ public class PlatformCacheWriteMetricsTask extends ComputeTaskAdapter<Long, Obje
         }
 
         /** {@inheritDoc} */
-        @Override public long getOffHeapMaxSize() {
-            return 28;
-        }
-
-        /** {@inheritDoc} */
         @Override public int getSize() {
             return 29;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedPartitionQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedPartitionQueryAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedPartitionQueryAbstractSelfTest.java
index 708fb1d..0a0afb4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedPartitionQueryAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedPartitionQueryAbstractSelfTest.java
@@ -45,7 +45,6 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.typedef.F;
@@ -137,9 +136,7 @@ public abstract class IgniteCacheDistributedPartitionQueryAbstractSelfTest exten
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        MemoryConfiguration memCfg = new MemoryConfiguration();
-        memCfg.setDefaultMemoryPolicyName("default");
-        memCfg.setMemoryPolicies(new MemoryPolicyConfiguration().setName("default").setSize(20 * 1024 * 1024));
+        MemoryConfiguration memCfg = new MemoryConfiguration().setDefaultMemoryPolicySize(20 * 1024 * 1024);
 
         cfg.setMemoryConfiguration(memCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
index 001f40b..943a5c8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
@@ -29,16 +29,15 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.typedef.CAX;
 import org.apache.ignite.internal.util.typedef.F;
@@ -91,9 +90,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
-        MemoryConfiguration memCfg = new MemoryConfiguration();
-        memCfg.setDefaultMemoryPolicyName("default");
-        memCfg.setMemoryPolicies(new MemoryPolicyConfiguration().setName("default").setSize(50 * 1024 * 1024));
+        MemoryConfiguration memCfg = new MemoryConfiguration().setDefaultMemoryPolicySize(50 * 1024 * 1024);
 
         c.setMemoryConfiguration(memCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
index 3db3050..fc765a4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
@@ -138,7 +138,11 @@ public abstract class DynamicIndexAbstractSelfTest extends AbstractSchemaSelfTes
 
         MemoryConfiguration memCfg = new MemoryConfiguration()
             .setDefaultMemoryPolicyName("default")
-            .setMemoryPolicies(new MemoryPolicyConfiguration().setName("default").setSize(32 * 1024 * 1024L)
+            .setMemoryPolicies(
+                new MemoryPolicyConfiguration()
+                    .setName("default")
+                    .setMaxSize(32 * 1024 * 1024L)
+                    .setInitialSize(32 * 1024 * 1024L)
         );
 
         cfg.setMemoryConfiguration(memCfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
index 0317672..c101d04 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.UUID;
 import junit.framework.TestCase;
 import org.apache.commons.io.Charsets;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -130,17 +131,16 @@ public class InlineIndexHelperTest extends TestCase {
 
     /** */
     public void testStringTruncate() throws Exception {
-        long[] sizes = new long[CPUS];
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
 
-        for (int i = 0; i < sizes.length; i++)
-            sizes[i] = 1024 * MB / CPUS;
+        JavaLogger log = new JavaLogger();
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
-            null,
-            new MemoryMetricsImpl(null),
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg),
             false);
 
         pageMem.start();
@@ -177,17 +177,16 @@ public class InlineIndexHelperTest extends TestCase {
 
     /** */
     public void testBytes() throws Exception {
-        long[] sizes = new long[CPUS];
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
 
-        for (int i = 0; i < sizes.length; i++)
-            sizes[i] = 1024 * MB / CPUS;
+        JavaLogger log = new JavaLogger();
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
-            null,
-            new MemoryMetricsImpl(null),
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg),
             false);
 
         pageMem.start();
@@ -293,17 +292,16 @@ public class InlineIndexHelperTest extends TestCase {
 
     /** */
     private void testPutGet(Value v1, Value v2, Value v3) throws Exception {
-        long[] sizes = new long[CPUS];
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
 
-        for (int i = 0; i < sizes.length; i++)
-            sizes[i] = 1024 * MB / CPUS;
+        JavaLogger log = new JavaLogger();
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
-            null,
-            new MemoryMetricsImpl(null),
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg),
             false);
 
         pageMem.start();

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/cpp/core-test/config/cache-identity-32.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-identity-32.xml b/modules/platforms/cpp/core-test/config/cache-identity-32.xml
index 4a8a68f..1d43678 100644
--- a/modules/platforms/cpp/core-test/config/cache-identity-32.xml
+++ b/modules/platforms/cpp/core-test/config/cache-identity-32.xml
@@ -33,14 +33,17 @@
     <bean parent="grid.cfg">
         <property name="memoryConfiguration">
             <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="systemCacheMemorySize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
+
                 <property name="defaultMemoryPolicyName" value="dfltPlc"/>
 
                 <property name="memoryPolicies">
                     <list>
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="dfltPlc"/>
-                            <property name="size" value="#{100 * 1024 * 1024}"/>
+                            <property name="maxSize" value="#{100 * 1024 * 1024}"/>
+                            <property name="initialSize" value="#{100 * 1024 * 1024}"/>
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/cpp/core-test/config/cache-query-32.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-query-32.xml b/modules/platforms/cpp/core-test/config/cache-query-32.xml
index 6927705..ddbd690 100644
--- a/modules/platforms/cpp/core-test/config/cache-query-32.xml
+++ b/modules/platforms/cpp/core-test/config/cache-query-32.xml
@@ -33,14 +33,16 @@
     <bean parent="grid.cfg">
         <property name="memoryConfiguration">
             <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="systemCacheMemorySize" value="41943040"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
                 <property name="defaultMemoryPolicyName" value="dfltPlc"/>
 
                 <property name="memoryPolicies">
                     <list>
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="dfltPlc"/>
-                            <property name="size" value="103833600"/>
+                            <property name="maxSize" value="103833600"/>
+                            <property name="initialSize" value="103833600"/>
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/cpp/core-test/config/cache-query-continuous-32.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-query-continuous-32.xml b/modules/platforms/cpp/core-test/config/cache-query-continuous-32.xml
index b5f9854..84a5c60 100644
--- a/modules/platforms/cpp/core-test/config/cache-query-continuous-32.xml
+++ b/modules/platforms/cpp/core-test/config/cache-query-continuous-32.xml
@@ -29,14 +29,16 @@
     <bean parent="grid.cfg">
         <property name="memoryConfiguration">
             <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="systemCacheMemorySize" value="41943040"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
                 <property name="defaultMemoryPolicyName" value="dfltPlc"/>
 
                 <property name="memoryPolicies">
                     <list>
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="dfltPlc"/>
-                            <property name="size" value="103833600"/>
+                            <property name="maxSize" value="103833600"/>
+                            <property name="initialSize" value="103833600"/>
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/cpp/core-test/config/cache-store-32.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-store-32.xml b/modules/platforms/cpp/core-test/config/cache-store-32.xml
index f2b6682..ed46d4e 100644
--- a/modules/platforms/cpp/core-test/config/cache-store-32.xml
+++ b/modules/platforms/cpp/core-test/config/cache-store-32.xml
@@ -33,14 +33,16 @@
     <bean parent="grid.cfg">
         <property name="memoryConfiguration">
             <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="systemCacheMemorySize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
                 <property name="defaultMemoryPolicyName" value="dfltPlc"/>
 
                 <property name="memoryPolicies">
                     <list>
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="dfltPlc"/>
-                            <property name="size" value="#{100 * 1024 * 1024}"/>
+                            <property name="maxSize" value="#{100 * 1024 * 1024}"/>
+                            <property name="initialSize" value="#{100 * 1024 * 1024}"/>
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/cpp/core-test/config/cache-test-32.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-test-32.xml b/modules/platforms/cpp/core-test/config/cache-test-32.xml
index 3535ae4..889f246 100644
--- a/modules/platforms/cpp/core-test/config/cache-test-32.xml
+++ b/modules/platforms/cpp/core-test/config/cache-test-32.xml
@@ -33,14 +33,16 @@
     <bean parent="grid.cfg">
         <property name="memoryConfiguration">
             <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="systemCacheMemorySize" value="41943040"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
                 <property name="defaultMemoryPolicyName" value="dfltPlc"/>
 
                 <property name="memoryPolicies">
                     <list>
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="dfltPlc"/>
-                            <property name="size" value="103833600"/>
+                            <property name="maxSize" value="103833600"/>
+                            <property name="initialSize" value="103833600"/>
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/cpp/odbc-test/config/queries-test-32.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-32.xml b/modules/platforms/cpp/odbc-test/config/queries-test-32.xml
index dd7cfb6..f7d9ff4 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test-32.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test-32.xml
@@ -30,14 +30,16 @@
     <bean parent="queries.cfg">
         <property name="memoryConfiguration">
             <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="systemCacheMemorySize" value="41943040"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
                 <property name="defaultMemoryPolicyName" value="dfltPlc"/>
 
                 <property name="memoryPolicies">
                     <list>
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="dfltPlc"/>
-                            <property name="size" value="103833600"/>
+                            <property name="maxSize" value="#{100 * 1024 * 1024}"/>
+                            <property name="initialSize" value="#{100 * 1024 * 1024}"/>
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/cpp/odbc-test/config/queries-test-noodbc-32.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc-32.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc-32.xml
index 8060107..01cae20 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc-32.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc-32.xml
@@ -30,14 +30,16 @@
     <bean parent="ignite.cfg">
         <property name="memoryConfiguration">
             <bean class="org.apache.ignite.configuration.MemoryConfiguration">
-                <property name="systemCacheMemorySize" value="41943040"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
                 <property name="defaultMemoryPolicyName" value="dfltPlc"/>
 
                 <property name="memoryPolicies">
                     <list>
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="dfltPlc"/>
-                            <property name="size" value="103833600"/>
+                            <property name="maxSize" value="#{100 * 1024 * 1024}"/>
+                            <property name="initialSize" value="#{100 * 1024 * 1024}"/>
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/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 9af103b..cf70970 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -75,7 +75,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                         new MemoryPolicyConfiguration
                         {
                             Name = "myMemPolicy",
-                            Size = 99 * 1024 * 1024
+                            InitialSize = 77 * 1024 * 1024,
+                            MaxSize = 99 * 1024 * 1024
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
index b409a5a..4b587a9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
@@ -165,7 +165,6 @@ namespace Apache.Ignite.Core.Tests.Cache
                 Assert.AreEqual(25, metrics.OffHeapPrimaryEntriesCount);
                 Assert.AreEqual(26, metrics.OffHeapBackupEntriesCount);
                 Assert.AreEqual(27, metrics.OffHeapAllocatedSize);
-                Assert.AreEqual(28, metrics.OffHeapMaxSize);
                 Assert.AreEqual(29, metrics.Size);
                 Assert.AreEqual(30, metrics.KeySize);
                 Assert.AreEqual(true, metrics.IsEmpty);

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/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 2bf7478..dd0669a 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
@@ -42,5 +42,19 @@
                 <property name="socketTimeout" value="300" />
             </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>
     </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/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 dfd0d09..bc0321e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -134,9 +134,9 @@ namespace Apache.Ignite.Core.Tests
                                 <iPluginConfiguration type='Apache.Ignite.Core.Tests.Plugin.TestIgnitePluginConfiguration, Apache.Ignite.Core.Tests' />
                             </pluginConfigurations>
                             <eventStorageSpi type='MemoryEventStorageSpi' expirationTimeout='00:00:23.45' maxEventCount='129' />
-                            <memoryConfiguration concurrencyLevel='3' defaultMemoryPolicyName='dfPlc' pageSize='45' systemCacheMemorySize='67'>
+                            <memoryConfiguration concurrencyLevel='3' defaultMemoryPolicyName='dfPlc' pageSize='45' systemCacheInitialSize='67' systemCacheMaxSize='68'>
                                 <memoryPolicies>
-                                    <memoryPolicyConfiguration emptyPagesPoolSize='1' evictionThreshold='0.2' name='dfPlc' pageEvictionMode='RandomLru' size='89' swapFilePath='abc' />
+                                    <memoryPolicyConfiguration emptyPagesPoolSize='1' evictionThreshold='0.2' name='dfPlc' pageEvictionMode='RandomLru' initialSize='89' maxSize='98' swapFilePath='abc' />
                                 </memoryPolicies>
                             </memoryConfiguration>
                         </igniteConfig>";
@@ -261,7 +261,8 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(3, memCfg.ConcurrencyLevel);
             Assert.AreEqual("dfPlc", memCfg.DefaultMemoryPolicyName);
             Assert.AreEqual(45, memCfg.PageSize);
-            Assert.AreEqual(67, memCfg.SystemCacheMemorySize);
+            Assert.AreEqual(67, memCfg.SystemCacheInitialSize);
+            Assert.AreEqual(68, memCfg.SystemCacheMaxSize);
 
             var memPlc = memCfg.MemoryPolicies.Single();
             Assert.AreEqual(1, memPlc.EmptyPagesPoolSize);
@@ -269,7 +270,8 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual("dfPlc", memPlc.Name);
             Assert.AreEqual(DataPageEvictionMode.RandomLru, memPlc.PageEvictionMode);
             Assert.AreEqual("abc", memPlc.SwapFilePath);
-            Assert.AreEqual(89, memPlc.Size);
+            Assert.AreEqual(89, memPlc.InitialSize);
+            Assert.AreEqual(98, memPlc.MaxSize);
         }
 
         /// <summary>
@@ -804,14 +806,16 @@ namespace Apache.Ignite.Core.Tests
                     ConcurrencyLevel = 3,
                     DefaultMemoryPolicyName = "somePolicy",
                     PageSize = 4,
-                    SystemCacheMemorySize = 5,
+                    SystemCacheInitialSize = 5,
+                    SystemCacheMaxSize = 6,
                     MemoryPolicies = new[]
                     {
                         new MemoryPolicyConfiguration
                         {
                             Name = "myDefaultPlc",
                             PageEvictionMode = DataPageEvictionMode.Random2Lru,
-                            Size = 345 * 1024 * 1024,
+                            InitialSize = 245 * 1024 * 1024,
+                            MaxSize = 345 * 1024 * 1024,
                             EvictionThreshold = 0.88,
                             EmptyPagesPoolSize = 77,
                             SwapFilePath = "myPath1"
@@ -820,7 +824,6 @@ namespace Apache.Ignite.Core.Tests
                         {
                             Name = "customPlc",
                             PageEvictionMode = DataPageEvictionMode.RandomLru,
-                            Size = 456 * 1024 * 1024,
                             EvictionThreshold = 0.77,
                             EmptyPagesPoolSize = 66,
                             SwapFilePath = "somePath2"

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 5f4a8ca..ebca7c4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -199,7 +199,8 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(memCfg.PageSize, resMemCfg.PageSize);
                 Assert.AreEqual(memCfg.ConcurrencyLevel, resMemCfg.ConcurrencyLevel);
                 Assert.AreEqual(memCfg.DefaultMemoryPolicyName, resMemCfg.DefaultMemoryPolicyName);
-                Assert.AreEqual(memCfg.SystemCacheMemorySize, resMemCfg.SystemCacheMemorySize);
+                Assert.AreEqual(memCfg.SystemCacheInitialSize, resMemCfg.SystemCacheInitialSize);
+                Assert.AreEqual(memCfg.SystemCacheMaxSize, resMemCfg.SystemCacheMaxSize);
                 Assert.IsNotNull(memCfg.MemoryPolicies);
                 Assert.IsNotNull(resMemCfg.MemoryPolicies);
                 Assert.AreEqual(2, memCfg.MemoryPolicies.Count);
@@ -211,7 +212,7 @@ namespace Apache.Ignite.Core.Tests
                     var resPlc = resMemCfg.MemoryPolicies.Skip(i).First();
 
                     Assert.AreEqual(plc.PageEvictionMode, resPlc.PageEvictionMode);
-                    Assert.AreEqual(plc.Size, resPlc.Size);
+                    Assert.AreEqual(plc.MaxSize, resPlc.MaxSize);
                     Assert.AreEqual(plc.EmptyPagesPoolSize, resPlc.EmptyPagesPoolSize);
                     Assert.AreEqual(plc.EvictionThreshold, resPlc.EvictionThreshold);
                     Assert.AreEqual(plc.Name, resPlc.Name);
@@ -245,6 +246,22 @@ namespace Apache.Ignite.Core.Tests
                 var disco = resCfg.DiscoverySpi as TcpDiscoverySpi;
                 Assert.IsNotNull(disco);
                 Assert.AreEqual(TimeSpan.FromMilliseconds(300), disco.SocketTimeout);
+
+                // Check memory configuration defaults.
+                var mem = resCfg.MemoryConfiguration;
+
+                Assert.IsNotNull(mem);
+                Assert.AreEqual("dfltPlc", mem.DefaultMemoryPolicyName);
+                Assert.AreEqual(MemoryConfiguration.DefaultPageSize, mem.PageSize);
+                Assert.AreEqual(MemoryConfiguration.DefaultSystemCacheInitialSize, mem.SystemCacheInitialSize);
+                Assert.AreEqual(MemoryConfiguration.DefaultSystemCacheMaxSize, mem.SystemCacheMaxSize);
+
+                var plc = mem.MemoryPolicies.Single();
+                Assert.AreEqual("dfltPlc", plc.Name);
+                Assert.AreEqual(MemoryPolicyConfiguration.DefaultEmptyPagesPoolSize, plc.EmptyPagesPoolSize);
+                Assert.AreEqual(MemoryPolicyConfiguration.DefaultEvictionThreshold, plc.EvictionThreshold);
+                Assert.AreEqual(MemoryPolicyConfiguration.DefaultInitialSize, plc.InitialSize);
+                Assert.AreEqual(MemoryPolicyConfiguration.DefaultMaxSize, plc.MaxSize);
             }
         }
 
@@ -446,7 +463,9 @@ namespace Apache.Ignite.Core.Tests
         {
             var props = obj.GetType().GetProperties();
 
-            foreach (var prop in props.Where(p => p.Name != "SelectorsCount" && p.Name != "ReadStripesNumber"))
+            foreach (var prop in props.Where(p => p.Name != "SelectorsCount" && p.Name != "ReadStripesNumber" &&
+                                                  !(p.Name == "MaxSize" &&
+                                                    p.DeclaringType == typeof(MemoryPolicyConfiguration))))
             {
                 var attr = prop.GetCustomAttributes(true).OfType<DefaultValueAttribute>().FirstOrDefault();
                 var propValue = prop.GetValue(obj, null);
@@ -567,14 +586,15 @@ namespace Apache.Ignite.Core.Tests
                     ConcurrencyLevel = 3,
                     DefaultMemoryPolicyName = "myDefaultPlc",
                     PageSize = 2048,
-                    SystemCacheMemorySize = 13 * 1024 * 1024,
+                    SystemCacheInitialSize = 13 * 1024 * 1024,
+                    SystemCacheMaxSize = 15 * 1024 * 1024,
                     MemoryPolicies = new[]
                     {
                         new MemoryPolicyConfiguration
                         {
                             Name = "myDefaultPlc",
                             PageEvictionMode = DataPageEvictionMode.Random2Lru,
-                            Size = 345 * 1024 * 1024,
+                            MaxSize = 345 * 1024 * 1024,
                             EvictionThreshold = 0.88,
                             EmptyPagesPoolSize = 77,
                             SwapFilePath = "myPath1"
@@ -583,7 +603,7 @@ namespace Apache.Ignite.Core.Tests
                         {
                             Name = "customPlc",
                             PageEvictionMode = DataPageEvictionMode.RandomLru,
-                            Size = 456 * 1024 * 1024,
+                            MaxSize = 456 * 1024 * 1024,
                             EvictionThreshold = 0.77,
                             EmptyPagesPoolSize = 66,
                             SwapFilePath = "somePath2"

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
index f3897e6..a6263d7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/DataPageEvictionMode.cs
@@ -34,10 +34,10 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <para />
         /// Once a memory region defined by a memory policy is configured, an off-heap array is allocated to track
         /// last usage timestamp for every individual data page. The size of the array equals to
-        /// <see cref="MemoryPolicyConfiguration.Size"/> / <see cref="MemoryConfiguration.PageSize"/>.
+        /// <see cref="MemoryPolicyConfiguration.MaxSize"/> / <see cref="MemoryConfiguration.PageSize"/>.
         /// <para />
         /// When a data page is accessed, its timestamp gets updated in the tracking array. The page index in the
-        /// tracking array equals to pageAddress / <see cref="MemoryPolicyConfiguration.Size"/>.
+        /// tracking array equals to pageAddress / <see cref="MemoryPolicyConfiguration.MaxSize"/>.
         /// <para />
         /// When some pages need to be evicted, the algorithm randomly chooses 5 indexes from the tracking array and
         /// evicts a page with the latest timestamp. If some of the indexes point to non-data pages

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
index 9c4bb35..36d06a7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryConfiguration.cs
@@ -46,9 +46,14 @@ namespace Apache.Ignite.Core.Cache.Configuration
     public class MemoryConfiguration
     {
         /// <summary>
-        /// The default system cache memory size.
+        /// Default size of a memory chunk reserved for system cache initially.
         /// </summary>
-        public const long DefaultSystemCacheMemorySize = 100 * 1024 * 1024;
+        public const long DefaultSystemCacheInitialSize = 40 * 1024 * 1024;
+
+        /// <summary>
+        /// Default max size of a memory chunk for the system cache.
+        /// </summary>
+        public const long DefaultSystemCacheMaxSize = 100 * 1024 * 1024;
 
         /// <summary>
         /// The default page size.
@@ -65,7 +70,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// </summary>
         public MemoryConfiguration()
         {
-            SystemCacheMemorySize = DefaultSystemCacheMemorySize;
+            SystemCacheInitialSize = DefaultSystemCacheInitialSize;
+            SystemCacheMaxSize = DefaultSystemCacheMaxSize;
             PageSize = DefaultPageSize;
             DefaultMemoryPolicyName = DefaultDefaultMemoryPolicyName;
         }
@@ -78,7 +84,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
         {
             Debug.Assert(reader != null);
 
-            SystemCacheMemorySize = reader.ReadLong();
+            SystemCacheInitialSize = reader.ReadLong();
+            SystemCacheMaxSize = reader.ReadLong();
             PageSize = reader.ReadInt();
             ConcurrencyLevel = reader.ReadInt();
             DefaultMemoryPolicyName = reader.ReadString();
@@ -101,7 +108,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
         {
             Debug.Assert(writer != null);
 
-            writer.WriteLong(SystemCacheMemorySize);
+            writer.WriteLong(SystemCacheInitialSize);
+            writer.WriteLong(SystemCacheMaxSize);
             writer.WriteInt(PageSize);
             writer.WriteInt(ConcurrencyLevel);
             writer.WriteString(DefaultMemoryPolicyName);
@@ -129,8 +137,14 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <summary>
         /// Gets or sets the size of a memory chunk reserved for system cache needs.
         /// </summary>
-        [DefaultValue(DefaultSystemCacheMemorySize)]
-        public long SystemCacheMemorySize { get; set; }
+        [DefaultValue(DefaultSystemCacheInitialSize)]
+        public long SystemCacheInitialSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum memory region size reserved for system cache.
+        /// </summary>
+        [DefaultValue(DefaultSystemCacheMaxSize)]
+        public long SystemCacheMaxSize { get; set; }
 
         /// <summary>
         /// Gets or sets the size of the memory page.

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
index fe4e91f..e6e9153 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/MemoryPolicyConfiguration.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
 {
     using System.ComponentModel;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl;
 
     /// <summary>
     /// Defines page memory policy configuration. See <see cref="MemoryConfiguration.MemoryPolicies"/>.
@@ -36,6 +37,16 @@ namespace Apache.Ignite.Core.Cache.Configuration
         public const int DefaultEmptyPagesPoolSize = 100;
 
         /// <summary>
+        /// The default initial size.
+        /// </summary>
+        public const long DefaultInitialSize = 256 * 1024 * 1024;
+
+        /// <summary>
+        /// The default maximum size, equals to 80% of total RAM.
+        /// </summary>
+        public static readonly long DefaultMaxSize = (long) ((long) NativeMethods.GetTotalPhysicalMemory() * 0.8);
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="MemoryPolicyConfiguration"/> class.
         /// </summary>
         public MemoryPolicyConfiguration()
@@ -43,6 +54,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
             EvictionThreshold = DefaultEvictionThreshold;
             EmptyPagesPoolSize = DefaultEmptyPagesPoolSize;
             Name = MemoryConfiguration.DefaultDefaultMemoryPolicyName;
+            InitialSize = DefaultInitialSize;
+            MaxSize = DefaultMaxSize;
         }
 
         /// <summary>
@@ -52,7 +65,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
         internal MemoryPolicyConfiguration(IBinaryRawReader reader)
         {
             Name = reader.ReadString();
-            Size = reader.ReadLong();
+            InitialSize = reader.ReadLong();
+            MaxSize = reader.ReadLong();
             SwapFilePath = reader.ReadString();
             PageEvictionMode = (DataPageEvictionMode) reader.ReadInt();
             EvictionThreshold = reader.ReadDouble();
@@ -65,7 +79,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
         internal void Write(IBinaryRawWriter writer)
         {
             writer.WriteString(Name);
-            writer.WriteLong(Size);
+            writer.WriteLong(InitialSize);
+            writer.WriteLong(MaxSize);
             writer.WriteString(SwapFilePath);
             writer.WriteInt((int) PageEvictionMode);
             writer.WriteDouble(EvictionThreshold);
@@ -80,10 +95,17 @@ namespace Apache.Ignite.Core.Cache.Configuration
         public string Name { get; set; }
 
         /// <summary>
-        /// Gets or sets the maximum memory region size defined by this memory policy.
-        /// If the whole data can not fit into the memory region an out of memory exception will be thrown.
+        /// Gets or sets initial memory region size defined by this memory policy.
+        /// When the used memory size exceeds this value, new chunks of memory will be allocated.
+        /// </summary>
+        [DefaultValue(DefaultInitialSize)]
+        public long InitialSize { get; set; }
+
+        /// <summary>
+        /// Sets maximum memory region size defined by this memory policy. The total size should not be less
+        /// than 10 MB due to internal data structures overhead.
         /// </summary>
-        public long Size { get; set; }
+        public long MaxSize { get; set; }
 
         /// <summary>
         /// Gets or sets the the path to the memory-mapped file the memory region defined by this memory policy
@@ -97,7 +119,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <summary>
         /// Gets or sets the page eviction mode. If <see cref="DataPageEvictionMode.Disabled"/> is used (default)
         /// then an out of memory exception will be thrown if the memory region usage,
-        /// defined by this memory policy, goes beyond <see cref="Size"/>.
+        /// defined by this memory policy, goes beyond <see cref="MaxSize"/>.
         /// </summary>
         public DataPageEvictionMode PageEvictionMode { get; set; }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
index 596322b..8289aaf 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
@@ -249,14 +249,6 @@ namespace Apache.Ignite.Core.Cache
         long OffHeapAllocatedSize { get; }
 
         /// <summary>
-        /// Gets off-heap memory maximum size.
-        /// </summary>
-        /// <returns>
-        /// Off-heap memory maximum size.
-        /// </returns>
-        long OffHeapMaxSize { get; }
-
-        /// <summary>
         /// Gets number of non-null values in the cache.
         /// </summary>
         /// <returns>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/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 9098d89..295457a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1183,7 +1183,12 @@
                                                         <xs:documentation>Page eviction mode.</xs:documentation>
                                                     </xs:annotation>
                                                 </xs:attribute>
-                                                <xs:attribute name="size" type="xs:int" use="required">
+                                                <xs:attribute name="initialSize" type="xs:long">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Initial memory region size defined by this memory policy.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
+                                                <xs:attribute name="maxSize" type="xs:long">
                                                     <xs:annotation>
                                                         <xs:documentation>Maximum memory region size defined by this memory policy.</xs:documentation>
                                                     </xs:annotation>
@@ -1214,9 +1219,14 @@
                                 <xs:documentation>Size of the memory page.</xs:documentation>
                             </xs:annotation>
                         </xs:attribute>
-                        <xs:attribute name="systemCacheMemorySize" type="xs:int">
+                        <xs:attribute name="systemCacheInitialSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Initial size of a memory chunk reserved for system cache needs.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="systemCacheMaxSize" type="xs:int">
                             <xs:annotation>
-                                <xs:documentation>Size of a memory chunk reserved for system cache needs.</xs:documentation>
+                                <xs:documentation>Maximum size of a memory chunk reserved for system cache needs.</xs:documentation>
                             </xs:annotation>
                         </xs:attribute>
                     </xs:complexType>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
index 53ff810..9ce713f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
@@ -110,9 +110,6 @@ namespace Apache.Ignite.Core.Impl.Cache
         private readonly long _offHeapAllocatedSize;
 
         /** */
-        private readonly long _offHeapMaxSize;
-
-        /** */
         private readonly int _size;
 
         /** */
@@ -248,7 +245,6 @@ namespace Apache.Ignite.Core.Impl.Cache
             _offHeapPrimaryEntriesCount = reader.ReadLong();
             _offHeapBackupEntriesCount = reader.ReadLong();
             _offHeapAllocatedSize = reader.ReadLong();
-            _offHeapMaxSize = reader.ReadLong();
             _size = reader.ReadInt();
             _keySize = reader.ReadInt();
             _isEmpty = reader.ReadBoolean();
@@ -370,9 +366,6 @@ namespace Apache.Ignite.Core.Impl.Cache
         public long OffHeapAllocatedSize { get { return _offHeapAllocatedSize; } }
 
         /** <inheritDoc /> */
-        public long OffHeapMaxSize { get { return _offHeapMaxSize; } }
-
-        /** <inheritDoc /> */
         public int Size { get { return _size; } }
 
         /** <inheritDoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
index 3403dee..0004772 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
@@ -45,5 +45,49 @@ namespace Apache.Ignite.Core.Impl
         [DllImport("kernel32.dll", SetLastError = true, CharSet = CharSet.Ansi, BestFitMapping = false, 
             ThrowOnUnmappableChar = true)]
         internal static extern IntPtr LoadLibrary(string path);
+
+        /// <summary>
+        /// Gets the total physical memory.
+        /// </summary>
+        internal static ulong GetTotalPhysicalMemory()
+        {
+            var status = new MEMORYSTATUSEX();
+            status.Init();
+
+            GlobalMemoryStatusEx(ref status);
+
+            return status.ullTotalPhys;
+        }
+
+        /// <summary>
+        /// Globals the memory status.
+        /// </summary>
+        [return: MarshalAs(UnmanagedType.Bool)]
+        [DllImport("kernel32.dll", CharSet = CharSet.Auto, SetLastError = true)]
+        private static extern bool GlobalMemoryStatusEx([In, Out] ref MEMORYSTATUSEX lpBuffer);
+
+        [StructLayout(LayoutKind.Sequential, CharSet = CharSet.Auto)]
+        // ReSharper disable InconsistentNaming
+        // ReSharper disable MemberCanBePrivate.Local
+        private struct MEMORYSTATUSEX
+        {
+            public uint dwLength;
+            public readonly uint dwMemoryLoad;
+            public readonly ulong ullTotalPhys;
+            public readonly ulong ullAvailPhys;
+            public readonly ulong ullTotalPageFile;
+            public readonly ulong ullAvailPageFile;
+            public readonly ulong ullTotalVirtual;
+            public readonly ulong ullAvailVirtual;
+            public readonly ulong ullAvailExtendedVirtual;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="MEMORYSTATUSEX"/> struct.
+            /// </summary>
+            public void Init()
+            {
+                dwLength = (uint) Marshal.SizeOf(typeof(MEMORYSTATUSEX));
+            }
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9c6a0d7..9a672cc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -524,7 +524,7 @@
                                         <mkdir dir="${basedir}/target/release-package/benchmarks" />
 
                                         <copy todir="${basedir}/target/release-package/benchmarks/">
-                                            <fileset dir="${basedir}/modules/yardstick/target/assembly/"/>
+                                            <fileset dir="${basedir}/modules/yardstick/target/assembly/" />
                                         </copy>
 
                                         <!--todo: only required jars should be exported to /benchmarks/libs during compilation-->
@@ -541,38 +541,38 @@
                                         <delete>
                                             <fileset dir="${basedir}/target/release-package/benchmarks/config/">
                                                 <include name="*.*" />
-                                                <exclude name="benchmark.properties"/>
-                                                <exclude name="benchmark-remote.properties"/>
-                                                <exclude name="benchmark-sample.properties"/>
-                                                <exclude name="benchmark-remote-sample.properties"/>
-                                                <exclude name="benchmark-multicast.properties"/>
-                                                <exclude name="ignite-base-config.xml"/>
-                                                <exclude name="ignite-localhost-config.xml"/>
-                                                <exclude name="ignite-remote-config.xml"/>
-                                                <exclude name="ignite-multicast-config.xml"/>
+                                                <exclude name="benchmark.properties" />
+                                                <exclude name="benchmark-remote.properties" />
+                                                <exclude name="benchmark-sample.properties" />
+                                                <exclude name="benchmark-remote-sample.properties" />
+                                                <exclude name="benchmark-multicast.properties" />
+                                                <exclude name="ignite-base-config.xml" />
+                                                <exclude name="ignite-localhost-config.xml" />
+                                                <exclude name="ignite-remote-config.xml" />
+                                                <exclude name="ignite-multicast-config.xml" />
                                             </fileset>
                                         </delete>
 
                                         <mkdir dir="${basedir}/target/release-package/benchmarks/sources/src" />
 
                                         <copy todir="${basedir}/target/release-package/benchmarks/sources/src/">
-                                            <fileset dir="${basedir}/modules/yardstick/src"/>
+                                            <fileset dir="${basedir}/modules/yardstick/src" />
                                         </copy>
 
                                         <mkdir dir="${basedir}/target/release-package/benchmarks/sources/config" />
 
                                         <copy todir="${basedir}/target/release-package/benchmarks/sources/config/">
-                                            <fileset dir="${basedir}/target/release-package/benchmarks/config"/>
+                                            <fileset dir="${basedir}/target/release-package/benchmarks/config" />
                                         </copy>
 
                                         <copy file="${basedir}/modules/yardstick/pom-standalone.xml"
                                               tofile="${basedir}/target/release-package/benchmarks/sources/pom.xml"/>
 
                                         <replaceregexp byline="true">
-                                            <regexp pattern="to_be_replaced_by_ignite_version"/>
-                                            <substitution expression="${project.version}"/>
-                                            <fileset dir="${basedir}/target/release-package/benchmarks/sources/" >
-                                                <include name="pom.xml"/>
+                                            <regexp pattern="to_be_replaced_by_ignite_version" />
+                                            <substitution expression="${project.version}" />
+                                            <fileset dir="${basedir}/target/release-package/benchmarks/sources/">
+                                                <include name="pom.xml" />
                                             </fileset>
                                         </replaceregexp>
 


[3/3] ignite git commit: IGNITE-5072 - Updated memory metrics to comply with other metrics

Posted by ag...@apache.org.
IGNITE-5072 - Updated memory metrics to comply with other metrics


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

Branch: refs/heads/ignite-2.0
Commit: 11c23b628af0cec6cf8ddf2da9a510c460877f22
Parents: f5fe301
Author: Sergey Chugunov <se...@gmail.com>
Authored: Thu Apr 27 19:56:02 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Apr 27 19:56:02 2017 +0300

----------------------------------------------------------------------
 examples/config/example-memory-policies.xml     |  16 +-
 .../benchmarks/jmh/tree/BPlusTreeBenchmark.java |  15 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   1 -
 .../java/org/apache/ignite/MemoryMetrics.java   |  73 ++--
 .../org/apache/ignite/cache/CacheMetrics.java   |   8 -
 .../configuration/DataPageEvictionMode.java     |   4 +-
 .../configuration/MemoryConfiguration.java      | 109 ++++-
 .../MemoryPolicyConfiguration.java              |  82 +++-
 .../apache/ignite/internal/IgniteKernal.java    |  11 +-
 .../ignite/internal/mem/DirectMemory.java       |  55 ---
 .../internal/mem/DirectMemoryProvider.java      |  19 +-
 .../mem/file/MappedFileMemoryProvider.java      | 153 ++-----
 .../mem/unsafe/UnsafeMemoryProvider.java        |  69 ++--
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 408 +++++++++++++------
 .../cache/CacheClusterMetricsMXBeanImpl.java    |   5 -
 .../cache/CacheLocalMetricsMXBeanImpl.java      |   5 -
 .../processors/cache/CacheMetricsImpl.java      |   5 -
 .../processors/cache/CacheMetricsSnapshot.java  |  13 -
 .../IgniteCacheDatabaseSharedManager.java       | 235 +++++++----
 .../cache/database/MemoryMetricsImpl.java       |  24 +-
 .../cache/database/MemoryMetricsMXBeanImpl.java | 108 +++++
 .../cache/database/MemoryMetricsSnapshot.java   |  85 ++++
 .../processors/cache/database/MemoryPolicy.java |   7 +-
 .../evict/FairFifoPageEvictionTracker.java      |   6 +-
 .../evict/PageAbstractEvictionTracker.java      |  85 +---
 .../evict/Random2LruPageEvictionTracker.java    |   6 +-
 .../evict/RandomLruPageEvictionTracker.java     |   6 +-
 .../processors/igfs/IgfsDataManager.java        |   2 +-
 .../platform/cache/PlatformCache.java           |   1 -
 .../utils/PlatformConfigurationUtils.java       |  18 +-
 .../service/GridServiceProcessor.java           |   9 +-
 .../ignite/internal/util/IgniteUtils.java       |  28 +-
 .../visor/node/VisorMemoryConfiguration.java    |   2 +-
 .../node/VisorMemoryPolicyConfiguration.java    |   2 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |   4 -
 .../ignite/mxbean/MemoryMetricsMXBean.java      |  83 ++--
 .../internal/ClusterNodeMetricsSelfTest.java    |   6 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  18 +-
 .../cache/CacheConfigurationLeakTest.java       |   2 +-
 .../CacheMemoryPolicyConfigurationTest.java     |  10 +-
 .../cache/MemoryPolicyConfigValidationTest.java | 121 +++++-
 .../MemoryPolicyInitializationTest.java         |  16 +-
 .../paged/PageEvictionAbstractTest.java         |   4 +-
 .../TxPessimisticDeadlockDetectionTest.java     |   2 +-
 .../processors/database/BPlusTreeSelfTest.java  |  26 +-
 .../database/FreeListImplSelfTest.java          |  26 +-
 .../database/IgniteDbDynamicCacheSelfTest.java  |   2 +-
 .../database/MemoryMetricsSelfTest.java         |   5 +-
 .../database/MetadataStorageSelfTest.java       |  21 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   2 +-
 .../platform/PlatformCacheWriteMetricsTask.java |   5 -
 ...stributedPartitionQueryAbstractSelfTest.java |   5 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   7 +-
 .../index/DynamicIndexAbstractSelfTest.java     |   6 +-
 .../h2/database/InlineIndexHelperTest.java      |  40 +-
 .../cpp/core-test/config/cache-identity-32.xml  |   7 +-
 .../cpp/core-test/config/cache-query-32.xml     |   6 +-
 .../config/cache-query-continuous-32.xml        |   6 +-
 .../cpp/core-test/config/cache-store-32.xml     |   6 +-
 .../cpp/core-test/config/cache-test-32.xml      |   6 +-
 .../cpp/odbc-test/config/queries-test-32.xml    |   6 +-
 .../odbc-test/config/queries-test-noodbc-32.xml |   6 +-
 .../Cache/CacheConfigurationTest.cs             |   3 +-
 .../Cache/CacheMetricsTest.cs                   |   1 -
 .../Config/spring-test.xml                      |  14 +
 .../IgniteConfigurationSerializerTest.cs        |  17 +-
 .../IgniteConfigurationTest.cs                  |  32 +-
 .../Cache/Configuration/DataPageEvictionMode.cs |   4 +-
 .../Cache/Configuration/MemoryConfiguration.cs  |  28 +-
 .../Configuration/MemoryPolicyConfiguration.cs  |  34 +-
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |   8 -
 .../IgniteConfigurationSection.xsd              |  16 +-
 .../Impl/Cache/CacheMetricsImpl.cs              |   7 -
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |  44 ++
 pom.xml                                         |  32 +-
 75 files changed, 1484 insertions(+), 885 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/examples/config/example-memory-policies.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-memory-policies.xml b/examples/config/example-memory-policies.xml
index 121b8a5..86c7502 100644
--- a/examples/config/example-memory-policies.xml
+++ b/examples/config/example-memory-policies.xml
@@ -38,6 +38,8 @@
                 <property name="defaultMemoryPolicyName" value="Default_Region"/>
                 <!-- Setting the page size to 4 KB -->
                 <property name="pageSize" value="4096"/>
+                <property name="systemCacheInitialSize" value="#{40 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
 
                 <!-- Defining several memory policies for different memory regions -->
                 <property name="memoryPolicies">
@@ -49,7 +51,7 @@
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="Default_Region"/>
                             <!-- 100 MB memory region with disabled eviction -->
-                            <property name="size" value="#{100 * 1024 * 1024}"/>
+                            <property name="initialSize" value="#{100 * 1024 * 1024}"/>
                         </bean>
 
                         <!--
@@ -57,8 +59,10 @@
                         -->
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="20MB_Region_Eviction"/>
-                            <!-- 20 MB memory region. -->
-                            <property name="size" value="#{20 * 1024 * 1024}"/>
+                            <!-- Memory region of 20 MB initial size. -->
+                            <property name="initialSize" value="#{20 * 1024 * 1024}"/>
+                            <!-- Maximum size is 40 MB. -->
+                            <property name="maxSize" value="#{40 * 1024 * 1024}"/>
                             <!-- Enabling eviction for this memory region -->
                             <property name="pageEvictionMode" value="RANDOM_2_LRU"/>
                         </bean>
@@ -69,8 +73,10 @@
                         -->
                         <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
                             <property name="name" value="15MB_Region_Swapping"/>
-                            <!-- 15 MB memory region. -->
-                            <property name="size" value="#{15 * 1024 * 1024}"/>
+                            <!-- Memory region of 15 MB initial size. -->
+                            <property name="initialSize" value="#{15 * 1024 * 1024}"/>
+                            <!-- Maximum size is 30 MB. -->
+                            <property name="maxSize" value="#{30 * 1024 * 1024}"/>
                             <!-- Setting a name of the swapping file. -->
                             <property name="swapFilePath" value="memoryPolicyExampleSwap"/>
                         </bean>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
index 5833e1f..c9ad0cf 100644
--- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
@@ -21,6 +21,7 @@ import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.benchmarks.jmh.JmhAbstractBenchmark;
 import org.apache.ignite.internal.benchmarks.jmh.runner.JmhIdeBenchmarkRunner;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
@@ -209,12 +210,14 @@ public class BPlusTreeBenchmark extends JmhAbstractBenchmark {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
+
         PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+            new UnsafeMemoryProvider(new JavaLogger()),
             null,
             PAGE_SIZE,
-            null,
-            new MemoryMetricsImpl(null),
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg),
             false);
 
         pageMem.start();
@@ -281,8 +284,7 @@ public class BPlusTreeBenchmark extends JmhAbstractBenchmark {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             return PageUtils.getLong(pageAddr, offset(idx));
         }
     }
@@ -318,8 +320,7 @@ public class BPlusTreeBenchmark extends JmhAbstractBenchmark {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             return PageUtils.getLong(pageAddr, offset(idx));
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 7445264..267f4f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -615,7 +615,6 @@ public interface Ignite extends AutoCloseable {
      */
     public void resetLostPartitions(Collection<String> cacheNames);
 
-
     /**
      * Returns collection {@link MemoryMetrics} objects providing information about memory usage in current Ignite instance.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/MemoryMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/MemoryMetrics.java b/modules/core/src/main/java/org/apache/ignite/MemoryMetrics.java
index b4097d1..96eedfe 100644
--- a/modules/core/src/main/java/org/apache/ignite/MemoryMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/MemoryMetrics.java
@@ -17,36 +17,45 @@
 
 package org.apache.ignite;
 
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.mxbean.MemoryMetricsMXBean;
+
 /**
- * Interface provides methods to access metrics of memory usage on local instance of Ignite.
+ * An interface to collect metrics about page memory usage on Ignite node. Overall page memory architecture
+ * is described in {@link MemoryConfiguration} javadoc.
+ * <p>
+ * As multiple page memories may be configured on a single Ignite node; memory metrics will be collected
+ * for each page memory separately.
+ * </p>
+ * <p>
+ * There are two ways to access metrics on local node.
+ * <ol>
+ *     <li>
+ *       Firstly, collection of metrics can be obtained through {@link Ignite#memoryMetrics()} call.<br/>
+ *       Please pay attention that this call returns snapshots of memory metrics and not live objects.
+ *     </li>
+ *     <li>
+ *       Secondly, all {@link MemoryMetrics} on local node are exposed through JMX interface. <br/>
+ *       See {@link MemoryMetricsMXBean} interface describing information provided about metrics
+ *       and page memory configuration.
+ *     </li>
+ * </ol>
+ * </p>
+ * <p>
+ * Also users must be aware that using memory metrics has some overhead and for performance reasons is turned off
+ * by default.
+ * For turning them on both {@link MemoryPolicyConfiguration#setMetricsEnabled(boolean)} configuration property
+ * or {@link MemoryMetricsMXBean#enableMetrics()} method of JMX bean can be used.
+ * </p>
  */
 public interface MemoryMetrics {
     /**
-     * @return Memory policy name.
+     * @return Name of memory region metrics are collected for.
      */
     public String getName();
 
     /**
-     * @return Returns size (in MBytes) of MemoryPolicy observed by this MemoryMetrics MBean.
-     */
-    public int getSize();
-
-    /**
-     * @return Path of memory-mapped file used to swap PageMemory pages to disk.
-     */
-    public String getSwapFilePath();
-
-    /**
-     * Enables collecting memory metrics.
-     */
-    public void enableMetrics();
-
-    /**
-     * Disables collecting memory metrics.
-     */
-    public void disableMetrics();
-
-    /**
      * @return Total number of allocated pages.
      */
     public long getTotalAllocatedPages();
@@ -72,24 +81,4 @@ public interface MemoryMetrics {
      * @return Free space to overall size ratio across all pages in FreeList.
      */
     public float getPagesFillFactor();
-
-    /**
-     * Sets interval of time (in seconds) to monitor allocation rate.
-     *
-     * E.g. after setting rateTimeInterval to 60 seconds subsequent calls to {@link #getAllocationRate()}
-     * will return average allocation rate (pages per second) for the last minute.
-     *
-     * @param rateTimeInterval Time interval used to calculate allocation/eviction rate.
-     */
-    public void rateTimeInterval(int rateTimeInterval);
-
-    /**
-     * Sets number of subintervals the whole rateTimeInterval will be split into to calculate allocation rate,
-     * 5 by default.
-     * Setting it to bigger number allows more precise calculation and smaller drops of allocationRate metric
-     * when next subinterval has to be recycled but introduces bigger calculation overhead.
-     *
-     * @param subInts Number of subintervals.
-     */
-    public void subIntervals(int subInts);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
index c0eb98e..3e25aa5 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
@@ -109,7 +109,6 @@ public interface CacheMetrics {
      */
     public float getAverageRemoveTime();
 
-
     /**
      * The mean time to execute tx commit.
      *
@@ -230,13 +229,6 @@ public interface CacheMetrics {
     public long getOffHeapAllocatedSize();
 
     /**
-     * Gets off-heap memory maximum size.
-     *
-     * @return Off-heap memory maximum size.
-     */
-    public long getOffHeapMaxSize();
-
-    /**
      * Gets number of non-{@code null} values in the cache.
      *
      * @return Number of non-{@code null} values in the cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/configuration/DataPageEvictionMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataPageEvictionMode.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataPageEvictionMode.java
index 1aec15a..f61e870 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/DataPageEvictionMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataPageEvictionMode.java
@@ -33,9 +33,9 @@ public enum DataPageEvictionMode {
      * <ul>
      * <li>Once a memory region defined by a memory policy is configured, an off-heap array is allocated to track
      * last usage timestamp for every individual data page. The size of the array is calculated this way - size =
-     * ({@link MemoryPolicyConfiguration#getSize()} / {@link MemoryConfiguration#pageSize})</li>
+     * ({@link MemoryPolicyConfiguration#getMaxSize()} / {@link MemoryConfiguration#pageSize})</li>
      * <li>When a data page is accessed, its timestamp gets updated in the tracking array. The page index in the
-     * tracking array is calculated this way - index = (pageAddress / {@link MemoryPolicyConfiguration#getSize()}</li>
+     * tracking array is calculated this way - index = (pageAddress / {@link MemoryPolicyConfiguration#getMaxSize()}</li>
      * <li>When it's required to evict some pages, the algorithm randomly chooses 5 indexes from the tracking array and
      * evicts a page with the latest timestamp. If some of the indexes point to non-data pages (index or system pages)
      * then the algorithm picks other pages.</li>

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
index f88a95a..cadc033 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
@@ -33,7 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
  * <p>
  * If initial size of the default memory region doesn't satisfy requirements or it's required to have multiple memory
  * regions with different properties then {@link MemoryPolicyConfiguration} can be used for both scenarios.
- * For instance, Using memory policies you can define memory regions of different maximum size, eviction policies,
+ * For instance, using memory policies you can define memory regions of different maximum size, eviction policies,
  * swapping options, etc. Once you define a new memory region you can bind particular Ignite caches to it.
  * <p>
  * To learn more about memory policies refer to {@link MemoryPolicyConfiguration} documentation.
@@ -42,14 +42,14 @@ import org.apache.ignite.internal.util.typedef.internal.U;
  *     {@code
  *     <property name="memoryConfiguration">
  *         <bean class="org.apache.ignite.configuration.MemoryConfiguration">
- *             <property name="systemCacheMemorySize" value="#{100 * 1024 * 1024}"/>
+ *             <property name="systemCacheInitialSize" value="#{100 * 1024 * 1024}"/>
  *             <property name="defaultMemoryPolicyName" value="default_mem_plc"/>
  *
  *             <property name="memoryPolicies">
  *                 <list>
  *                     <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
  *                         <property name="name" value="default_mem_plc"/>
- *                         <property name="size" value="#{5 * 1024 * 1024 * 1024}"/>
+ *                         <property name="initialSize" value="#{5 * 1024 * 1024 * 1024}"/>
  *                     </bean>
  *                 </list>
  *             </property>
@@ -62,11 +62,22 @@ public class MemoryConfiguration implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Default memory policy's size (1 GB). */
-    public static final long DFLT_MEMORY_POLICY_SIZE = 1024 * 1024 * 1024;
+    /** Default memory policy start size (256 MB). */
+    public static final long DFLT_MEMORY_POLICY_INITIAL_SIZE = 256 * 1024 * 1024;
 
-    /** Default size of a memory chunk for the system cache (100 MB). */
-    public static final long DFLT_SYS_CACHE_MEM_SIZE = 100 * 1024 * 1024;
+    /** Fraction of available memory to allocate for default MemoryPolicy. */
+    private static final double DFLT_MEMORY_POLICY_FRACTION = 0.8;
+
+    /** Default memory policy's size is 80% of physical memory available on current machine. */
+    public static final long DFLT_MEMORY_POLICY_MAX_SIZE = Math.max(
+        (long)(DFLT_MEMORY_POLICY_FRACTION * U.getTotalMemoryAvailable()),
+        DFLT_MEMORY_POLICY_INITIAL_SIZE);
+
+    /** Default initial size of a memory chunk for the system cache (40 MB). */
+    private static final long DFLT_SYS_CACHE_INIT_SIZE = 40 * 1024 * 1024;
+
+    /** Default max size of a memory chunk for the system cache (100 MB). */
+    private static final long DFLT_SYS_CACHE_MAX_SIZE = 100 * 1024 * 1024;
 
     /** Default memory page size. */
     public static final int DFLT_PAGE_SIZE = 2 * 1024;
@@ -74,8 +85,11 @@ public class MemoryConfiguration implements Serializable {
     /** This name is assigned to default MemoryPolicy if no user-defined default MemPlc is specified */
     public static final String DFLT_MEM_PLC_DEFAULT_NAME = "default";
 
-    /** Size of a memory chunk reserved for system cache needs. */
-    private long sysCacheMemSize = DFLT_SYS_CACHE_MEM_SIZE;
+    /** Size of a memory chunk reserved for system cache initially. */
+    private long sysCacheInitSize = DFLT_SYS_CACHE_INIT_SIZE;
+
+    /** Maximum size of system cache. */
+    private long sysCacheMaxSize = DFLT_SYS_CACHE_MAX_SIZE;
 
     /** Memory page size. */
     private int pageSize = DFLT_PAGE_SIZE;
@@ -86,27 +100,55 @@ public class MemoryConfiguration implements Serializable {
     /** A name of the memory policy that defines the default memory region. */
     private String dfltMemPlcName = DFLT_MEM_PLC_DEFAULT_NAME;
 
+    /** Size of memory (in bytes) to use for default MemoryPolicy. */
+    private Long dfltMemPlcSize;
+
     /** Memory policies. */
     private MemoryPolicyConfiguration[] memPlcs;
 
     /**
-     * Gets size of a memory chunk reserved for system cache needs.
+     * Initial size of a memory region reserved for system cache.
+     *
+     * @return Size in bytes.
+     */
+    public long getSystemCacheInitialSize() {
+        return sysCacheInitSize;
+    }
+
+    /**
+     * Sets initial size of a memory region reserved for system cache.
+     *
+     * Default value is {@link #DFLT_SYS_CACHE_INIT_SIZE}
+     *
+     * @param sysCacheInitSize Size in bytes.
+     *
+     * @return {@code this} for chaining.
+     */
+    public MemoryConfiguration setSystemCacheInitialSize(long sysCacheInitSize) {
+        this.sysCacheInitSize = sysCacheInitSize;
+
+        return this;
+    }
+
+    /**
+     * Maximum memory region size reserved for system cache.
      *
      * @return Size in bytes.
      */
-    public long getSystemCacheMemorySize() {
-        return sysCacheMemSize;
+    public long getSystemCacheMaxSize() {
+        return sysCacheMaxSize;
     }
 
     /**
-     * Sets the size of a memory chunk reserved for system cache needs.
+     * Sets maximum memory region size reserved for system cache. The total size should not be less than 10 MB
+     * due to internal data structures overhead.
      *
-     * Default value is {@link #DFLT_SYS_CACHE_MEM_SIZE}
+     * @param sysCacheMaxSize Maximum size in bytes for system cache memory region.
      *
-     * @param sysCacheMemSize Size in bytes.
+     * @return {@code this} for chaining.
      */
-    public MemoryConfiguration setSystemCacheMemorySize(long sysCacheMemSize) {
-        this.sysCacheMemSize = sysCacheMemSize;
+    public MemoryConfiguration setSystemCacheMaxSize(long sysCacheMaxSize) {
+        this.sysCacheMaxSize = sysCacheMaxSize;
 
         return this;
     }
@@ -171,7 +213,12 @@ public class MemoryConfiguration implements Serializable {
     public MemoryPolicyConfiguration createDefaultPolicyConfig() {
         MemoryPolicyConfiguration memPlc = new MemoryPolicyConfiguration();
 
-        memPlc.setSize(DFLT_MEMORY_POLICY_SIZE);
+        long maxSize = (dfltMemPlcSize != null) ? dfltMemPlcSize : DFLT_MEMORY_POLICY_MAX_SIZE;
+
+        if (maxSize < DFLT_MEMORY_POLICY_INITIAL_SIZE)
+            memPlc.setInitialSize(maxSize);
+
+        memPlc.setMaxSize(maxSize);
 
         return memPlc;
     }
@@ -198,6 +245,32 @@ public class MemoryConfiguration implements Serializable {
     }
 
     /**
+     * Gets a size for default memory policy overridden by user.
+     *
+     * @return default memory policy size overridden by user or -1 if nothing was specified.
+     */
+    public long getDefaultMemoryPolicySize() {
+        return (dfltMemPlcSize != null) ? dfltMemPlcSize : -1;
+    }
+
+    /**
+     * Overrides size of default memory policy which is created automatically.
+     *
+     * If user doesn't specify any memory policy configuration, a default one with default size
+     * (80% of available RAM) is created by Ignite.
+     *
+     * This property allows user to specify desired size of default memory policy
+     * without having to use more verbose syntax of MemoryPolicyConfiguration elements.
+     *
+     * @param dfltMemPlcSize Size of default memory policy overridden by user.
+     */
+    public MemoryConfiguration setDefaultMemoryPolicySize(long dfltMemPlcSize) {
+        this.dfltMemPlcSize = dfltMemPlcSize;
+
+        return this;
+    }
+
+    /**
      * Gets a name of default memory policy.
      *
      * @return A name of a custom memory policy configured with {@code MemoryConfiguration} or {@code null} of the

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
index d0adcf9..55da5bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -39,18 +39,19 @@ import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_MEM_PLC_D
  *                 <list>
  *                      <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
  *                          <property name="name" value="Default_Region"/>
- *                          <property name="size" value="#{100 * 1024 * 1024}"/>
+ *                          <property name="initialSize" value="#{100 * 1024 * 1024}"/>
  *                      </bean>
  *
  *                      <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
  *                          <property name="name" value="20MB_Region_Eviction"/>
- *                          <property name="size" value="#{20 * 1024 * 1024}"/>
+ *                          <property name="initialSize" value="#{20 * 1024 * 1024}"/>
  *                          <property name="pageEvictionMode" value="RANDOM_2_LRU"/>
  *                      </bean>
  *
  *                      <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
  *                          <property name="name" value="25MB_Region_Swapping"/>
- *                          <property name="size" value="#{25 * 1024 * 1024}"/>
+ *                          <property name="initialSize" value="#{25 * 1024 * 1024}"/>
+ *                          <property name="initialSize" value="#{100 * 1024 * 1024}"/>
  *                          <property name="swapFilePath" value="memoryPolicyExampleSwap"/>
  *                      </bean>
  *                  </list>
@@ -62,11 +63,17 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Default metrics enabled flag. */
+    public static final boolean DFLT_METRICS_ENABLED = false;
+
     /** Memory policy name. */
     private String name = DFLT_MEM_PLC_DEFAULT_NAME;
 
+    /** Memory policy start size. */
+    private long initialSize = MemoryConfiguration.DFLT_MEMORY_POLICY_INITIAL_SIZE;
+
     /** Memory policy maximum size. */
-    private long size;
+    private long maxSize = MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE;
 
     /** An optional path to a memory mapped file for this memory policy. */
     private String swapFilePath;
@@ -83,6 +90,9 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /** Minimum number of empty pages in reuse lists. */
     private int emptyPagesPoolSize = 100;
 
+    /** */
+    private boolean metricsEnabled = DFLT_METRICS_ENABLED;
+
     /**
      * Gets memory policy name.
      *
@@ -98,6 +108,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
      * If not specified, {@link MemoryConfiguration#DFLT_MEM_PLC_DEFAULT_NAME} value is used.
      *
      * @param name Memory policy name.
+     * @return {@code this} for chaining.
      */
     public MemoryPolicyConfiguration setName(String name) {
         this.name = name;
@@ -111,16 +122,42 @@ public final class MemoryPolicyConfiguration implements Serializable {
      *
      * @return Size in bytes.
      */
-    public long getSize() {
-        return size;
+    public long getMaxSize() {
+        return maxSize;
     }
 
     /**
      * Sets maximum memory region size defined by this memory policy. The total size should not be less than 10 MB
      * due to the internal data structures overhead.
+     *
+     * @param maxSize Maximum memory policy size in bytes.
+     * @return {@code this} for chaining.
      */
-    public MemoryPolicyConfiguration setSize(long size) {
-        this.size = size;
+    public MemoryPolicyConfiguration setMaxSize(long maxSize) {
+        this.maxSize = maxSize;
+
+        return this;
+    }
+
+    /**
+     * Gets initial memory region size defined by this memory policy. When the used memory size exceeds this value,
+     * new chunks of memory will be allocated.
+     *
+     * @return Memory policy start size.
+     */
+    public long getInitialSize() {
+        return initialSize;
+    }
+
+    /**
+     * Sets initial memory region size defined by this memory policy. When the used memory size exceeds this value,
+     * new chunks of memory will be allocated.
+     *
+     * @param initialSize Memory policy initial size.
+     * @return {@code this} for chaining.
+     */
+    public MemoryPolicyConfiguration setInitialSize(long initialSize) {
+        this.initialSize = initialSize;
 
         return this;
     }
@@ -140,6 +177,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
      * Sets a path to the memory-mapped file.
      *
      * @param swapFilePath A Path to the memory mapped file.
+     * @return {@code this} for chaining.
      */
     public MemoryPolicyConfiguration setSwapFilePath(String swapFilePath) {
         this.swapFilePath = swapFilePath;
@@ -150,7 +188,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /**
      * Gets memory pages eviction mode. If {@link DataPageEvictionMode#DISABLED} is used (default) then an out of
      * memory exception will be thrown if the memory region usage, defined by this memory policy, goes beyond its
-     * capacity which is {@link #getSize()}.
+     * capacity which is {@link #getMaxSize()}.
      *
      * @return Memory pages eviction algorithm. {@link DataPageEvictionMode#DISABLED} used by default.
      */
@@ -162,6 +200,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
      * Sets memory pages eviction mode.
      *
      * @param evictionMode Eviction mode.
+     * @return {@code this} for chaining.
      */
     public MemoryPolicyConfiguration setPageEvictionMode(DataPageEvictionMode evictionMode) {
         pageEvictionMode = evictionMode;
@@ -183,6 +222,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
      * Sets memory pages eviction threshold.
      *
      * @param evictionThreshold Eviction threshold.
+     * @return {@code this} for chaining.
      */
     public MemoryPolicyConfiguration setEvictionThreshold(double evictionThreshold) {
         this.evictionThreshold = evictionThreshold;
@@ -213,10 +253,34 @@ public final class MemoryPolicyConfiguration implements Serializable {
      * Increase this parameter if {@link IgniteOutOfMemoryException} occurred with enabled page eviction.
      *
      * @param emptyPagesPoolSize Empty pages pool size.
+     * @return {@code this} for chaining.
      */
     public MemoryPolicyConfiguration setEmptyPagesPoolSize(int emptyPagesPoolSize) {
         this.emptyPagesPoolSize = emptyPagesPoolSize;
 
         return this;
     }
+
+    /**
+     * Gets whether memory metrics are enabled by default on node startup. Memory metrics can be enabled and disabled
+     * at runtime via memory metrics MX bean.
+     *
+     * @return Metrics enabled flag.
+     */
+    public boolean isMetricsEnabled() {
+        return metricsEnabled;
+    }
+
+    /**
+     * Sets memory metrics enabled flag. If this flag is {@code true}, metrics will be enabled on node startup.
+     * Memory metrics can be enabled and disabled at runtime via memory metrics MX bean.
+     *
+     * @param metricsEnabled Metrics enanabled flag.
+     * @return {@code this} for chaining.
+     */
+    public MemoryPolicyConfiguration setMetricsEnabled(boolean metricsEnabled) {
+        this.metricsEnabled = metricsEnabled;
+
+        return this;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index f0bf29b..8ba6a88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2434,7 +2434,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             return;
 
         U.log(log, "System cache's MemoryPolicy size is configured to " +
-            (memCfg.getSystemCacheMemorySize() / (1024 * 1024)) + " MB. " +
+            (memCfg.getSystemCacheInitialSize() / (1024 * 1024)) + " MB. " +
             "Use MemoryConfiguration.systemCacheMemorySize property to change the setting.");
     }
 
@@ -3399,7 +3399,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public Collection<MemoryMetrics> memoryMetrics() {
-        return ctx.cache().context().database().memoryMetrics();
+        guard();
+
+        try {
+            return ctx.cache().context().database().memoryMetrics();
+        }
+        finally {
+            unguard();
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java
deleted file mode 100644
index 2211a4c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java
+++ /dev/null
@@ -1,55 +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.
- */
-
-package org.apache.ignite.internal.mem;
-
-import java.util.List;
-
-/**
- *
- */
-public class DirectMemory {
-    /** Will be set if  */
-    private boolean restored;
-
-    /** */
-    private List<DirectMemoryRegion> regions;
-
-    /**
-     * @param restored Restored flag.
-     * @param regions Memory fragments.
-     */
-    public DirectMemory(boolean restored, List<DirectMemoryRegion> regions) {
-        this.restored = restored;
-        this.regions = regions;
-    }
-
-    /**
-     * @return Restored flag. If {@code true}, the memory fragments were successfully restored since the previous
-     *      usage and can be reused.
-     */
-    public boolean restored() {
-        return restored;
-    }
-
-    /**
-     * @return Memory fragments.
-     */
-    public List<DirectMemoryRegion> regions() {
-        return regions;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemoryProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemoryProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemoryProvider.java
index 5c73576..a90c6b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemoryProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemoryProvider.java
@@ -18,8 +18,23 @@
 package org.apache.ignite.internal.mem;
 
 /**
- *
+ * Direct memory provider interface. Not thread-safe.
  */
 public interface DirectMemoryProvider {
-    public DirectMemory memory();
+    /**
+     * @param chunkSizes Initializes provider with the chunk sizes.
+     */
+    public void initialize(long[] chunkSizes);
+
+    /**
+     * Shuts down the provider. Will deallocate all previously allocated regions.
+     */
+    public void shutdown();
+
+    /**
+     * Attempts to allocate next memory region. Will return {@code null} if no more regions are available.
+     *
+     * @return Next memory region.
+     */
+    public DirectMemoryRegion nextRegion();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
index d442baf..b0d8d9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
@@ -22,18 +22,17 @@ import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.List;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.mem.DirectMemory;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
-import org.apache.ignite.lifecycle.LifecycleAware;
+import org.apache.ignite.internal.mem.DirectMemoryRegion;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  *
  */
-public class MappedFileMemoryProvider implements DirectMemoryProvider, LifecycleAware {
+public class MappedFileMemoryProvider implements DirectMemoryProvider {
     /** */
     private static final String ALLOCATOR_FILE_PREFIX = "allocator-";
 
@@ -50,33 +49,26 @@ public class MappedFileMemoryProvider implements DirectMemoryProvider, Lifecycle
     /** File allocation path. */
     private final File allocationPath;
 
-    /** Clean flag. If true, existing files will be deleted on start. */
-    private boolean clean;
-
-    /** */
-    private final long[] sizes;
-
     /** */
-    private boolean restored;
+    private long[] sizes;
 
     /** */
     private List<MappedFile> mappedFiles;
 
     /**
      * @param allocationPath Allocation path.
-     * @param clean Clean flag. If true, restore procedure will be ignored even if
-     *      allocation folder contains valid files.
-     * @param sizes Sizes of memory chunks to allocate.
      */
-    public MappedFileMemoryProvider(IgniteLogger log, File allocationPath, boolean clean, long[] sizes) {
+    public MappedFileMemoryProvider(IgniteLogger log, File allocationPath) {
         this.log = log;
         this.allocationPath = allocationPath;
-        this.clean = clean;
-        this.sizes = sizes;
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws IgniteException {
+    @Override public void initialize(long[] sizes) {
+        this.sizes = sizes;
+
+        mappedFiles = new ArrayList<>(sizes.length);
+
         if (!allocationPath.exists()) {
             if (!allocationPath.mkdirs())
                 throw new IgniteException("Failed to initialize allocation path (make sure directory is " +
@@ -88,107 +80,19 @@ public class MappedFileMemoryProvider implements DirectMemoryProvider, Lifecycle
 
         File[] files = allocationPath.listFiles(ALLOCATOR_FILTER);
 
-        Arrays.sort(files, new Comparator<File>() {
-            /** {@inheritDoc} */
-            @Override public int compare(File o1, File o2) {
-                return o1.getName().compareTo(o2.getName());
-            }
-        });
-
-        if (files.length == sizes.length) {
-            for (int i = 0; i < files.length; i++) {
-                File file = files[i];
-
-                if (file.length() != sizes[i]) {
-                    clean = true;
-
-                    break;
-                }
-            }
-        }
-        else
-            clean = true;
-
-        if (files.length == 0 || clean) {
-            if (files.length != 0) {
-                log.info("Will clean up the following files upon start: " + Arrays.asList(files));
-
-                for (File file : files) {
-                    if (!file.delete())
-                        throw new IgniteException("Failed to delete allocated file on start (make sure file is not " +
-                            "opened by another process and current user has enough rights): " + file);
-                }
-            }
-
-            allocateClean();
-
-            return;
-        }
-
-        log.info("Restoring memory state from the files: " + Arrays.asList(files));
+        if (files.length != 0) {
+            log.info("Will clean up the following files upon start: " + Arrays.asList(files));
 
-        mappedFiles = new ArrayList<>(files.length);
-
-        try {
             for (File file : files) {
-                MappedFile mapped = new MappedFile(file, 0);
-
-                mappedFiles.add(mapped);
-            }
-        }
-        catch (IOException e) {
-            // Close all files allocated so far.
-            try {
-                for (MappedFile mapped : mappedFiles)
-                    mapped.close();
-            }
-            catch (IOException e0) {
-                e.addSuppressed(e0);
-            }
-
-            throw new IgniteException(e);
-        }
-
-        restored = true;
-    }
-
-    /**
-     * Allocates clear memory state.
-     */
-    private void allocateClean() {
-        mappedFiles = new ArrayList<>(sizes.length);
-
-        try {
-            int idx = 0;
-
-            for (long size : sizes) {
-                File file = new File(allocationPath, ALLOCATOR_FILE_PREFIX + alignInt(idx));
-
-                MappedFile mappedFile = new MappedFile(file, size);
-
-                mappedFiles.add(mappedFile);
-
-                idx++;
-            }
-        }
-        catch (IOException e) {
-            // Close all files allocated so far.
-            try {
-                for (MappedFile mapped : mappedFiles)
-                    mapped.close();
-            }
-            catch (IOException e0) {
-                e.addSuppressed(e0);
+                if (!file.delete())
+                    throw new IgniteException("Failed to delete allocated file on start (make sure file is not " +
+                        "opened by another process and current user has enough rights): " + file);
             }
-
-            throw new IgniteException(e);
         }
-
-        log.info("Allocated clean memory state at location: " + allocationPath.getAbsolutePath());
     }
 
     /** {@inheritDoc} */
-    @Override public void stop() throws IgniteException {
+    @Override public void shutdown() {
         for (MappedFile file : mappedFiles) {
             try {
                 file.close();
@@ -201,9 +105,28 @@ public class MappedFileMemoryProvider implements DirectMemoryProvider, Lifecycle
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public DirectMemory memory() {
-        return new DirectMemory(restored, (List)mappedFiles);
+    @Override public DirectMemoryRegion nextRegion() {
+        try {
+            if (mappedFiles.size() == sizes.length)
+                return null;
+
+            int idx = mappedFiles.size();
+
+            long chunkSize = sizes[idx];
+
+            File file = new File(allocationPath, ALLOCATOR_FILE_PREFIX + alignInt(idx));
+
+            MappedFile mappedFile = new MappedFile(file, chunkSize);
+
+            mappedFiles.add(mappedFile);
+
+            return mappedFile;
+        }
+        catch (IOException e) {
+            U.error(log, "Failed to allocate next memory-mapped region", e);
+
+            return null;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
index 4cdecde..ef101d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
@@ -20,70 +20,71 @@ package org.apache.ignite.internal.mem.unsafe;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.mem.DirectMemory;
-import org.apache.ignite.internal.mem.DirectMemoryRegion;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
+import org.apache.ignite.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.UnsafeChunk;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.lifecycle.LifecycleAware;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  *
  */
-public class UnsafeMemoryProvider implements DirectMemoryProvider, LifecycleAware {
+public class UnsafeMemoryProvider implements DirectMemoryProvider {
     /** */
-    private final long[] sizes;
+    private long[] sizes;
 
     /** */
     private List<DirectMemoryRegion> regions;
 
+    /** */
+    private IgniteLogger log;
+
     /**
-     * @param sizes Sizes of segments.
+     * @param log Ignite logger to use.
      */
-    public UnsafeMemoryProvider(long[] sizes) {
-        this.sizes = sizes;
+    public UnsafeMemoryProvider(IgniteLogger log) {
+        this.log = log;
     }
 
     /** {@inheritDoc} */
-    @Override public DirectMemory memory() {
-        return new DirectMemory(false, regions);
+    @Override public void initialize(long[] sizes) {
+        this.sizes = sizes;
+
+        regions = new ArrayList<>();
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws IgniteException {
-        regions = new ArrayList<>();
+    @Override public void shutdown() {
+        for (Iterator<DirectMemoryRegion> it = regions.iterator(); it.hasNext(); ) {
+            DirectMemoryRegion chunk = it.next();
 
-        long allocated = 0;
+            GridUnsafe.freeMemory(chunk.address());
 
-        for (long size : sizes) {
-            long ptr = GridUnsafe.allocateMemory(size);
+            // Safety.
+            it.remove();
+        }
+    }
 
-            if (ptr <= 0) {
-                for (DirectMemoryRegion region : regions)
-                    GridUnsafe.freeMemory(region.address());
+    /** {@inheritDoc} */
+    @Override public DirectMemoryRegion nextRegion() {
+        if (regions.size() == sizes.length)
+            return null;
 
-                throw new IgniteException("Failed to allocate memory [allocated=" + allocated +
-                    ", requested=" + size + ']');
-            }
+        long chunkSize = sizes[regions.size()];
 
-            DirectMemoryRegion chunk = new UnsafeChunk(ptr, size);
+        long ptr = GridUnsafe.allocateMemory(chunkSize);
 
-            regions.add(chunk);
+        if (ptr <= 0) {
+            U.error(log, "Failed to allocate next memory chunk: " + U.readableSize(chunkSize, true));
 
-            allocated += size;
+            return null;
         }
-    }
 
-    /** {@inheritDoc} */
-    @Override public void stop() throws IgniteException {
-        for (Iterator<DirectMemoryRegion> it = regions.iterator(); it.hasNext(); ) {
-            DirectMemoryRegion chunk = it.next();
+        DirectMemoryRegion region = new UnsafeChunk(ptr, chunkSize);
 
-            GridUnsafe.freeMemory(chunk.address());
+        regions.add(region);
 
-            // Safety.
-            it.remove();
-        }
+        return region;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index 872e496..1d968c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
@@ -20,12 +20,13 @@ package org.apache.ignite.internal.pagemem.impl;
 import java.io.Closeable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
-import org.apache.ignite.internal.mem.DirectMemory;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.IgniteOutOfMemoryException;
@@ -38,7 +39,6 @@ import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lifecycle.LifecycleAware;
 
 import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
 
@@ -98,6 +98,21 @@ public class PageMemoryNoStoreImpl implements PageMemory {
      */
     public static final int PAGE_OVERHEAD = LOCK_OFFSET + OffheapReadWriteLock.LOCK_SIZE;
 
+    /** Number of bits required to store segment index. */
+    private static final int SEG_BITS = 4;
+
+    /** Number of bits required to store segment index. */
+    private static final int SEG_CNT = (1 << SEG_BITS);
+
+    /** Number of bits left to store page index. */
+    private static final int IDX_BITS = PageIdUtils.PAGE_IDX_SIZE - SEG_BITS;
+
+    /** Segment mask. */
+    private static final int SEG_MASK = ~(-1 << SEG_BITS);
+
+    /** Index mask. */
+    private static final int IDX_MASK = ~(-1 << IDX_BITS);
+
     /** Page size. */
     private int sysPageSize;
 
@@ -113,20 +128,14 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     /** Object to collect memory usage metrics. */
     private final MemoryMetricsImpl memMetrics;
 
-    /** Segments array. */
-    private Segment[] segments;
-
-    /** Number of bits required to store segment index. */
-    private int segBits;
-
-    /** Number of bits left to store page index. */
-    private int idxBits;
-
     /** */
-    private int segMask;
+    private AtomicLong freePageListHead = new AtomicLong(INVALID_REL_PTR);
+
+    /** Segments array. */
+    private volatile Segment[] segments;
 
     /** */
-    private int idxMask;
+    private final AtomicInteger allocatedPages = new AtomicInteger();
 
     /** */
     private AtomicInteger selector = new AtomicInteger();
@@ -135,6 +144,9 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     private OffheapReadWriteLock rwLock;
 
     /** */
+    private final int totalPages;
+
+    /** */
     private final boolean trackAcquiredPages;
 
     /**
@@ -168,35 +180,46 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
         assert sysPageSize % 8 == 0 : sysPageSize;
 
+        totalPages = (int)(memPlcCfg.getMaxSize() / sysPageSize);
+
         // TODO configure concurrency level.
         rwLock = new OffheapReadWriteLock(128);
     }
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteException {
-        if (directMemoryProvider instanceof LifecycleAware)
-            ((LifecycleAware)directMemoryProvider).start();
+        long startSize = memoryPolicyCfg.getInitialSize();
+        long maxSize = memoryPolicyCfg.getMaxSize();
 
-        DirectMemory memory = directMemoryProvider.memory();
+        long[] chunks = new long[SEG_CNT];
 
-        segments = new Segment[memory.regions().size()];
+        chunks[0] = startSize;
 
-        for (int i = 0; i < segments.length; i++) {
-            segments[i] = new Segment(i, memory.regions().get(i));
+        long total = startSize;
 
-            segments[i].init();
-        }
+        long allocChunkSize = Math.max((maxSize - startSize) / (SEG_CNT - 1), 256 * 1024 * 1024);
+
+        int lastIdx = 0;
 
-        segBits = Integer.SIZE - Integer.numberOfLeadingZeros(segments.length - 1);
+        for (int i = 1; i < SEG_CNT; i++) {
+            long allocSize = Math.min(allocChunkSize, maxSize - total);
 
-        // Reserve at least one bit for segment.
-        if (segBits == 0)
-            segBits = 1;
+            if (allocSize <= 0)
+                break;
+
+            chunks[i] = allocSize;
+
+            total += allocSize;
+
+            lastIdx = i;
+        }
+
+        if (lastIdx != SEG_CNT - 1)
+            chunks = Arrays.copyOf(chunks, lastIdx + 1);
 
-        idxBits = PageIdUtils.PAGE_IDX_SIZE - segBits;
+        directMemoryProvider.initialize(chunks);
 
-        segMask = ~(-1 << segBits);
-        idxMask = ~(-1 << idxBits);
+        addSegment(null);
     }
 
     /** {@inheritDoc} */
@@ -205,8 +228,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         if (log.isDebugEnabled())
             log.debug("Stopping page memory.");
 
-        if (directMemoryProvider instanceof LifecycleAware)
-            ((LifecycleAware)directMemoryProvider).stop();
+        directMemoryProvider.shutdown();
 
         if (directMemoryProvider instanceof Closeable) {
             try {
@@ -227,35 +249,34 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     @Override public long allocatePage(int cacheId, int partId, byte flags) {
         memMetrics.incrementTotalAllocatedPages();
 
-        long relPtr = INVALID_REL_PTR;
+        long relPtr = borrowFreePage();
         long absPtr = 0;
 
-        for (Segment seg : segments) {
-            relPtr = seg.borrowFreePage();
+        if (relPtr != INVALID_REL_PTR) {
+            int pageIdx = PageIdUtils.pageIndex(relPtr);
 
-            if (relPtr != INVALID_REL_PTR) {
-                absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
+            Segment seg = segment(pageIdx);
 
-                break;
-            }
+            absPtr = seg.absolute(pageIdx);
         }
 
         // No segments contained a free page.
         if (relPtr == INVALID_REL_PTR) {
-            int segAllocIdx = nextRoundRobinIndex();
-
-            for (int i = 0; i < segments.length; i++) {
-                int idx = (segAllocIdx + i) % segments.length;
-
-                Segment seg = segments[idx];
+            Segment[] seg0 = segments;
+            Segment allocSeg = seg0[seg0.length - 1];
 
-                relPtr = seg.allocateFreePage(flags);
+            while (allocSeg != null) {
+                relPtr = allocSeg.allocateFreePage(flags);
 
                 if (relPtr != INVALID_REL_PTR) {
-                    absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
+                    if (relPtr != INVALID_REL_PTR) {
+                        absPtr = allocSeg.absolute(PageIdUtils.pageIndex(relPtr));
 
-                    break;
+                        break;
+                    }
                 }
+                else
+                    allocSeg = addSegment(seg0);
             }
         }
 
@@ -263,7 +284,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             throw new IgniteOutOfMemoryException("Not enough memory allocated " +
                 "(consider increasing memory policy size or enabling evictions) " +
                 "[policyName=" + memoryPolicyCfg.getName() +
-                ", size=" + U.readableSize(memoryPolicyCfg.getSize(), true) + "]"
+                ", size=" + U.readableSize(memoryPolicyCfg.getMaxSize(), true) + "]"
             );
 
         assert (relPtr & ~PageIdUtils.PAGE_IDX_MASK) == 0;
@@ -281,9 +302,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
     /** {@inheritDoc} */
     @Override public boolean freePage(int cacheId, long pageId) {
-        Segment seg = segment(PageIdUtils.pageIndex(pageId));
-
-        seg.releaseFreePage(pageId);
+        releaseFreePage(pageId);
 
         return true;
     }
@@ -334,6 +353,13 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /**
+     * @return Total number of pages may be allocated for this instance.
+     */
+    public int totalPages() {
+        return totalPages;
+    }
+
+    /**
      * @return Total number of acquired pages.
      */
     public long acquiredPages() {
@@ -382,7 +408,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
      * @return Segment index.
      */
     private int segmentIndex(long pageIdx) {
-        return (int)((pageIdx >> idxBits) & segMask);
+        return (int)((pageIdx >> IDX_BITS) & SEG_MASK);
     }
 
     /**
@@ -393,8 +419,8 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     private long fromSegmentIndex(int segIdx, long pageIdx) {
         long res = 0;
 
-        res = (res << segBits) | (segIdx & segMask);
-        res = (res << idxBits) | (pageIdx & idxMask);
+        res = (res << SEG_BITS) | (segIdx & SEG_MASK);
+        res = (res << IDX_BITS) | (pageIdx & IDX_MASK);
 
         return res;
     }
@@ -428,7 +454,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /** {@inheritDoc} */
-    public long readLockForce(int cacheId, long pageId, long page) {
+    @Override public long readLockForce(int cacheId, long pageId, long page) {
         if (rwLock.readLock(page + LOCK_OFFSET, -1))
             return page + PAGE_OVERHEAD;
 
@@ -457,19 +483,169 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeUnlock(int cacheId, long pageId, long page,
+    @Override public void writeUnlock(
+        int cacheId,
+        long pageId,
+        long page,
         Boolean walPlc,
-        boolean dirtyFlag) {
+        boolean dirtyFlag
+    ) {
         long actualId = PageIO.getPageId(page + PAGE_OVERHEAD);
+
         rwLock.writeUnlock(page + LOCK_OFFSET, PageIdUtils.tag(actualId));
     }
 
+    /** {@inheritDoc} */
     @Override public boolean isDirty(int cacheId, long pageId, long page) {
         // always false for page no store.
         return false;
     }
 
     /**
+     * @param pageIdx Page index.
+     * @return Total page sequence number.
+     */
+    public int pageSequenceNumber(int pageIdx) {
+        Segment seg = segment(pageIdx);
+
+        return seg.sequenceNumber(pageIdx);
+    }
+
+    /**
+     * @param seqNo Page sequence number.
+     * @return Page index.
+     */
+    public int pageIndex(int seqNo) {
+        Segment[] segs = segments;
+
+        int low = 0, high = segs.length - 1;
+
+        while (low <= high) {
+            int mid = (low + high) >>> 1;
+
+            Segment seg = segs[mid];
+
+            int cmp = seg.containsPageBySequence(seqNo);
+
+            if (cmp < 0)
+                high = mid - 1;
+            else if (cmp > 0) {
+                low = mid + 1;
+            }
+            else
+                return seg.pageIndex(seqNo);
+        }
+
+        throw new IgniteException("Allocated page must always be present in one of the segments [seqNo=" + seqNo +
+            ", segments=" + Arrays.toString(segs) + ']');
+    }
+
+    /**
+     * @param pageId Page ID to release.
+     */
+    private void releaseFreePage(long pageId) {
+        int pageIdx = PageIdUtils.pageIndex(pageId);
+
+        // Clear out flags and file ID.
+        long relPtr = PageIdUtils.pageId(0, (byte)0, pageIdx);
+
+        Segment seg = segment(pageIdx);
+
+        long absPtr = seg.absolute(pageIdx);
+
+        // Second, write clean relative pointer instead of page ID.
+        writePageId(absPtr, relPtr);
+
+        // Third, link the free page.
+        while (true) {
+            long freePageRelPtrMasked = freePageListHead.get();
+
+            long freePageRelPtr = freePageRelPtrMasked & RELATIVE_PTR_MASK;
+
+            GridUnsafe.putLong(absPtr, freePageRelPtr);
+
+            if (freePageListHead.compareAndSet(freePageRelPtrMasked, relPtr)) {
+                allocatedPages.decrementAndGet();
+
+                return;
+            }
+        }
+    }
+
+    /**
+     * @return Relative pointer to a free page that was borrowed from the allocated pool.
+     */
+    private long borrowFreePage() {
+        while (true) {
+            long freePageRelPtrMasked = freePageListHead.get();
+
+            long freePageRelPtr = freePageRelPtrMasked & ADDRESS_MASK;
+
+            if (freePageRelPtr != INVALID_REL_PTR) {
+                int pageIdx = PageIdUtils.pageIndex(freePageRelPtr);
+
+                Segment seg = segment(pageIdx);
+
+                long freePageAbsPtr = seg.absolute(pageIdx);
+                long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
+                long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
+
+                if (freePageListHead.compareAndSet(freePageRelPtrMasked, nextFreePageRelPtr | cnt)) {
+                    GridUnsafe.putLong(freePageAbsPtr, PAGE_MARKER);
+
+                    allocatedPages.incrementAndGet();
+
+                    return freePageRelPtr;
+                }
+            }
+            else
+                return INVALID_REL_PTR;
+        }
+    }
+
+    /**
+     * Attempts to add a new memory segment.
+     *
+     * @param oldRef Old segments array. If this method observes another segments array, it will allocate a new
+     *      segment (if possible). If the array has already been updated, it will return the last element in the
+     *      new array.
+     * @return Added segment, if successfull, {@code null} if failed to add.
+     */
+    private synchronized Segment addSegment(Segment[] oldRef) {
+        if (segments == oldRef) {
+            DirectMemoryRegion region = directMemoryProvider.nextRegion();
+
+            // No more memory is available.
+            if (region == null)
+                return null;
+
+            if (oldRef != null) {
+                if (log.isInfoEnabled())
+                    log.info("Allocated next memory segment [plcName=" + memoryPolicyCfg.getName() +
+                        ", chunkSize=" + U.readableSize(region.size(), true) + ']');
+            }
+
+            Segment[] newRef = new Segment[oldRef == null ? 1 : oldRef.length + 1];
+
+            if (oldRef != null)
+                System.arraycopy(oldRef, 0, newRef, 0, oldRef.length);
+
+            Segment lastSeg = oldRef == null ? null : oldRef[oldRef.length - 1];
+
+            Segment allocated = new Segment(newRef.length - 1, region, lastSeg == null ? 0 : lastSeg.sumPages());
+
+            allocated.init();
+
+            newRef[newRef.length - 1] = allocated;
+
+            segments = newRef;
+        }
+
+        // Only this synchronized method writes to segments, so it is safe to read twice.
+        return segments[segments.length - 1];
+    }
+
+    /**
      *
      */
     private class Segment extends ReentrantReadWriteLock {
@@ -482,9 +658,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         /** Direct memory chunk. */
         private DirectMemoryRegion region;
 
-        /** Pointer to the address of the free page list. */
-        private long freePageListPtr;
-
         /** Last allocated page index. */
         private long lastAllocatedIdxPtr;
 
@@ -492,7 +665,10 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         private long pagesBase;
 
         /** */
-        private final AtomicInteger allocatedPages;
+        private int pagesInPrevSegments;
+
+        /** */
+        private int maxPages;
 
         /** */
         private final AtomicInteger acquiredPages;
@@ -500,12 +676,13 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         /**
          * @param idx Index.
          * @param region Memory region to use.
+         * @param pagesInPrevSegments Number of pages in previously allocated segments.
          */
-        private Segment(int idx, DirectMemoryRegion region) {
+        private Segment(int idx, DirectMemoryRegion region, int pagesInPrevSegments) {
             this.idx = idx;
             this.region = region;
+            this.pagesInPrevSegments = pagesInPrevSegments;
 
-            allocatedPages = new AtomicInteger();
             acquiredPages = new AtomicInteger();
         }
 
@@ -515,10 +692,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         private void init() {
             long base = region.address();
 
-            freePageListPtr = base;
-
-            base += 8;
-
             lastAllocatedIdxPtr = base;
 
             base += 8;
@@ -526,8 +699,11 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             // Align by 8 bytes.
             pagesBase = (base + 7) & ~0x7;
 
-            GridUnsafe.putLong(freePageListPtr, INVALID_REL_PTR);
             GridUnsafe.putLong(lastAllocatedIdxPtr, 0);
+
+            long limit = region.address() + region.size();
+
+            maxPages = (int)((limit - pagesBase) / sysPageSize);
         }
 
         /**
@@ -556,7 +732,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
          * @return Absolute pointer.
          */
         private long absolute(int pageIdx) {
-            pageIdx &= idxMask;
+            pageIdx &= IDX_MASK;
 
             long off = ((long)pageIdx) * sysPageSize;
 
@@ -564,75 +740,34 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         }
 
         /**
-         * @return Total number of loaded pages for the segment.
+         * @param pageIdx Page index with encoded segment.
+         * @return Absolute page sequence number.
          */
-        private int allocatedPages() {
-            return allocatedPages.get();
+        private int sequenceNumber(int pageIdx) {
+            pageIdx &= IDX_MASK;
+
+            return pagesInPrevSegments + pageIdx;
         }
 
         /**
-         * @return Total number of currently acquired pages.
+         * @return Page sequence number upper bound.
          */
-        private int acquiredPages() {
-            return acquiredPages.get();
+        private int sumPages() {
+            return pagesInPrevSegments + maxPages;
         }
 
         /**
-         * @param pageId Page ID to release.
+         * @return Total number of loaded pages for the segment.
          */
-        private void releaseFreePage(long pageId) {
-            int pageIdx = PageIdUtils.pageIndex(pageId);
-
-            // Clear out flags and file ID.
-            long relPtr = PageIdUtils.pageId(0, (byte)0, pageIdx);
-
-            long absPtr = absolute(pageIdx);
-
-            // Second, write clean relative pointer instead of page ID.
-            writePageId(absPtr, relPtr);
-
-            // Third, link the free page.
-            while (true) {
-                long freePageRelPtrMasked = GridUnsafe.getLong(freePageListPtr);
-
-                long freePageRelPtr = freePageRelPtrMasked & RELATIVE_PTR_MASK;
-
-                GridUnsafe.putLong(absPtr, freePageRelPtr);
-
-                if (GridUnsafe.compareAndSwapLong(null, freePageListPtr, freePageRelPtrMasked, relPtr)) {
-                    allocatedPages.decrementAndGet();
-
-                    return;
-                }
-            }
+        private int allocatedPages() {
+            return allocatedPages.get();
         }
 
         /**
-         * @return Relative pointer to a free page that was borrowed from the allocated pool.
+         * @return Total number of currently acquired pages.
          */
-        private long borrowFreePage() {
-            while (true) {
-                long freePageRelPtrMasked = GridUnsafe.getLong(freePageListPtr);
-
-                long freePageRelPtr = freePageRelPtrMasked & ADDRESS_MASK;
-                long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
-
-                if (freePageRelPtr != INVALID_REL_PTR) {
-                    long freePageAbsPtr = absolute(PageIdUtils.pageIndex(freePageRelPtr));
-
-                    long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
-
-                    if (GridUnsafe.compareAndSwapLong(null, freePageListPtr, freePageRelPtrMasked, nextFreePageRelPtr | cnt)) {
-                        GridUnsafe.putLong(freePageAbsPtr, PAGE_MARKER);
-
-                        allocatedPages.incrementAndGet();
-
-                        return freePageRelPtr;
-                    }
-                }
-                else
-                    return INVALID_REL_PTR;
-            }
+        private int acquiredPages() {
+            return acquiredPages.get();
         }
 
         /**
@@ -644,7 +779,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             long limit = region.address() + region.size();
 
             while (true) {
-                long lastIdx = GridUnsafe.getLong(lastAllocatedIdxPtr);
+                long lastIdx = GridUnsafe.getLongVolatile(null, lastAllocatedIdxPtr);
 
                 // Check if we have enough space to allocate a page.
                 if (pagesBase + (lastIdx + 1) * sysPageSize > limit)
@@ -671,5 +806,28 @@ public class PageMemoryNoStoreImpl implements PageMemory {
                 }
             }
         }
+
+        /**
+         * @param seqNo Page sequence number.
+         * @return {@code 0} if this segment contains the page with the given sequence number,
+         *      {@code -1} if one of the previous segments contains the page with the given sequence number,
+         *      {@code 1} if one of the next segments contains the page with the given sequence number.
+         */
+        public int containsPageBySequence(int seqNo) {
+            if (seqNo < pagesInPrevSegments)
+                return -1;
+            else if (seqNo < pagesInPrevSegments + maxPages)
+                return 0;
+            else
+                return 1;
+        }
+
+        /**
+         * @param seqNo Page sequence number.
+         * @return Page index
+         */
+        public int pageIndex(int seqNo) {
+            return PageIdUtils.pageIndex(fromSegmentIndex(idx, seqNo - pagesInPrevSegments));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java
index c633fde..deaaac0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java
@@ -104,11 +104,6 @@ class CacheClusterMetricsMXBeanImpl implements CacheMetricsMXBean {
     }
 
     /** {@inheritDoc} */
-    @Override public long getOffHeapMaxSize() {
-        return cache.clusterMetrics().getOffHeapMaxSize();
-    }
-
-    /** {@inheritDoc} */
     @Override public int getSize() {
         return cache.clusterMetrics().getSize();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java
index cdab58a..0fb7568 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java
@@ -104,11 +104,6 @@ class CacheLocalMetricsMXBeanImpl implements CacheMetricsMXBean {
     }
 
     /** {@inheritDoc} */
-    @Override public long getOffHeapMaxSize() {
-        return cache.metrics0().getOffHeapMaxSize();
-    }
-
-    /** {@inheritDoc} */
     @Override public int getSize() {
         return cache.metrics0().getSize();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 1d03e64..d4ad8e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -220,11 +220,6 @@ public class CacheMetricsImpl implements CacheMetrics {
     }
 
     /** {@inheritDoc} */
-    @Override public long getOffHeapMaxSize() {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
     @Override public int getSize() {
         GridCacheAdapter<?, ?> cache = cctx.cache();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
index a8d7693..67f646a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
@@ -104,9 +104,6 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     /** Memory size allocated in off-heap. */
     private long offHeapAllocatedSize;
 
-    /** Off-heap memory maximum size*/
-    private long offHeapMaxSize;
-
     /** Number of non-{@code null} values in the cache. */
     private int size;
 
@@ -249,7 +246,6 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         offHeapPrimaryEntriesCnt = m.getOffHeapPrimaryEntriesCount();
         offHeapBackupEntriesCnt = m.getOffHeapBackupEntriesCount();
         offHeapAllocatedSize = m.getOffHeapAllocatedSize();
-        offHeapMaxSize = m.getOffHeapMaxSize();
 
         size = m.getSize();
         keySize = m.getKeySize();
@@ -314,8 +310,6 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         isReadThrough = loc.isReadThrough();
         isWriteThrough = loc.isWriteThrough();
 
-        offHeapMaxSize = loc.getOffHeapMaxSize();
-
         for (CacheMetrics e : metrics) {
             reads += e.getCacheGets();
             puts += e.getCachePuts();
@@ -570,11 +564,6 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public long getOffHeapMaxSize() {
-        return offHeapMaxSize;
-    }
-
-    /** {@inheritDoc} */
     @Override public int getSize() {
         return size;
     }
@@ -776,7 +765,6 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         out.writeLong(offHeapPrimaryEntriesCnt);
         out.writeLong(offHeapBackupEntriesCnt);
         out.writeLong(offHeapAllocatedSize);
-        out.writeLong(offHeapMaxSize);
 
         out.writeInt(dhtEvictQueueCurrSize);
         out.writeInt(txThreadMapSize);
@@ -825,7 +813,6 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         offHeapPrimaryEntriesCnt = in.readLong();
         offHeapBackupEntriesCnt = in.readLong();
         offHeapAllocatedSize = in.readLong();
-        offHeapMaxSize = in.readLong();
 
         dhtEvictQueueCurrSize = in.readInt();
         txThreadMapSize = in.readInt();


[2/3] ignite git commit: IGNITE-5072 - Updated memory metrics to comply with other metrics

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
index ae594fa..5062d0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.processors.cache.database;
 
 import java.io.File;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
@@ -28,8 +30,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
@@ -38,8 +40,8 @@ import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.pagemem.snapshot.StartFullSnapshotAckDiscoveryMessage;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
+import org.apache.ignite.internal.pagemem.snapshot.StartFullSnapshotAckDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
@@ -52,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.database.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.database.freelist.FreeListImpl;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.mxbean.MemoryMetricsMXBean;
@@ -67,7 +70,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     static final String SYSTEM_MEMORY_POLICY_NAME = "sysMemPlc";
 
     /** Minimum size of memory chunk */
-    private static final long MIN_PAGE_MEMORY_SIZE = 1024 * 1024;
+    private static final long MIN_PAGE_MEMORY_SIZE = 10 * 1024 * 1024;
 
     /** */
     protected Map<String, MemoryPolicy> memPlcMap;
@@ -125,25 +128,32 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     private void registerMetricsMBeans() {
         IgniteConfiguration cfg = cctx.gridConfig();
 
-        for (MemoryMetrics memMetrics : memMetricsMap.values())
-            registerMetricsMBean((MemoryMetricsImpl) memMetrics, cfg);
+        for (MemoryMetrics memMetrics : memMetricsMap.values()) {
+            MemoryPolicyConfiguration memPlcCfg = memPlcMap.get(memMetrics.getName()).config();
+
+            registerMetricsMBean((MemoryMetricsImpl)memMetrics, memPlcCfg, cfg);
+        }
     }
 
     /**
      * @param memMetrics Memory metrics.
+     * @param memPlcCfg Memory policy configuration.
+     * @param cfg Ignite configuration.
      */
-    private void registerMetricsMBean(MemoryMetricsImpl memMetrics, IgniteConfiguration cfg) {
+    private void registerMetricsMBean(MemoryMetricsImpl memMetrics,
+        MemoryPolicyConfiguration memPlcCfg,
+        IgniteConfiguration cfg) {
         try {
             U.registerMBean(
                     cfg.getMBeanServer(),
                     cfg.getIgniteInstanceName(),
                     "MemoryMetrics",
-                    memMetrics.getName(),
-                    memMetrics,
+                    memPlcCfg.getName(),
+                    new MemoryMetricsMXBeanImpl(memMetrics, memPlcCfg),
                     MemoryMetricsMXBean.class);
         }
         catch (JMException e) {
-            log.warning("Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'");
+            U.error(log, "Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'", e);
         }
     }
 
@@ -196,35 +206,35 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param dbCfg Database config.
+     * @param memCfg Database config.
      */
-    protected void initPageMemoryPolicies(MemoryConfiguration dbCfg) {
-        MemoryPolicyConfiguration[] memPlcsCfgs = dbCfg.getMemoryPolicies();
+    protected void initPageMemoryPolicies(MemoryConfiguration memCfg) {
+        MemoryPolicyConfiguration[] memPlcsCfgs = memCfg.getMemoryPolicies();
 
         if (memPlcsCfgs == null) {
             //reserve place for default and system memory policies
             memPlcMap = U.newHashMap(2);
             memMetricsMap = U.newHashMap(2);
 
-            addMemoryPolicy(dbCfg,
-                    dbCfg.createDefaultPolicyConfig(),
-                    DFLT_MEM_PLC_DEFAULT_NAME);
+            addMemoryPolicy(memCfg,
+                memCfg.createDefaultPolicyConfig(),
+                DFLT_MEM_PLC_DEFAULT_NAME);
 
-            log.warning("No user-defined default MemoryPolicy found; system default of 1GB size will be used.");
+            U.warn(log, "No user-defined default MemoryPolicy found; system default of 1GB size will be used.");
         }
         else {
-            String dfltMemPlcName = dbCfg.getDefaultMemoryPolicyName();
+            String dfltMemPlcName = memCfg.getDefaultMemoryPolicyName();
 
             if (DFLT_MEM_PLC_DEFAULT_NAME.equals(dfltMemPlcName) && !hasCustomDefaultMemoryPolicy(memPlcsCfgs)) {
                 //reserve additional place for default and system memory policies
                 memPlcMap = U.newHashMap(memPlcsCfgs.length + 2);
                 memMetricsMap = U.newHashMap(memPlcsCfgs.length + 2);
 
-                addMemoryPolicy(dbCfg,
-                        dbCfg.createDefaultPolicyConfig(),
-                        DFLT_MEM_PLC_DEFAULT_NAME);
+                addMemoryPolicy(memCfg,
+                    memCfg.createDefaultPolicyConfig(),
+                    DFLT_MEM_PLC_DEFAULT_NAME);
 
-                log.warning("No user-defined default MemoryPolicy found; system default will be allocated.");
+                U.warn(log, "No user-defined default MemoryPolicy found; system default of 1GB size will be used.");
             }
             else {
                 //reserve additional space for system memory policy only
@@ -233,12 +243,12 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
             }
 
             for (MemoryPolicyConfiguration memPlcCfg : memPlcsCfgs)
-                addMemoryPolicy(dbCfg, memPlcCfg, memPlcCfg.getName());
+                addMemoryPolicy(memCfg, memPlcCfg, memPlcCfg.getName());
         }
 
-        addMemoryPolicy(dbCfg,
-                createSystemMemoryPolicy(dbCfg.getSystemCacheMemorySize()),
-                SYSTEM_MEMORY_POLICY_NAME);
+        addMemoryPolicy(memCfg,
+            createSystemMemoryPolicy(memCfg.getSystemCacheInitialSize(), memCfg.getSystemCacheMaxSize()),
+            SYSTEM_MEMORY_POLICY_NAME);
     }
 
     /**
@@ -265,7 +275,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         if (memPlcName.equals(dfltMemPlcName))
             dfltMemPlc = memPlc;
         else if (memPlcName.equals(DFLT_MEM_PLC_DEFAULT_NAME))
-            log.warning("Memory Policy with name 'default' isn't used as a default. " +
+            U.warn(log, "Memory Policy with name 'default' isn't used as a default. " +
                     "Please check Memory Policies configuration.");
     }
 
@@ -291,25 +301,32 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param sysCacheMemSize size of PageMemory to be created for system cache.
+     * @param sysCacheInitSize Initial size of PageMemory to be created for system cache.
+     * @param sysCacheMaxSize Maximum size of PageMemory to be created for system cache.
+     *
+     * @return {@link MemoryPolicyConfiguration configuration} of MemoryPolicy for system cache.
      */
-    private MemoryPolicyConfiguration createSystemMemoryPolicy(long sysCacheMemSize) {
+    private MemoryPolicyConfiguration createSystemMemoryPolicy(long sysCacheInitSize, long sysCacheMaxSize) {
         MemoryPolicyConfiguration res = new MemoryPolicyConfiguration();
 
         res.setName(SYSTEM_MEMORY_POLICY_NAME);
-        res.setSize(sysCacheMemSize);
+        res.setInitialSize(sysCacheInitSize);
+        res.setMaxSize(sysCacheMaxSize);
 
         return res;
     }
 
     /**
-     * @param dbCfg configuration to validate.
+     * @param memCfg configuration to validate.
      */
-    private void validateConfiguration(MemoryConfiguration dbCfg) throws IgniteCheckedException {
-        MemoryPolicyConfiguration[] plcCfgs = dbCfg.getMemoryPolicies();
+    private void validateConfiguration(MemoryConfiguration memCfg) throws IgniteCheckedException {
+        MemoryPolicyConfiguration[] plcCfgs = memCfg.getMemoryPolicies();
 
         Set<String> plcNames = (plcCfgs != null) ? U.<String>newHashSet(plcCfgs.length) : new HashSet<String>(0);
 
+        checkSystemMemoryPolicySizeConfiguration(memCfg.getSystemCacheInitialSize(),
+            memCfg.getSystemCacheMaxSize());
+
         if (plcCfgs != null) {
             for (MemoryPolicyConfiguration plcCfg : plcCfgs) {
                 assert plcCfg != null;
@@ -318,24 +335,68 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
                 checkPolicySize(plcCfg);
 
-                checkPolicyEvictionProperties(plcCfg, dbCfg);
+                checkPolicyEvictionProperties(plcCfg, memCfg);
             }
         }
 
-        checkDefaultPolicyConfiguration(dbCfg.getDefaultMemoryPolicyName(), plcNames);
+        checkDefaultPolicyConfiguration(
+                memCfg.getDefaultMemoryPolicyName(),
+                memCfg.getDefaultMemoryPolicySize(),
+                plcNames);
+    }
+
+    /**
+     * @param sysCacheInitSize System cache initial size.
+     * @param sysCacheMaxSize System cache max size.
+     *
+     * @throws IgniteCheckedException In case of validation violation.
+     */
+    private void checkSystemMemoryPolicySizeConfiguration(long sysCacheInitSize, long sysCacheMaxSize) throws IgniteCheckedException {
+        if (sysCacheInitSize < MIN_PAGE_MEMORY_SIZE)
+            throw new IgniteCheckedException("Initial size for system cache must have size more than 10MB (use " +
+                "MemoryConfiguration.systemCacheInitialSize property to set correct size in bytes); " +
+                "size: " + U.readableSize(sysCacheInitSize, true)
+            );
+
+        if (sysCacheMaxSize < sysCacheInitSize)
+            throw new IgniteCheckedException("MaxSize of system cache must not be smaller than " +
+                "initialSize [initSize=" + U.readableSize(sysCacheInitSize, true) +
+                ", maxSize=" + U.readableSize(sysCacheMaxSize, true) + "]. " +
+                "Use MemoryConfiguration.systemCacheInitialSize/MemoryConfiguration.systemCacheMaxSize " +
+                "properties to set correct sizes in bytes."
+            );
     }
 
     /**
      * @param dfltPlcName Default MemoryPolicy name.
+     * @param dfltPlcSize Default size of MemoryPolicy overridden by user (equals to -1 if wasn't specified by user).
      * @param plcNames All MemoryPolicy names.
      * @throws IgniteCheckedException In case of validation violation.
      */
-    private static void checkDefaultPolicyConfiguration(String dfltPlcName, Set<String> plcNames) throws IgniteCheckedException {
+    private static void checkDefaultPolicyConfiguration(
+        String dfltPlcName,
+        long dfltPlcSize,
+        Collection<String> plcNames
+    ) throws IgniteCheckedException {
+        if (dfltPlcSize != -1) {
+            if (!F.eq(dfltPlcName, MemoryConfiguration.DFLT_MEM_PLC_DEFAULT_NAME))
+                throw new IgniteCheckedException("User-defined MemoryPolicy configuration " +
+                    "and defaultMemoryPolicySize properties are set at the same time. " +
+                    "Delete either MemoryConfiguration.defaultMemoryPolicySize property " +
+                    "or user-defined default MemoryPolicy configuration");
+
+            if (dfltPlcSize < MIN_PAGE_MEMORY_SIZE)
+                throw new IgniteCheckedException("User-defined default MemoryPolicy size is less than 1MB. " +
+                        "Use MemoryConfiguration.defaultMemoryPolicySize property to set correct size.");
+        }
+
         if (!DFLT_MEM_PLC_DEFAULT_NAME.equals(dfltPlcName)) {
             if (dfltPlcName.isEmpty())
                 throw new IgniteCheckedException("User-defined default MemoryPolicy name must be non-empty");
+
             if (!plcNames.contains(dfltPlcName))
-                throw new IgniteCheckedException("User-defined default MemoryPolicy name must be presented among configured MemoryPolices: " + dfltPlcName);
+                throw new IgniteCheckedException("User-defined default MemoryPolicy name " +
+                    "must be presented among configured MemoryPolices: " + dfltPlcName);
         }
     }
 
@@ -344,8 +405,17 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @throws IgniteCheckedException If config is invalid.
      */
     private static void checkPolicySize(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
-        if (plcCfg.getSize() < MIN_PAGE_MEMORY_SIZE)
-            throw new IgniteCheckedException("MemoryPolicy must have size more than 1MB: " + plcCfg.getName());
+        if (plcCfg.getInitialSize() < MIN_PAGE_MEMORY_SIZE)
+            throw new IgniteCheckedException("MemoryPolicy must have size more than 10MB (use " +
+                "MemoryPolicyConfiguration.initialSize property to set correct size in bytes) " +
+                "[name=" + plcCfg.getName() + ", size=" + U.readableSize(plcCfg.getInitialSize(), true) + "]"
+            );
+
+        if (plcCfg.getMaxSize() < plcCfg.getInitialSize())
+            throw new IgniteCheckedException("MemoryPolicy maxSize must not be smaller than " +
+                "initialSize [name=" + plcCfg.getName() +
+                ", initSize=" + U.readableSize(plcCfg.getInitialSize(), true) +
+                ", maxSize=" + U.readableSize(plcCfg.getMaxSize(), true) + ']');
     }
 
     /**
@@ -366,7 +436,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         if (plcCfg.getEmptyPagesPoolSize() <= 10)
             throw new IgniteCheckedException("Evicted pages pool size should be greater than 10: " + plcCfg.getName());
 
-        long maxPoolSize = plcCfg.getSize() / dbCfg.getPageSize() / 10;
+        long maxPoolSize = plcCfg.getMaxSize() / dbCfg.getPageSize() / 10;
 
         if (plcCfg.getEmptyPagesPoolSize() >= maxPoolSize) {
             throw new IgniteCheckedException("Evicted pages pool size should be lesser than " + maxPoolSize +
@@ -379,9 +449,11 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param observedNames Names of MemoryPolicies observed before.
      * @throws IgniteCheckedException If config is invalid.
      */
-    private static void checkPolicyName(String plcName, Set<String> observedNames) throws IgniteCheckedException {
+    private static void checkPolicyName(String plcName, Collection<String> observedNames)
+        throws IgniteCheckedException {
         if (plcName == null || plcName.isEmpty())
-            throw new IgniteCheckedException("User-defined MemoryPolicyConfiguration must have non-null and non-empty name.");
+            throw new IgniteCheckedException("User-defined MemoryPolicyConfiguration must have non-null and " +
+                "non-empty name.");
 
         if (observedNames.contains(plcName))
             throw new IgniteCheckedException("Two MemoryPolicies have the same name: " + plcName);
@@ -420,7 +492,17 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @return MemoryMetrics for all MemoryPolicies configured in Ignite instance.
      */
     public Collection<MemoryMetrics> memoryMetrics() {
-        return memMetricsMap != null ? memMetricsMap.values() : null;
+        if (!F.isEmpty(memMetricsMap)) {
+            // Intentionally return a collection copy to make it explicitly serializable.
+            Collection<MemoryMetrics> res = new ArrayList<>(memMetricsMap.size());
+
+            for (MemoryMetrics metrics : memMetricsMap.values())
+                res.add(new MemoryMetricsSnapshot(metrics));
+
+            return res;
+        }
+        else
+            return Collections.emptyList();
     }
 
     /**
@@ -472,6 +554,20 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 memPlc.pageMemory().stop();
 
                 memPlc.evictionTracker().stop();
+
+                IgniteConfiguration cfg = cctx.gridConfig();
+
+                try {
+                    cfg.getMBeanServer().unregisterMBean(
+                        U.makeMBeanName(
+                            cfg.getIgniteInstanceName(),
+                            "MemoryMetrics",
+                            memPlc.memoryMetrics().getName()));
+                }
+                catch (JMException e) {
+                    U.error(log, "Failed to unregister MBean for memory metrics: " +
+                        memPlc.memoryMetrics().getName(), e);
+                }
             }
         }
     }
@@ -590,7 +686,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         if (plcCfg.getPageEvictionMode() == DataPageEvictionMode.DISABLED)
             return;
 
-        long memorySize = plcCfg.getSize();
+        long memorySize = plcCfg.getMaxSize();
 
         PageMemory pageMem = memPlc.pageMemory();
 
@@ -614,36 +710,31 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param dbCfg memory configuration with common parameters.
-     * @param plc memory policy with PageMemory specific parameters.
+     * @param memCfg memory configuration with common parameters.
+     * @param plcCfg memory policy with PageMemory specific parameters.
      * @param memMetrics {@link MemoryMetrics} object to collect memory usage metrics.
      * @return Memory policy instance.
      */
-    private MemoryPolicy initMemory(MemoryConfiguration dbCfg, MemoryPolicyConfiguration plc, MemoryMetricsImpl memMetrics) {
-        long[] sizes = calculateFragmentSizes(
-                dbCfg.getConcurrencyLevel(),
-                plc.getSize());
-
-        File allocPath = buildAllocPath(plc);
+    private MemoryPolicy initMemory(MemoryConfiguration memCfg, MemoryPolicyConfiguration plcCfg, MemoryMetricsImpl memMetrics) {
+        File allocPath = buildAllocPath(plcCfg);
 
         DirectMemoryProvider memProvider = allocPath == null ?
-            new UnsafeMemoryProvider(sizes) :
+            new UnsafeMemoryProvider(log) :
             new MappedFileMemoryProvider(
                 log,
-                allocPath,
-                true,
-                sizes);
+                allocPath);
 
-        PageMemory pageMem = createPageMemory(memProvider, dbCfg.getPageSize(), plc, memMetrics);
+        PageMemory pageMem = createPageMemory(memProvider, memCfg, plcCfg, memMetrics);
 
-        return new MemoryPolicy(pageMem, plc, memMetrics, createPageEvictionTracker(plc, pageMem));
+        return new MemoryPolicy(pageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg,
+            (PageMemoryNoStoreImpl)pageMem));
     }
 
     /**
      * @param plc Memory Policy Configuration.
      * @param pageMem Page memory.
      */
-    private PageEvictionTracker createPageEvictionTracker(MemoryPolicyConfiguration plc, PageMemory pageMem) {
+    private PageEvictionTracker createPageEvictionTracker(MemoryPolicyConfiguration plc, PageMemoryNoStoreImpl pageMem) {
         if (Boolean.getBoolean("override.fair.fifo.page.eviction.tracker"))
             return new FairFifoPageEvictionTracker(pageMem, plc, cctx);
 
@@ -658,28 +749,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * Calculate fragment sizes for a cache with given size and concurrency level.
-     * @param concLvl Concurrency level.
-     * @param cacheSize Cache size.
-     */
-    protected long[] calculateFragmentSizes(int concLvl, long cacheSize) {
-        if (concLvl < 1)
-            concLvl = Runtime.getRuntime().availableProcessors();
-
-        long fragmentSize = cacheSize / concLvl;
-
-        if (fragmentSize < 1024 * 1024)
-            fragmentSize = 1024 * 1024;
-
-        long[] sizes = new long[concLvl];
-
-        for (int i = 0; i < concLvl; i++)
-            sizes[i] = fragmentSize;
-
-        return sizes;
-    }
-
-    /**
      * Builds allocation path for memory mapped file to be used with PageMemory.
      *
      * @param plc MemoryPolicyConfiguration.
@@ -701,18 +770,18 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * Creates PageMemory with given size and memory provider.
      *
      * @param memProvider Memory provider.
-     * @param pageSize Page size.
+     * @param memCfg Memory configuartion.
      * @param memPlcCfg Memory policy configuration.
      * @param memMetrics MemoryMetrics to collect memory usage metrics.
      * @return PageMemory instance.
      */
     protected PageMemory createPageMemory(
         DirectMemoryProvider memProvider,
-        int pageSize,
+        MemoryConfiguration memCfg,
         MemoryPolicyConfiguration memPlcCfg,
         MemoryMetricsImpl memMetrics
     ) {
-        return new PageMemoryNoStoreImpl(log, memProvider, cctx, pageSize, memPlcCfg, memMetrics, false);
+        return new PageMemoryNoStoreImpl(log, memProvider, cctx, memCfg.getPageSize(), memPlcCfg, memMetrics, false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
index ed4cae0..4a7e951 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
@@ -18,17 +18,17 @@ package org.apache.ignite.internal.processors.cache.database;
 
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.processors.cache.database.freelist.FreeListImpl;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.mxbean.MemoryMetricsMXBean;
 import org.jsr166.LongAdder8;
 
 /**
  *
  */
-public class MemoryMetricsImpl implements MemoryMetricsMXBean {
+public class MemoryMetricsImpl implements MemoryMetrics {
     /** */
     private FreeListImpl freeList;
 
@@ -68,6 +68,8 @@ public class MemoryMetricsImpl implements MemoryMetricsMXBean {
     public MemoryMetricsImpl(MemoryPolicyConfiguration memPlcCfg) {
         this.memPlcCfg = memPlcCfg;
 
+        metricsEnabled = memPlcCfg.isMetricsEnabled();
+
         for (int i = 0; i < subInts; i++)
             allocRateCounters[i] = new LongAdder8();
     }
@@ -78,16 +80,6 @@ public class MemoryMetricsImpl implements MemoryMetricsMXBean {
     }
 
     /** {@inheritDoc} */
-    @Override public int getSize() {
-        return (int) (memPlcCfg.getSize() / (1024 * 1024));
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSwapFilePath() {
-        return memPlcCfg.getSwapFilePath();
-    }
-
-    /** {@inheritDoc} */
     @Override public long getTotalAllocatedPages() {
         return metricsEnabled ? totalAllocatedPages.longValue() : 0;
     }
@@ -264,19 +256,19 @@ public class MemoryMetricsImpl implements MemoryMetricsMXBean {
     }
 
     /** {@inheritDoc} */
-    @Override public void enableMetrics() {
+    public void enableMetrics() {
         metricsEnabled = true;
     }
 
     /** {@inheritDoc} */
-    @Override public void disableMetrics() {
+    public void disableMetrics() {
         metricsEnabled = false;
     }
 
     /**
      * @param rateTimeInterval Time interval used to calculate allocation/eviction rate.
      */
-    @Override public void rateTimeInterval(int rateTimeInterval) {
+    public void rateTimeInterval(int rateTimeInterval) {
         this.rateTimeInterval = rateTimeInterval;
     }
 
@@ -285,7 +277,7 @@ public class MemoryMetricsImpl implements MemoryMetricsMXBean {
      *
      * @param subInts Number of subintervals.
      */
-    @Override public void subIntervals(int subInts) {
+    public void subIntervals(int subInts) {
         assert subInts > 0;
 
         if (this.subInts == subInts)

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsMXBeanImpl.java
new file mode 100644
index 0000000..05c6677
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsMXBeanImpl.java
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+package org.apache.ignite.internal.processors.cache.database;
+
+import org.apache.ignite.MemoryMetrics;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.mxbean.MemoryMetricsMXBean;
+
+/**
+ * MBean to expose {@link MemoryMetrics} through JMX interface.
+ */
+class MemoryMetricsMXBeanImpl implements MemoryMetricsMXBean {
+    /** */
+    private final MemoryMetricsImpl memMetrics;
+
+    /** */
+    private final MemoryPolicyConfiguration memPlcCfg;
+
+    /**
+     * @param memMetrics MemoryMetrics instance to expose through JMX interface.
+     * @param memPlcCfg configuration of memory policy this MX Bean is created for.
+     */
+    MemoryMetricsMXBeanImpl(MemoryMetricsImpl memMetrics,
+        MemoryPolicyConfiguration memPlcCfg
+    ) {
+        this.memMetrics = memMetrics;
+        this.memPlcCfg = memPlcCfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getAllocationRate() {
+        return memMetrics.getAllocationRate();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getEvictionRate() {
+        return memMetrics.getEvictionRate();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getLargeEntriesPagesPercentage() {
+        return memMetrics.getLargeEntriesPagesPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getPagesFillFactor() {
+        return memMetrics.getPagesFillFactor();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getTotalAllocatedPages() {
+        return memMetrics.getTotalAllocatedPages();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rateTimeInterval(int rateTimeInterval) {
+        memMetrics.rateTimeInterval(rateTimeInterval);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void subIntervals(int subInts) {
+        memMetrics.subIntervals(subInts);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void enableMetrics() {
+        memMetrics.enableMetrics();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disableMetrics() {
+        memMetrics.disableMetrics();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getName() {
+        return memMetrics.getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getInitialSize() {
+        return (int) (memPlcCfg.getInitialSize() / (1024 * 1024));
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxSize() {
+        return (int) (memPlcCfg.getMaxSize() / (1024 * 1024));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSwapFilePath() {
+        return memPlcCfg.getSwapFilePath();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsSnapshot.java
new file mode 100644
index 0000000..5f337bd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsSnapshot.java
@@ -0,0 +1,85 @@
+/*
+ * 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.
+ */
+
+package org.apache.ignite.internal.processors.cache.database;
+
+import org.apache.ignite.MemoryMetrics;
+
+/**
+ *
+ */
+public class MemoryMetricsSnapshot implements MemoryMetrics {
+    /** */
+    private String name;
+
+    /** */
+    private long totalAllocatedPages;
+
+    /** */
+    private float allocationRate;
+
+    /** */
+    private float evictionRate;
+
+    /** */
+    private float largeEntriesPagesPercentage;
+
+    /** */
+    private float pagesFillFactor;
+
+    /**
+     * @param metrics Metrics instance to take a copy.
+     */
+    public MemoryMetricsSnapshot(MemoryMetrics metrics) {
+        name = metrics.getName();
+        totalAllocatedPages = metrics.getTotalAllocatedPages();
+        allocationRate = metrics.getAllocationRate();
+        evictionRate = metrics.getEvictionRate();
+        largeEntriesPagesPercentage = metrics.getLargeEntriesPagesPercentage();
+        pagesFillFactor = metrics.getPagesFillFactor();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getName() {
+        return name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getTotalAllocatedPages() {
+        return totalAllocatedPages;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getAllocationRate() {
+        return allocationRate;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getEvictionRate() {
+        return evictionRate;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getLargeEntriesPagesPercentage() {
+        return largeEntriesPagesPercentage;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getPagesFillFactor() {
+        return pagesFillFactor;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicy.java
index 90e5ac1..cb35d33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicy.java
@@ -16,7 +16,6 @@
  */
 package org.apache.ignite.internal.processors.cache.database;
 
-import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.evict.PageEvictionTracker;
@@ -29,7 +28,7 @@ public class MemoryPolicy {
     private final PageMemory pageMem;
 
     /** */
-    private final MemoryMetrics memMetrics;
+    private final MemoryMetricsImpl memMetrics;
 
     /** */
     private final MemoryPolicyConfiguration cfg;
@@ -46,7 +45,7 @@ public class MemoryPolicy {
     public MemoryPolicy(
         PageMemory pageMem,
         MemoryPolicyConfiguration cfg,
-        MemoryMetrics memMetrics,
+        MemoryMetricsImpl memMetrics,
         PageEvictionTracker evictionTracker) {
         this.pageMem = pageMem;
         this.memMetrics = memMetrics;
@@ -71,7 +70,7 @@ public class MemoryPolicy {
     /**
      * @return Memory Metrics.
      */
-    public MemoryMetrics memoryMetrics() {
+    public MemoryMetricsImpl memoryMetrics() {
         return memMetrics;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/FairFifoPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/FairFifoPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/FairFifoPageEvictionTracker.java
index 8847013..b7c6b57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/FairFifoPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/FairFifoPageEvictionTracker.java
@@ -14,6 +14,7 @@
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
+
 package org.apache.ignite.internal.processors.cache.database.evict;
 
 import java.util.LinkedList;
@@ -21,7 +22,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 
 /**
@@ -36,7 +37,8 @@ public class FairFifoPageEvictionTracker extends PageAbstractEvictionTracker {
      * @param plcCfg Memory policy configuration.
      * @param sharedCtx Shared context.
      */
-    public FairFifoPageEvictionTracker(PageMemory pageMem,
+    public FairFifoPageEvictionTracker(
+        PageMemoryNoStoreImpl pageMem,
         MemoryPolicyConfiguration plcCfg,
         GridCacheSharedContext sharedCtx) {
         super(pageMem, plcCfg, sharedCtx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
index 61f62fd..c61aced 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
@@ -18,10 +18,9 @@ package org.apache.ignite.internal.processors.cache.database.evict;
 
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
@@ -42,7 +41,7 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
     private static final int DAY = 24 * 60 * 60 * 1000;
 
     /** Page memory. */
-    protected final PageMemory pageMem;
+    protected final PageMemoryNoStoreImpl pageMem;
 
     /** Tracking array size. */
     protected final int trackingSize;
@@ -53,21 +52,13 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
     /** Shared context. */
     private final GridCacheSharedContext sharedCtx;
 
-    /* TODO: IGNITE-4921: Will be removed after segments refactoring >>>> */
-    protected final int segBits;
-    protected final int idxBits;
-    protected final int segMask;
-    protected final int idxMask;
-    protected final int segmentPageCount;
-    /* <<<< */
-
     /**
      * @param pageMem Page memory.
      * @param plcCfg Memory policy configuration.
      * @param sharedCtx Shared context.
      */
     PageAbstractEvictionTracker(
-        PageMemory pageMem,
+        PageMemoryNoStoreImpl pageMem,
         MemoryPolicyConfiguration plcCfg,
         GridCacheSharedContext sharedCtx
     ) {
@@ -75,33 +66,7 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
 
         this.sharedCtx = sharedCtx;
 
-        MemoryConfiguration memCfg = sharedCtx.kernalContext().config().getMemoryConfiguration();
-
-        /* TODO: IGNITE-4921: Will be removed after segments refactoring >>>> */
-        int concurrencyLevel = memCfg.getConcurrencyLevel();
-
-        if (concurrencyLevel < 1)
-            concurrencyLevel = Runtime.getRuntime().availableProcessors();
-
-        int pageSize = memCfg.getPageSize();
-
-        long segSize = plcCfg.getSize() / concurrencyLevel;
-
-        if (segSize < 1024 * 1024)
-            segSize = 1024 * 1024;
-
-        segmentPageCount = (int)(segSize / pageSize);
-
-        segBits = Integer.SIZE - Integer.numberOfLeadingZeros(concurrencyLevel - 1);
-
-        idxBits = PageIdUtils.PAGE_IDX_SIZE - segBits;
-
-        segMask = ~(-1 << segBits);
-
-        idxMask = ~(-1 << idxBits);
-        /* <<<< */
-
-        trackingSize = segmentPageCount << segBits;
+        trackingSize = pageMem.totalPages();
 
         baseCompactTs = (U.currentTimeMillis() - DAY) >> COMPACT_TS_SHIFT;
         // We subtract day to avoid fail in case of daylight shift or timezone change.
@@ -191,15 +156,7 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
      * @return Position of page in tracking array.
      */
     int trackingIdx(int pageIdx) {
-        int inSegmentPageIdx = inSegmentPageIdx(pageIdx);
-
-        assert inSegmentPageIdx < segmentPageCount : inSegmentPageIdx;
-
-        int trackingIdx = segmentIdx(pageIdx) * segmentPageCount + inSegmentPageIdx;
-
-        assert trackingIdx < trackingSize : trackingIdx;
-
-        return trackingIdx;
+        return pageMem.pageSequenceNumber(pageIdx);
     }
 
     /**
@@ -209,36 +166,6 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
      * @return Page index.
      */
     int pageIdx(int trackingIdx) {
-        assert trackingIdx < trackingSize;
-
-        long res = 0;
-
-        long segIdx = trackingIdx / segmentPageCount;
-        long pageIdx = trackingIdx % segmentPageCount;
-
-        res = (res << segBits) | (segIdx & segMask);
-        res = (res << idxBits) | (pageIdx & idxMask);
-
-        assert (res & (-1L << 32)) == 0 : res;
-
-        return (int)res;
-    }
-
-    /* TODO: IGNITE-4921: Will be removed after segments refactoring >>>> */
-    /**
-     * @param pageIdx Page index.
-     * @return Number of segment.
-     */
-    private int segmentIdx(int pageIdx) {
-        return (pageIdx >> idxBits) & segMask;
-    }
-
-    /**
-     * @param pageIdx Page index.
-     * @return Number of page inside segment.
-     */
-    private int inSegmentPageIdx(int pageIdx) {
-        return pageIdx & idxMask;
+        return pageMem.pageIndex(trackingIdx);
     }
-    /* <<<< */
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
index f0ad813..21ebba1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
@@ -23,7 +23,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.LT;
@@ -54,7 +54,7 @@ public class Random2LruPageEvictionTracker extends PageAbstractEvictionTracker {
      * @param sharedCtx Shared context.
      */
     public Random2LruPageEvictionTracker(
-        PageMemory pageMem,
+        PageMemoryNoStoreImpl pageMem,
         MemoryPolicyConfiguration plcCfg,
         GridCacheSharedContext<?, ?> sharedCtx
     ) {
@@ -62,7 +62,7 @@ public class Random2LruPageEvictionTracker extends PageAbstractEvictionTracker {
 
         MemoryConfiguration memCfg = sharedCtx.kernalContext().config().getMemoryConfiguration();
 
-        assert plcCfg.getSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
+        assert plcCfg.getMaxSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
 
         log = sharedCtx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
index 8818b1c..d241148 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
@@ -14,6 +14,7 @@
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
+
 package org.apache.ignite.internal.processors.cache.database.evict;
 
 import java.util.concurrent.ThreadLocalRandom;
@@ -24,6 +25,7 @@ import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.LT;
@@ -58,11 +60,11 @@ public class RandomLruPageEvictionTracker extends PageAbstractEvictionTracker {
         MemoryPolicyConfiguration plcCfg,
         GridCacheSharedContext<?, ?> sharedCtx
     ) {
-        super(pageMem, plcCfg, sharedCtx);
+        super((PageMemoryNoStoreImpl)pageMem, plcCfg, sharedCtx);
 
         MemoryConfiguration memCfg = sharedCtx.kernalContext().config().getMemoryConfiguration();
 
-        assert plcCfg.getSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
+        assert plcCfg.getMaxSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
 
         log = sharedCtx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index 7fa2355..621b833 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -245,7 +245,7 @@ public class IgfsDataManager extends IgfsManager {
     public long maxSpaceSize() {
         MemoryPolicy plc = dataCachePrj.context().memoryPolicy();
 
-        long size = plc != null ? plc.config().getSize() : 0;
+        long size = plc != null ? plc.config().getMaxSize() : 0;
 
         return (size <= 0) ? 0 : size ;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index 9a08b2b..c61b75e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -1415,7 +1415,6 @@ public class PlatformCache extends PlatformAbstractTarget {
         writer.writeLong(metrics.getOffHeapPrimaryEntriesCount());
         writer.writeLong(metrics.getOffHeapBackupEntriesCount());
         writer.writeLong(metrics.getOffHeapAllocatedSize());
-        writer.writeLong(metrics.getOffHeapMaxSize());
         writer.writeInt(metrics.getSize());
         writer.writeInt(metrics.getKeySize());
         writer.writeBoolean(metrics.isEmpty());

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/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 908b63c..98b438d 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
@@ -649,7 +649,8 @@ public class PlatformConfigurationUtils {
                 break;
         }
 
-        cfg.setMemoryConfiguration(readMemoryConfiguration(in));
+        if (in.readBoolean())
+            cfg.setMemoryConfiguration(readMemoryConfiguration(in));
 
         readPluginConfiguration(cfg, in);
     }
@@ -1327,12 +1328,10 @@ public class PlatformConfigurationUtils {
      * @return Config.
      */
     private static MemoryConfiguration readMemoryConfiguration(BinaryRawReader in) {
-        if (!in.readBoolean())
-            return null;
-
         MemoryConfiguration res = new MemoryConfiguration();
 
-        res.setSystemCacheMemorySize(in.readLong())
+        res.setSystemCacheInitialSize(in.readLong())
+                .setSystemCacheMaxSize(in.readLong())
                 .setPageSize(in.readInt())
                 .setConcurrencyLevel(in.readInt())
                 .setDefaultMemoryPolicyName(in.readString());
@@ -1346,7 +1345,8 @@ public class PlatformConfigurationUtils {
                 MemoryPolicyConfiguration cfg = new MemoryPolicyConfiguration();
 
                 cfg.setName(in.readString())
-                        .setSize(in.readLong())
+                        .setInitialSize(in.readLong())
+                        .setMaxSize(in.readLong())
                         .setSwapFilePath(in.readString())
                         .setPageEvictionMode(DataPageEvictionMode.values()[in.readInt()])
                         .setEvictionThreshold(in.readDouble())
@@ -1375,7 +1375,8 @@ public class PlatformConfigurationUtils {
 
         w.writeBoolean(true);
 
-        w.writeLong(cfg.getSystemCacheMemorySize());
+        w.writeLong(cfg.getSystemCacheInitialSize());
+        w.writeLong(cfg.getSystemCacheMaxSize());
         w.writeInt(cfg.getPageSize());
         w.writeInt(cfg.getConcurrencyLevel());
         w.writeString(cfg.getDefaultMemoryPolicyName());
@@ -1387,7 +1388,8 @@ public class PlatformConfigurationUtils {
 
             for (MemoryPolicyConfiguration plc : plcs) {
                 w.writeString(plc.getName());
-                w.writeLong(plc.getSize());
+                w.writeLong(plc.getInitialSize());
+                w.writeLong(plc.getMaxSize());
                 w.writeString(plc.getSwapFilePath());
                 w.writeInt(plc.getPageEvictionMode().ordinal());
                 w.writeDouble(plc.getEvictionThreshold());

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index afe9fea..0c8f857 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -20,14 +20,11 @@ package org.apache.ignite.internal.processors.service;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
@@ -48,7 +45,6 @@ import org.apache.ignite.compute.ComputeJobContext;
 import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
-import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridClosureCallMode;
 import org.apache.ignite.internal.GridKernalContext;
@@ -73,9 +69,7 @@ import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateProposed
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
-import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessage;
-import org.apache.ignite.internal.processors.marshaller.MappingProposedMessage;
-import org.apache.ignite.internal.processors.continuous.AbstractContinuousMessage;
+import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridEmptyIterator;
@@ -92,7 +86,6 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 59d334a..6f8728c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -177,8 +177,6 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteNodeAttributes;
-import org.apache.ignite.internal.binary.BinaryObjectEx;
-import org.apache.ignite.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.compute.ComputeTaskCancelledCheckedException;
@@ -5629,6 +5627,28 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Gets amount of RAM memory available on this machine.
+     *
+     * @return Total amount of memory in bytes or -1 if any exception happened.
+     */
+    public static long getTotalMemoryAvailable() {
+        MBeanServer mBeanSrv = ManagementFactory.getPlatformMBeanServer();
+
+        Object attr;
+
+        try {
+            attr = mBeanSrv.getAttribute(
+                    ObjectName.getInstance("java.lang", "type", "OperatingSystem"),
+                    "TotalPhysicalMemorySize");
+        }
+        catch (Exception e) {
+            return -1;
+        }
+
+        return (attr instanceof Long) ? (Long) attr : -1;
+    }
+
+    /**
      * Gets compilation MBean.
      *
      * @return Compilation MBean.
@@ -9554,7 +9574,7 @@ public abstract class IgniteUtils {
     public static <T extends R, R> List<R> arrayList(Collection<T> c, @Nullable IgnitePredicate<? super T>... p) {
         assert c != null;
 
-        return IgniteUtils.arrayList(c, c.size(), p);
+        return arrayList(c, c.size(), p);
     }
 
     /**
@@ -10113,4 +10133,4 @@ public abstract class IgniteUtils {
             }
         };
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
index 7a0bc76..b756938 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
@@ -66,7 +66,7 @@ public class VisorMemoryConfiguration extends VisorDataTransferObject {
     public VisorMemoryConfiguration(MemoryConfiguration memCfg) {
         assert memCfg != null;
 
-        sysCacheMemSize = memCfg.getSystemCacheMemorySize();
+        sysCacheMemSize = memCfg.getSystemCacheInitialSize();
         pageSize = memCfg.getPageSize();
         concLvl = memCfg.getConcurrencyLevel();
         dfltMemPlcName = memCfg.getDefaultMemoryPolicyName();

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
index d117e5f..dedd71c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
@@ -70,7 +70,7 @@ public class VisorMemoryPolicyConfiguration extends VisorDataTransferObject {
         assert plc != null;
 
         name = plc.getName();
-        size = plc.getSize();
+        size = plc.getMaxSize();
         swapFilePath = plc.getSwapFilePath();
         pageEvictionMode = plc.getPageEvictionMode();
         evictionThreshold = plc.getEvictionThreshold();

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
index 61d2984..3d05763 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
@@ -144,10 +144,6 @@ public interface CacheMetricsMXBean extends CacheStatisticsMXBean, CacheMXBean,
     public long getOffHeapAllocatedSize();
 
     /** {@inheritDoc} */
-    @MXBeanDescription("Off-heap memory maximum size.")
-    public long getOffHeapMaxSize();
-
-    /** {@inheritDoc} */
     @MXBeanDescription("Number of non-null values in the cache.")
     public int getSize();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/main/java/org/apache/ignite/mxbean/MemoryMetricsMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/MemoryMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/MemoryMetricsMXBean.java
index db75f57..6835073 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/MemoryMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/MemoryMetricsMXBean.java
@@ -24,24 +24,33 @@ import org.apache.ignite.MemoryMetrics;
 @MXBeanDescription("MBean that provides access to MemoryMetrics of current Ignite node.")
 public interface MemoryMetricsMXBean extends MemoryMetrics {
     /** {@inheritDoc} */
-    @MXBeanDescription("Name of PageMemory metrics are collected for.")
+    @MXBeanDescription("Name of MemoryPolicy metrics are collected for.")
     @Override public String getName();
 
-    /** {@inheritDoc} */
-    @MXBeanDescription("Size of PageMemory in MBytes.")
-    @Override public int getSize();
+    /**
+     * Initial size configured for MemoryPolicy on local node.
+     *
+     * @return Initial size in MB.
+     */
+    @MXBeanDescription("Initial size configured for MemoryPolicy on local node.")
+    public int getInitialSize();
 
-    /** {@inheritDoc} */
-    @MXBeanDescription("File path of memory-mapped swap file.")
-    @Override public String getSwapFilePath();
+    /**
+     * Maximum size configured for MemoryPolicy on local node.
+     *
+     * @return Maximum size in MB.
+     */
+    @MXBeanDescription("Maximum size configured for MemoryPolicy on local node.")
+    public int getMaxSize();
 
-    /** {@inheritDoc} */
-    @MXBeanDescription("Enables metrics gathering.")
-    @Override public void enableMetrics();
-
-    /** {@inheritDoc} */
-    @MXBeanDescription("Disables metrics gathering.")
-    @Override public void disableMetrics();
+    /**
+     * Path from MemoryPolicy configuration to directory where memory-mapped files used for swap are created.
+     * Depending on configuration may be absolute or relative; in the latter case it is relative to IGNITE_HOME.
+     *
+     * @return path to directory with memory-mapped files.
+     */
+    @MXBeanDescription("Path to directory with memory-mapped files.")
+    public String getSwapFilePath();
 
     /** {@inheritDoc} */
     @MXBeanDescription("Total number of allocated pages.")
@@ -63,27 +72,53 @@ public interface MemoryMetricsMXBean extends MemoryMetrics {
     @MXBeanDescription("Pages fill factor: size of all entries in cache over size of all allocated pages.")
     @Override public float getPagesFillFactor();
 
-    /** {@inheritDoc} */
+    /**
+     * Enables collecting memory metrics on local node.
+     */
+    @MXBeanDescription("Enables collecting memory metrics on local node.")
+    public void enableMetrics();
+
+    /**
+     * Disables collecting memory metrics on local node.
+     */
+    @MXBeanDescription("Disables collecting memory metrics on local node.")
+    public void disableMetrics();
+
+    /**
+     * Sets interval of time (in seconds) to monitor allocation rate.
+     *
+     * E.g. after setting rateTimeInterval to 60 seconds subsequent calls to {@link #getAllocationRate()}
+     * will return average allocation rate (pages per second) for the last minute.
+     *
+     * @param rateTimeInterval Time interval used to calculate allocation/eviction rate.
+     */
     @MXBeanDescription(
-            "Sets time interval average allocation rate (pages per second) is calculated over."
+        "Sets time interval average allocation rate (pages per second) is calculated over."
     )
     @MXBeanParametersNames(
-            "rateTimeInterval"
+        "rateTimeInterval"
     )
     @MXBeanParametersDescriptions(
-            "Time interval (in seconds) to set."
+        "Time interval (in seconds) to set."
     )
-    @Override public void rateTimeInterval(int rateTimeInterval);
+    public void rateTimeInterval(int rateTimeInterval);
 
-    /** {@inheritDoc} */
+    /**
+     * Sets number of subintervals the whole rateTimeInterval will be split into to calculate allocation rate,
+     * 5 by default.
+     * Setting it to bigger number allows more precise calculation and smaller drops of allocationRate metric
+     * when next subinterval has to be recycled but introduces bigger calculation overhead.
+     *
+     * @param subInts Number of subintervals.
+     */
     @MXBeanDescription(
-            "Sets number of subintervals to calculate allocationRate metrics."
+        "Sets number of subintervals to calculate allocationRate metrics."
     )
     @MXBeanParametersNames(
-            "subInts"
+        "subInts"
     )
     @MXBeanParametersDescriptions(
-            "Number of subintervals to set."
+        "Number of subintervals to set."
     )
-    @Override public void subIntervals(int subInts);
+    public void subIntervals(int subInts);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
index d204a39..26bd941 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
@@ -24,12 +24,12 @@ import org.apache.ignite.GridTestTask;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -111,7 +111,7 @@ public class ClusterNodeMetricsSelfTest extends GridCommonAbstractTest {
 
         final IgniteCache cache = ignite.getOrCreateCache(CACHE_NAME);
 
-        MemoryMetrics memMetrics = getDefaultMemoryPolicyMetrics(ignite);
+        MemoryMetricsImpl memMetrics = getDefaultMemoryPolicyMetrics(ignite);
 
         memMetrics.enableMetrics();
 
@@ -128,7 +128,7 @@ public class ClusterNodeMetricsSelfTest extends GridCommonAbstractTest {
     /**
      * @param ignite Ignite instance.
      */
-    private MemoryMetrics getDefaultMemoryPolicyMetrics(IgniteEx ignite) throws IgniteCheckedException {
+    private MemoryMetricsImpl getDefaultMemoryPolicyMetrics(IgniteEx ignite) throws IgniteCheckedException {
         return ignite.context().cache().context().database().memoryPolicy(null).memoryMetrics();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
index 0fe90cd..84db565 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
@@ -279,15 +280,18 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
     protected PageMemory memory() throws Exception {
         File memDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "pagemem", false);
 
-        long[] sizes = new long[10];
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(10 * 1024 * 1024);
 
-        for (int i = 0; i < sizes.length; i++)
-            sizes[i] = 1024 * 1024;
+        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), memDir);
 
-        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), memDir, true,
-            sizes);
-
-        return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE, null, new MemoryMetricsImpl(null), true);
+        return new PageMemoryNoStoreImpl(
+            log(),
+            provider,
+            null,
+            PAGE_SIZE,
+            plcCfg,
+            new MemoryMetricsImpl(plcCfg),
+            true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
index b97bb26..bf94d16 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
@@ -48,7 +48,7 @@ public class CacheConfigurationLeakTest extends GridCommonAbstractTest {
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
         plc.setName("dfltPlc");
-        plc.setSize(MemoryConfiguration.DFLT_MEMORY_POLICY_SIZE * 10);
+        plc.setMaxSize(MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE * 10);
 
         memCfg.setDefaultMemoryPolicyName("dfltPlc");
         memCfg.setMemoryPolicies(plc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMemoryPolicyConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMemoryPolicyConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMemoryPolicyConfigurationTest.java
index 92e7e84..326a830 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMemoryPolicyConfigurationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMemoryPolicyConfigurationTest.java
@@ -84,11 +84,12 @@ public class CacheMemoryPolicyConfigurationTest extends GridCommonAbstractTest {
 
         MemoryPolicyConfiguration dfltPlcCfg = new MemoryPolicyConfiguration();
         dfltPlcCfg.setName("dfltPlc");
-        dfltPlcCfg.setSize(1024 * 1024);
+        dfltPlcCfg.setInitialSize(10 * 1024 * 1024);
+        dfltPlcCfg.setMaxSize(10 * 1024 * 1024);
 
         MemoryPolicyConfiguration bigPlcCfg = new MemoryPolicyConfiguration();
         bigPlcCfg.setName("bigPlc");
-        bigPlcCfg.setSize(1024 * 1024 * 1024);
+        bigPlcCfg.setMaxSize(1024 * 1024 * 1024);
 
         memCfg.setMemoryPolicies(dfltPlcCfg, bigPlcCfg);
         memCfg.setDefaultMemoryPolicyName("dfltPlc");
@@ -137,11 +138,12 @@ public class CacheMemoryPolicyConfigurationTest extends GridCommonAbstractTest {
 
         MemoryPolicyConfiguration dfltPlcCfg = new MemoryPolicyConfiguration();
         dfltPlcCfg.setName("dfltPlc");
-        dfltPlcCfg.setSize(1024 * 1024);
+        dfltPlcCfg.setInitialSize(1024 * 1024);
+        dfltPlcCfg.setMaxSize(1024 * 1024);
 
         MemoryPolicyConfiguration bigPlcCfg = new MemoryPolicyConfiguration();
         bigPlcCfg.setName("bigPlc");
-        bigPlcCfg.setSize(1024 * 1024 * 1024);
+        bigPlcCfg.setMaxSize(1024 * 1024 * 1024);
 
         memCfg.setMemoryPolicies(dfltPlcCfg, bigPlcCfg);
         memCfg.setDefaultMemoryPolicyName("dfltPlc");

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
index 154e562..a1a05eb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
@@ -26,6 +26,15 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  *
  */
 public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
+    /** */
+    private static final String VALID_DEFAULT_MEM_PLC_NAME = "valid_dlft_mem_plc";
+
+    /** */
+    private static final String VALID_USER_MEM_PLC_NAME = "valid_user_mem_plc";
+
+    /** */
+    private static final String MISSING_DEFAULT_MEM_PLC_NAME = "missing_mem_plc";
+
     /** Configuration violation type to check. */
     private ValidationViolationType violationType;
 
@@ -61,9 +70,30 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
             case MISSING_USER_DEFINED_DEFAULT:
                 plcs = createMissingUserDefinedDefault();
 
-                memCfg.setDefaultMemoryPolicyName("missingMemoryPolicyName");
+                memCfg.setDefaultMemoryPolicyName(MISSING_DEFAULT_MEM_PLC_NAME);
+
+                break;
+
+            case TOO_SMALL_USER_DEFINED_DFLT_MEM_PLC_SIZE:
+                memCfg.setDefaultMemoryPolicySize(1);
+
+                break;
+
+            case DEFAULT_SIZE_IS_DEFINED_TWICE:
+                plcs = createValidUserDefault();
+
+                memCfg.setDefaultMemoryPolicyName(VALID_DEFAULT_MEM_PLC_NAME);
+                memCfg.setDefaultMemoryPolicySize(10 * 1014 * 1024);
+
+                break;
+
+            case MAX_SIZE_IS_SMALLER_THAN_INITIAL_SIZE:
+                plcs = createMaxSizeSmallerThanInitialSize();
 
                 break;
+
+            default:
+                fail("Violation type was not configured: " + violationType);
         }
 
         memCfg.setMemoryPolicies(plcs);
@@ -73,18 +103,35 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /**
+     *
+     */
+    private MemoryPolicyConfiguration[] createValidUserDefault() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy(VALID_DEFAULT_MEM_PLC_NAME, 100 * 1024 * 1024, 100 * 1024 * 1024);
+
+        return res;
+    }
+
+    /**
+     *
+     */
     private MemoryPolicyConfiguration[] createMissingUserDefinedDefault() {
         MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
 
-        res[0] = createMemoryPolicy("presentedPolicyCfg", 10 * 1024 * 1024);
+        res[0] = createMemoryPolicy(VALID_USER_MEM_PLC_NAME, 10 * 1024 * 1024, 10 * 1024 * 1024);
 
         return res;
     }
 
+    /**
+     *
+     */
     private MemoryPolicyConfiguration[] createPlcWithNullName() {
         MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
 
-        res[0] = createMemoryPolicy(null, 10 * 1024 * 1024);
+        res[0] = createMemoryPolicy(null, 10 * 1024 * 1024, 10 * 1024 * 1024);
 
         return res;
     }
@@ -95,7 +142,7 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     private MemoryPolicyConfiguration[] createTooSmallMemoryCfg() {
         MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
 
-        res[0] = createMemoryPolicy("dflt", 10);
+        res[0] = createMemoryPolicy(VALID_DEFAULT_MEM_PLC_NAME, 10, 10);
 
         return res;
     }
@@ -106,7 +153,7 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     private MemoryPolicyConfiguration[] createPlcWithReservedNameMisuseCfg() {
         MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
 
-        res[0] = createMemoryPolicy("sysMemPlc", 1024 * 1024);
+        res[0] = createMemoryPolicy("sysMemPlc", 1024 * 1024, 1024 * 1024);
 
         return res;
     }
@@ -117,8 +164,19 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     private MemoryPolicyConfiguration[] createPlcsWithNamesConflictCfg() {
         MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[2];
 
-        res[0] = createMemoryPolicy("cflt0", 1024 * 1024);
-        res[1] = createMemoryPolicy("cflt0", 1024 * 1024);
+        res[0] = createMemoryPolicy("cflt0", 10 * 1024 * 1024, 10 * 1024 * 1024);
+        res[1] = createMemoryPolicy("cflt0", 10 * 1024 * 1024, 10 * 1024 * 1024);
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private MemoryPolicyConfiguration[] createMaxSizeSmallerThanInitialSize() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy("invalidSize", 100 * 1024 * 1024, 10 * 1024 * 1024);
 
         return res;
     }
@@ -132,13 +190,15 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
 
     /**
      * @param name Name of MemoryPolicyConfiguration.
-     * @param size Size of MemoryPolicyConfiguration in bytes.
+     * @param initialSize Initial size of MemoryPolicyConfiguration in bytes.
+     * @param maxSize Max size of MemoryPolicyConfiguration in bytes.
      */
-    private MemoryPolicyConfiguration createMemoryPolicy(String name, long size) {
+    private MemoryPolicyConfiguration createMemoryPolicy(String name, long initialSize, long maxSize) {
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
         plc.setName(name);
-        plc.setSize(size);
+        plc.setInitialSize(initialSize);
+        plc.setMaxSize(maxSize);
 
         return plc;
     }
@@ -189,6 +249,34 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     }
 
     /**
+     * MemoryPolicy must be configured with size of at least 1MB.
+     */
+    public void testMaxSizeSmallerThanInitialSize() throws Exception {
+        violationType = ValidationViolationType.MAX_SIZE_IS_SMALLER_THAN_INITIAL_SIZE;
+
+        doTest(violationType);
+    }
+
+    /**
+     * User-defined size of default MemoryPolicy must be at least 1MB.
+     */
+    public void testUserDefinedDefaultMemoryTooSmall() throws Exception {
+        violationType = ValidationViolationType.TOO_SMALL_USER_DEFINED_DFLT_MEM_PLC_SIZE;
+
+        doTest(violationType);
+    }
+
+    /**
+     * Defining size of default MemoryPolicy twice with and through <b>defaultMemoryPolicySize</b> property
+     * and using <b>MemoryPolicyConfiguration</b> description is prohibited.
+     */
+    public void testDefaultMemoryPolicySizeDefinedTwice() throws Exception {
+        violationType = ValidationViolationType.DEFAULT_SIZE_IS_DEFINED_TWICE;
+
+        doTest(violationType);
+    }
+
+    /**
      * Tries to start ignite node with invalid configuration and checks that corresponding exception is thrown.
      *
      * @param violationType Configuration violation type.
@@ -220,13 +308,22 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
         SYSTEM_MEMORY_POLICY_NAME_MISUSE("'sysMemPlc' policy name is reserved for internal use."),
 
         /** */
-        TOO_SMALL_MEMORY_SIZE("MemoryPolicy must have size more than 1MB: "),
+        TOO_SMALL_MEMORY_SIZE("MemoryPolicy must have size more than 10MB "),
 
         /** */
         NULL_NAME_ON_USER_DEFINED_POLICY("User-defined MemoryPolicyConfiguration must have non-null and non-empty name."),
 
         /** */
-        MISSING_USER_DEFINED_DEFAULT("User-defined default MemoryPolicy name must be presented among configured MemoryPolices: ");
+        MISSING_USER_DEFINED_DEFAULT("User-defined default MemoryPolicy name must be presented among configured MemoryPolices: "),
+
+        /** */
+        DEFAULT_SIZE_IS_DEFINED_TWICE("User-defined MemoryPolicy configuration and defaultMemoryPolicySize properties are set at the same time."),
+
+        /** */
+        TOO_SMALL_USER_DEFINED_DFLT_MEM_PLC_SIZE("User-defined default MemoryPolicy size is less than 1MB."),
+
+        /** */
+        MAX_SIZE_IS_SMALLER_THAN_INITIAL_SIZE("MemoryPolicy maxSize must not be smaller than initialSize");
 
         /**
          * @param violationMsg Violation message.

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicyInitializationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicyInitializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicyInitializationTest.java
index 1e3f328..a1c9728 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicyInitializationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicyInitializationTest.java
@@ -117,7 +117,7 @@ public class MemoryPolicyInitializationTest extends GridCommonAbstractTest {
 
         MemoryPolicy dfltMemPlc = U.field(dbMgr, "dfltMemPlc");
 
-        assertTrue(dfltMemPlc.config().getSize() == USER_DEFAULT_MEM_PLC_SIZE);
+        assertTrue(dfltMemPlc.config().getMaxSize() == USER_DEFAULT_MEM_PLC_SIZE);
     }
 
     /**
@@ -141,7 +141,7 @@ public class MemoryPolicyInitializationTest extends GridCommonAbstractTest {
 
         MemoryPolicy dfltMemPlc = U.field(dbMgr, "dfltMemPlc");
 
-        assertTrue(dfltMemPlc.config().getSize() == USER_CUSTOM_MEM_PLC_SIZE);
+        assertTrue(dfltMemPlc.config().getMaxSize() == USER_CUSTOM_MEM_PLC_SIZE);
     }
 
     /**
@@ -232,11 +232,11 @@ public class MemoryPolicyInitializationTest extends GridCommonAbstractTest {
 
         memCfg.setMemoryPolicies(new MemoryPolicyConfiguration()
                 .setName(CUSTOM_NON_DEFAULT_MEM_PLC_NAME)
-                .setSize(USER_CUSTOM_MEM_PLC_SIZE),
+                .setMaxSize(USER_CUSTOM_MEM_PLC_SIZE),
 
                 new MemoryPolicyConfiguration()
                 .setName(DFLT_MEM_PLC_DEFAULT_NAME)
-                .setSize(USER_DEFAULT_MEM_PLC_SIZE)
+                .setMaxSize(USER_DEFAULT_MEM_PLC_SIZE)
         );
     }
 
@@ -249,7 +249,7 @@ public class MemoryPolicyInitializationTest extends GridCommonAbstractTest {
 
         memCfg.setMemoryPolicies(new MemoryPolicyConfiguration()
                 .setName(DFLT_MEM_PLC_DEFAULT_NAME)
-                .setSize(USER_DEFAULT_MEM_PLC_SIZE)
+                .setMaxSize(USER_DEFAULT_MEM_PLC_SIZE)
         );
     }
 
@@ -261,11 +261,11 @@ public class MemoryPolicyInitializationTest extends GridCommonAbstractTest {
 
         memCfg.setMemoryPolicies(new MemoryPolicyConfiguration()
                 .setName(DFLT_MEM_PLC_DEFAULT_NAME)
-                .setSize(USER_DEFAULT_MEM_PLC_SIZE),
+                .setMaxSize(USER_DEFAULT_MEM_PLC_SIZE),
 
                 new MemoryPolicyConfiguration()
                 .setName(CUSTOM_NON_DEFAULT_MEM_PLC_NAME)
-                .setSize(USER_CUSTOM_MEM_PLC_SIZE)
+                .setMaxSize(USER_CUSTOM_MEM_PLC_SIZE)
         );
     }
 
@@ -288,7 +288,7 @@ public class MemoryPolicyInitializationTest extends GridCommonAbstractTest {
 
         memCfg.setMemoryPolicies(new MemoryPolicyConfiguration()
                 .setName(CUSTOM_NON_DEFAULT_MEM_PLC_NAME)
-                .setSize(USER_CUSTOM_MEM_PLC_SIZE)
+                .setMaxSize(USER_CUSTOM_MEM_PLC_SIZE)
         );
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
index 39927be..bda7940 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
@@ -88,7 +88,9 @@ public class PageEvictionAbstractTest extends GridCommonAbstractTest {
 
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
-        plc.setSize(SIZE);
+        // This will test additional segment allocation.
+        plc.setInitialSize(SIZE / 2);
+        plc.setMaxSize(SIZE);
         plc.setEmptyPagesPoolSize(EMPTY_PAGES_POOL_SIZE);
         plc.setEvictionThreshold(EVICTION_THRESHOLD);
         plc.setName(DEFAULT_POLICY_NAME);

http://git-wip-us.apache.org/repos/asf/ignite/blob/11c23b62/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
index 4660972..bc297a2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
@@ -103,7 +103,7 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
         plc.setName("dfltPlc");
-        plc.setSize(MemoryConfiguration.DFLT_MEMORY_POLICY_SIZE * 10);
+        plc.setMaxSize(MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE * 10);
 
         memCfg.setDefaultMemoryPolicyName("dfltPlc");
         memCfg.setMemoryPolicies(plc);