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/03/30 17:25:31 UTC

[21/34] ignite git commit: IGNITE-4758 - Introduced multiple memory policies. Fixes #1595

IGNITE-4758 - Introduced multiple memory policies. Fixes #1595


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

Branch: refs/heads/ignite-3477-master
Commit: 7ac0fd0c9a6a7c89951f04489269556475bafdd2
Parents: 96286f2
Author: Sergey Chugunov <se...@gmail.com>
Authored: Wed Mar 29 15:27:05 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Mar 29 19:29:38 2017 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  25 ++
 .../configuration/MemoryConfiguration.java      | 104 ++++--
 .../MemoryPolicyConfiguration.java              |  80 +++++
 .../apache/ignite/internal/IgniteKernal.java    |  50 ++-
 .../processors/cache/GridCacheContext.java      |  43 +++
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  34 +-
 .../cache/database/CacheDataRowAdapter.java     |   4 +-
 .../IgniteCacheDatabaseSharedManager.java       | 354 ++++++++++++++++---
 .../processors/cache/database/MemoryPolicy.java |  53 +++
 .../processors/cache/database/RowStore.java     |   2 +-
 .../cache/distributed/near/GridNearTxLocal.java |   1 -
 .../visor/node/VisorGridConfiguration.java      |   2 +-
 .../visor/node/VisorMemoryConfiguration.java    |  55 ++-
 .../node/VisorMemoryPolicyConfiguration.java    |  72 ++++
 .../resources/META-INF/classnames.properties    |   1 +
 .../ignite/cache/LargeEntryUpdateTest.java      |   1 -
 .../cache/CacheConfigurationLeakTest.java       |   7 +-
 .../CacheMemoryPolicyConfigurationTest.java     | 164 +++++++++
 .../cache/MemoryPolicyConfigValidationTest.java | 241 +++++++++++++
 .../database/IgniteDbAbstractTest.java          |   2 -
 .../database/IgniteDbDynamicCacheSelfTest.java  |   9 +-
 .../IgniteDbMemoryLeakAbstractTest.java         |  10 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   3 +
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 .../query/h2/database/H2TreeIndex.java          |   5 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 196 +++++-----
 .../cpp/core-test/config/cache-query-32.xml     |  50 +++
 .../config/cache-query-continuous-32.xml        |  46 +++
 .../config/cache-query-continuous-default.xml   |  87 +++++
 .../core-test/config/cache-query-continuous.xml |  61 +---
 .../core-test/config/cache-query-default.xml    | 131 +++++++
 .../cpp/core-test/config/cache-query.xml        | 114 +-----
 .../cpp/core-test/config/cache-test-32.xml      |  50 +++
 .../cpp/core-test/config/cache-test-default.xml | 135 +++++++
 .../cpp/core-test/config/cache-test.xml         | 105 +-----
 .../cpp/core-test/project/vs/core-test.vcxproj  |   7 +
 .../project/vs/core-test.vcxproj.filters        |  21 ++
 .../cpp/core-test/src/cache_query_test.cpp      |   4 +
 .../platforms/cpp/core-test/src/cache_test.cpp  |   5 +
 .../cpp/core-test/src/continuous_query_test.cpp |   4 +
 .../cpp/core-test/src/ignition_test.cpp         |   4 +
 .../cpp/core-test/src/interop_test.cpp          |   4 +
 .../cpp/core-test/src/transactions_test.cpp     |   4 +
 .../cpp/odbc-test/config/queries-test-32.xml    |  47 +++
 .../odbc-test/config/queries-test-default.xml   |  38 ++
 .../odbc-test/config/queries-test-noodbc-32.xml |  47 +++
 .../cpp/odbc-test/config/queries-test.xml       |  11 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   3 +
 .../project/vs/odbc-test.vcxproj.filters        |   6 +
 .../cpp/odbc-test/src/api_robustness_test.cpp   |  20 +-
 .../cpp/odbc-test/src/queries_test.cpp          |   8 +
 .../odbc-test/src/sql_test_suite_fixture.cpp    |   6 +-
 53 files changed, 2057 insertions(+), 504 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 25398ca..84e1b01 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -223,6 +223,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Cache name. */
     private String name;
 
+    /** Name of {@link MemoryPolicyConfiguration} for this cache */
+    private String memPlcName;
+
     /** Threshold for concurrent loading of keys from {@link CacheStore}. */
     private int storeConcurrentLoadAllThreshold = DFLT_CONCURRENT_LOAD_ALL_THRESHOLD;
 
@@ -467,6 +470,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         longQryWarnTimeout = cc.getLongQueryWarningTimeout();
         offHeapMaxMem = cc.getOffHeapMaxMemory();
         maxConcurrentAsyncOps = cc.getMaxConcurrentAsyncOperations();
+        memPlcName = cc.getMemoryPolicyName();
         sqlIndexMaxInlineSize = cc.getSqlIndexMaxInlineSize();
         name = cc.getName();
         nearCfg = cc.getNearConfiguration();
@@ -528,6 +532,27 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * @return {@link MemoryPolicyConfiguration} name.
+     */
+    public String getMemoryPolicyName() {
+        return memPlcName;
+    }
+
+    /**
+     * Sets name of {@link MemoryPolicyConfiguration} for this cache.
+     *
+     * @param memPlcName MemoryPolicyConfiguration name. Can be null (default MemoryPolicyConfiguration will be used) but should not be empty.
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration<K, V> setMemoryPolicyName(String memPlcName) {
+        A.ensure(name == null || !name.isEmpty(), "Name cannot be empty.");
+
+        this.memPlcName = memPlcName;
+
+        return this;
+    }
+
+    /**
      * Gets cache eviction policy. By default, returns {@code null}
      * which means that evictions are disabled for cache.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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 242354e..1722781 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
@@ -22,30 +22,74 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
- * Database configuration used to configure database.
+ * Database configuration used to configure database and manage offheap memory of Ignite Node.
+ *
+ * <p>It may be configured under {@link IgniteConfiguration XML configuration} as follows:</p>
+ * <pre>
+ *     {@code
+ *     <property name="memoryConfiguration">
+ *         <bean class="org.apache.ignite.configuration.MemoryConfiguration">
+ *             <property name="systemCacheMemorySize" value="103833600"/>
+ *             <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="103833600"/>
+ *                     </bean>
+ *                     <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
+ *                         ...
+ *                     </bean>
+ *                 </list>
+ *             </property>
+ *         </bean>
+ *     </property>
+ *     }
+ * </pre>
  */
 public class MemoryConfiguration implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Default cache size is 1Gb. */
-    public static final long DFLT_PAGE_CACHE_SIZE = 1024 * 1024 * 1024;
+    /** Default MemoryPolicy size is 1GB. */
+    public static final long DFLT_MEMORY_POLICY_SIZE = 1024 * 1024 * 1024;
+
+    /** Default size of memory chunk for system cache is 100MB. */
+    public static final long DFLT_SYS_CACHE_MEM_SIZE = 100 * 1024 * 1024;
 
     /** Default page size. */
     public static final int DFLT_PAGE_SIZE = 2 * 1024;
 
+    /** Size of memory for system cache. */
+    private long sysCacheMemSize = DFLT_SYS_CACHE_MEM_SIZE;
+
     /** Page size. */
     private int pageSize = DFLT_PAGE_SIZE;
 
-    /** File cache allocation path. */
-    private String fileCacheAllocationPath;
-
-    /** Amount of memory allocated for the page cache. */
-    private long pageCacheSize = DFLT_PAGE_CACHE_SIZE;
-
     /** Concurrency level. */
     private int concLvl;
 
+    /** Name of MemoryPolicy to be used as default. */
+    private String dfltMemPlcName;
+
+    /** Memory policies. */
+    private MemoryPolicyConfiguration[] memPlcs;
+
+    /**
+     * @return memory size for system cache.
+     */
+    public long getSystemCacheMemorySize() {
+        return sysCacheMemSize;
+    }
+
+    /**
+     * @param sysCacheMemSize Memory size for system cache.
+     */
+    public void setSystemCacheMemorySize(long sysCacheMemSize) {
+        this.sysCacheMemSize = sysCacheMemSize;
+    }
+
     /**
      * @return Page size.
      */
@@ -64,31 +108,29 @@ public class MemoryConfiguration implements Serializable {
     }
 
     /**
-     * @return File allocation path.
+     * @return Array of MemoryPolicyConfiguration objects.
      */
-    public String getFileCacheAllocationPath() {
-        return fileCacheAllocationPath;
+    public MemoryPolicyConfiguration[] getMemoryPolicies() {
+        return memPlcs;
     }
 
     /**
-     * @param fileCacheAllocationPath File allocation path.
+     * @param memPlcs MemoryPolicyConfiguration instances.
      */
-    public void setFileCacheAllocationPath(String fileCacheAllocationPath) {
-        this.fileCacheAllocationPath = fileCacheAllocationPath;
+    public void setMemoryPolicies(MemoryPolicyConfiguration... memPlcs) {
+        this.memPlcs = memPlcs;
     }
 
     /**
-     * @return Page cache size, in bytes.
+     * @return default {@link MemoryPolicyConfiguration} instance.
      */
-    public long getPageCacheSize() {
-        return pageCacheSize;
-    }
+    public MemoryPolicyConfiguration createDefaultPolicyConfig() {
+        MemoryPolicyConfiguration memPlc = new MemoryPolicyConfiguration();
 
-    /**
-     * @param pageCacheSize Page cache size, in bytes.
-     */
-    public void setPageCacheSize(long pageCacheSize) {
-        this.pageCacheSize = pageCacheSize;
+        memPlc.setName(null);
+        memPlc.setSize(DFLT_MEMORY_POLICY_SIZE);
+
+        return memPlc;
     }
 
     /**
@@ -104,4 +146,18 @@ public class MemoryConfiguration implements Serializable {
     public void setConcurrencyLevel(int concLvl) {
         this.concLvl = concLvl;
     }
+
+    /**
+     * @return Name of MemoryPolicy to be used as default.
+     */
+    public String getDefaultMemoryPolicyName() {
+        return dfltMemPlcName;
+    }
+
+    /**
+     * @param dfltMemPlcName Name of MemoryPolicy to be used as default.
+     */
+    public void setDefaultMemoryPolicyName(String dfltMemPlcName) {
+        this.dfltMemPlcName = dfltMemPlcName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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
new file mode 100644
index 0000000..3bf1706
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -0,0 +1,80 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.database.MemoryPolicy;
+
+/**
+ * Configuration bean used for creating {@link MemoryPolicy} instances.
+ */
+public final class MemoryPolicyConfiguration implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Unique name of MemoryPolicy. */
+    private String name;
+
+    /** Size in bytes of {@link PageMemory} in bytes that will be created for this configuration. */
+    private long size;
+
+    /** Path for memory mapped file (won't be created if not configured). */
+    private String swapFilePath;
+
+    /**
+     * Unique name of MemoryPolicy.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @param name Unique name of MemoryPolicy.
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * Size in bytes of {@link PageMemory} in bytes that will be created for this configuration.
+     */
+    public long getSize() {
+        return size;
+    }
+
+    /**
+     * Size in bytes of {@link PageMemory} in bytes that will be created for this configuration.
+     */
+    public void setSize(long size) {
+        this.size = size;
+    }
+
+    /**
+     * @return Path for memory mapped file (won't be created if not configured).
+     */
+    public String getSwapFilePath() {
+        return swapFilePath;
+    }
+
+    /**
+     * @param swapFilePath Path for memory mapped file (won't be created if not configured)..
+     */
+    public void setSwapFilePath(String swapFilePath) {
+        this.swapFilePath = swapFilePath;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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 6252182..9a924a1 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
@@ -36,6 +36,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
+import java.util.HashMap;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
@@ -81,6 +82,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.binary.BinaryEnumCache;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
@@ -107,6 +109,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.database.MemoryPolicy;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.clock.GridClockSyncProcessor;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
@@ -775,6 +778,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ackClassPaths(rtBean);
         ackSystemProperties();
         ackEnvironmentVariables();
+        ackMemoryConfiguration();
         ackCacheConfiguration();
         ackP2pConfiguration();
         ackRebalanceConfiguration();
@@ -1185,7 +1189,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                                 sysPoolQSize = exec.getQueue().size();
                             }
 
-                            PageMemory pageMem = ctx.cache().context().database().pageMemory();
+                            int loadedPages = 0;
+
+                            for (GridCacheContext cctx : ctx.cache().context().cacheContexts()) {
+                                MemoryPolicy memPlc = cctx.memoryPolicy();
+
+                                loadedPages += memPlc != null ? memPlc.pageMemory().loadedPages() : 0;
+                            }
 
                             String id = U.id8(localNode().id());
 
@@ -1195,7 +1205,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                                 "    ^-- H/N/C [hosts=" + hosts + ", nodes=" + nodes + ", CPUs=" + cpus + "]" + NL +
                                 "    ^-- CPU [cur=" + dblFmt.format(cpuLoadPct) + "%, avg=" +
                                 dblFmt.format(avgCpuLoadPct) + "%, GC=" + dblFmt.format(gcPct) + "%]" + NL +
-                                "    ^-- PageMemory [pages=" + (pageMem != null ? pageMem.loadedPages() : 0) + "]" + NL +
+                                "    ^-- PageMemory [pages=" + loadedPages + "]" + NL +
                                 "    ^-- Heap [used=" + dblFmt.format(heapUsedInMBytes) + "MB, free=" +
                                 dblFmt.format(freeHeapPct) + "%, comm=" + dblFmt.format(heapCommInMBytes) + "MB]" + NL +
                                 "    ^-- Non heap [used=" + dblFmt.format(nonHeapUsedInMBytes) + "MB, free=" +
@@ -2415,6 +2425,18 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     /**
      *
      */
+    private void ackMemoryConfiguration() {
+        MemoryConfiguration memCfg = cfg.getMemoryConfiguration();
+
+        U.log(log, "System cache MemoryPolicy size is configured to " +
+                (memCfg.getSystemCacheMemorySize() / (1024 * 1024)) +
+        "MB size. " +
+                "Use MemoryConfiguration.systemCacheMemorySize property to change it.");
+    }
+
+    /**
+     *
+     */
     private void ackCacheConfiguration() {
         CacheConfiguration[] cacheCfgs = cfg.getCacheConfiguration();
 
@@ -2423,15 +2445,31 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         else {
             SB sb = new SB();
 
+            HashMap<String, ArrayList<String>> memPlcNamesMapping = new HashMap<>();
+
             for (CacheConfiguration c : cacheCfgs) {
-                String name = U.maskName(c.getName());
+                String cacheName = U.maskName(c.getName());
+
+                String memPlcName = U.maskName(c.getMemoryPolicyName());
+
+                if (!memPlcNamesMapping.containsKey(memPlcName))
+                    memPlcNamesMapping.put(memPlcName, new ArrayList<String>());
 
-                sb.a("'").a(name).a("', ");
+                ArrayList<String> cacheNames = memPlcNamesMapping.get(memPlcName);
+
+                cacheNames.add(cacheName);
             }
 
-            String names = sb.toString();
+            for (Map.Entry<String, ArrayList<String>> e : memPlcNamesMapping.entrySet()) {
+                sb.a("in '").a(e.getKey()).a("' memoryPolicy: [");
+
+                for (String s : e.getValue())
+                    sb.a("'").a(s).a("', ");
+
+                sb.d(sb.length() - 2, sb.length()).a("]");
+            }
 
-            U.log(log, "Configured caches [" + names.substring(0, names.length() - 2) + ']');
+            U.log(log, "Configured caches [" + sb.toString() + ']');
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index e8513de..ffc379c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -54,7 +54,11 @@ import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentManager;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.database.MemoryPolicy;
+import org.apache.ignite.internal.processors.cache.database.freelist.FreeList;
+import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
@@ -133,6 +137,15 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Cache shared context. */
     private GridCacheSharedContext<K, V> sharedCtx;
 
+    /** Memory policy. */
+    private MemoryPolicy memPlc;
+
+    /** FreeList instance this cache is associated with. */
+    private FreeList freeList;
+
+    /** ReuseList instance this cache is associated with */
+    private ReuseList reuseList;
+
     /** Logger. */
     private IgniteLogger log;
 
@@ -256,6 +269,8 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param sharedCtx Cache shared context.
      * @param cacheCfg Cache configuration.
      * @param cacheType Cache type.
+     * @param memPlc MemoryPolicy instance.
+     * @param freeList FreeList instance.
      * @param affNode {@code True} if local node is affinity node.
      * @param updatesAllowed Updates allowed flag.
      * @param evtMgr Cache event manager.
@@ -278,6 +293,9 @@ public class GridCacheContext<K, V> implements Externalizable {
         CacheType cacheType,
         boolean affNode,
         boolean updatesAllowed,
+        MemoryPolicy memPlc,
+        FreeList freeList,
+        ReuseList reuseList,
 
         /*
          * Managers in starting order!
@@ -321,6 +339,10 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.updatesAllowed = updatesAllowed;
         this.depEnabled = ctx.deploy().enabled() && !cacheObjects().isBinaryEnabled(cacheCfg);
 
+        this.memPlc = memPlc;
+        this.freeList = freeList;
+        this.reuseList = reuseList;
+
         /*
          * Managers in starting order!
          * ===========================
@@ -681,6 +703,27 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Memory policy.
+     */
+    public MemoryPolicy memoryPolicy() {
+        return memPlc;
+    }
+
+    /**
+     * @return Free List.
+     */
+    public FreeList freeList() {
+        return freeList;
+    }
+
+    /**
+     * @return Reuse List.
+     */
+    public ReuseList reuseList() {
+        return reuseList;
+    }
+
+    /**
      * Gets public name for cache.
      *
      * @return Public name of the cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 9d7b40f..d4f6d87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -73,6 +73,7 @@ import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.snapshot.StartFullSnapshotAckDiscoveryMessage;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
@@ -80,6 +81,9 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.database.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.database.MemoryPolicy;
+import org.apache.ignite.internal.processors.cache.database.freelist.FreeList;
+import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -708,6 +712,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             else
                 cacheType = CacheType.USER;
 
+        if (cacheType != CacheType.USER)
+            cfg.setMemoryPolicyName(sharedCtx.database().systemMemoryPolicyName());
+
         boolean template = cfg.getName() != null && cfg.getName().endsWith("*");
 
         DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx,
@@ -1220,7 +1227,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cacheCtx.onStarted();
 
         if (log.isInfoEnabled())
-            log.info("Started cache [name=" + U.maskName(cfg.getName()) + ", mode=" + cfg.getCacheMode() + ']');
+            log.info("Started cache [name=" + U.maskName(cfg.getName()) + ", memoryPolicyName=" + cfg.getMemoryPolicyName() + ", mode=" + cfg.getCacheMode() + ']');
     }
 
     /**
@@ -1455,6 +1462,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         boolean affNode = cfg.getCacheMode() == LOCAL || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter());
 
+        String memPlcName = cfg.getMemoryPolicyName();
+
+        MemoryPolicy memPlc = sharedCtx.database().memoryPolicy(memPlcName);
+        FreeList freeList = sharedCtx.database().freeList(memPlcName);
+        ReuseList reuseList = sharedCtx.database().reuseList(memPlcName);
+
         GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
             ctx,
             sharedCtx,
@@ -1462,6 +1475,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cacheType,
             affNode,
             updatesAllowed,
+            memPlc,
+            freeList,
+            reuseList,
 
             /*
              * Managers in starting order!
@@ -1590,6 +1606,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 cacheType,
                 affNode,
                 true,
+                memPlc,
+                freeList,
+                reuseList,
 
                 /*
                  * Managers in starting order!

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 16d3715..88cb708 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -162,9 +162,9 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
             pendingEntries = new PendingEntriesTree(cctx,
                 name,
-                cctx.shared().database().pageMemory(),
+                cctx.memoryPolicy().pageMemory(),
                 rootPage,
-                cctx.shared().database().globalReuseList(),
+                cctx.reuseList(),
                 true);
         }
     }
@@ -639,12 +639,12 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
      * @throws IgniteCheckedException If failed.
      */
     private long allocateForTree() throws IgniteCheckedException {
-        ReuseList reuseList = cctx.shared().database().globalReuseList();
+        ReuseList reuseList = cctx.reuseList();
 
         long pageId;
 
         if (reuseList == null || (pageId = reuseList.takeRecycledPage()) == 0L)
-            pageId = cctx.shared().database().pageMemory().allocatePage(cctx.cacheId(), INDEX_PARTITION, FLAG_IDX);
+            pageId = cctx.memoryPolicy().pageMemory().allocatePage(cctx.cacheId(), INDEX_PARTITION, FLAG_IDX);
 
         return pageId;
     }
@@ -663,7 +663,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
     /** {@inheritDoc} */
     @Override public ReuseList reuseListForIndex(String idxName) {
-        return cctx.shared().database().globalReuseList();
+        return cctx.reuseList();
     }
 
     /** {@inheritDoc} */
@@ -736,21 +736,16 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
      */
     protected CacheDataStore createCacheDataStore0(int p)
         throws IgniteCheckedException {
-        IgniteCacheDatabaseSharedManager dbMgr = cctx.shared().database();
-
         final long rootPage = allocateForTree();
 
-        FreeList freeList = cctx.shared().database().globalFreeList();
-
-        CacheDataRowStore rowStore = new CacheDataRowStore(cctx, freeList);
+        CacheDataRowStore rowStore = new CacheDataRowStore(cctx, cctx.freeList());
 
         String idxName = treeName(p);
 
         CacheDataTree dataTree = new CacheDataTree(idxName,
-            cctx.shared().database().globalReuseList(),
+            cctx.reuseList(),
             rowStore,
             cctx,
-            dbMgr.pageMemory(),
             rootPage,
             true);
 
@@ -1408,7 +1403,6 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param reuseList Reuse list.
          * @param rowStore Row store.
          * @param cctx Context.
-         * @param pageMem Page memory.
          * @param metaPageId Meta page ID.
          * @param initNew Initialize new index.
          * @throws IgniteCheckedException If failed.
@@ -1418,12 +1412,18 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             ReuseList reuseList,
             CacheDataRowStore rowStore,
             GridCacheContext cctx,
-            PageMemory pageMem,
             long metaPageId,
             boolean initNew
         ) throws IgniteCheckedException {
-            super(name, cctx.cacheId(), pageMem, cctx.shared().wal(), cctx.offheap().globalRemoveId(), metaPageId,
-                reuseList, DataInnerIO.VERSIONS, DataLeafIO.VERSIONS);
+            super(name,
+                    cctx.cacheId(),
+                    cctx.memoryPolicy().pageMemory(),
+                    cctx.shared().wal(),
+                    cctx.offheap().globalRemoveId(),
+                    metaPageId,
+                    reuseList,
+                    DataInnerIO.VERSIONS,
+                    DataLeafIO.VERSIONS);
 
             assert rowStore != null;
 
@@ -1472,7 +1472,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         private int compareKeys(KeyCacheObject key, final long link) throws IgniteCheckedException {
             byte[] bytes = key.valueBytes(cctx.cacheObjectContext());
 
-            PageMemory pageMem = cctx.shared().database().pageMemory();
+            PageMemory pageMem = cctx.memoryPolicy().pageMemory();
 
             try (Page page = page(pageId(link))) {
                 long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
index 5a62e75..72b7d02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
@@ -104,7 +104,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         boolean first = true;
 
         do {
-            PageMemory pageMem = cctx.shared().database().pageMemory();
+            PageMemory pageMem = cctx.memoryPolicy().pageMemory();
 
             try (Page page = page(pageId(nextLink), cctx)) {
                 long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
@@ -460,7 +460,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @throws IgniteCheckedException If failed.
      */
     private Page page(final long pageId, final GridCacheContext cctx) throws IgniteCheckedException {
-        return cctx.shared().database().pageMemory().page(cctx.cacheId(), pageId);
+        return cctx.memoryPolicy().pageMemory().page(cctx.cacheId(), pageId);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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 7fb9d0a..c1c2e67 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
@@ -19,11 +19,15 @@ package org.apache.ignite.internal.processors.cache.database;
 
 import java.io.File;
 import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
@@ -46,11 +50,26 @@ import org.jetbrains.annotations.Nullable;
  *
  */
 public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdapter implements IgniteChangeGlobalStateSupport {
+    /** MemoryPolicyConfiguration name reserved for internal caches. */
+    private static final String SYSTEM_MEMORY_POLICY_NAME = "sysMemPlc";
+
+    /** Minimum size of memory chunk */
+    private static final long MIN_PAGE_MEMORY_SIZE = 1024 * 1024;
+
+    /** */
+    protected Map<String, MemoryPolicy> memPlcMap;
+
+    /** */
+    protected MemoryPolicy dfltMemPlc;
+
+    /** */
+    private Map<String, FreeListImpl> freeListMap;
+
     /** */
-    protected PageMemory pageMem;
+    private FreeListImpl dfltFreeList;
 
     /** */
-    private FreeListImpl freeList;
+    private int pageSize;
 
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
@@ -62,33 +81,203 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @throws IgniteCheckedException If failed.
      */
     public void init() throws IgniteCheckedException {
-        if (pageMem == null) {
+        if (memPlcMap == null) {
             MemoryConfiguration dbCfg = cctx.kernalContext().config().getMemoryConfiguration();
 
             if (dbCfg == null)
                 dbCfg = new MemoryConfiguration();
 
-            pageMem = initMemory(dbCfg);
+            validateConfiguration(dbCfg);
+
+            pageSize = dbCfg.getPageSize();
+
+            initPageMemoryPolicies(dbCfg);
 
-            pageMem.start();
+            startPageMemoryPools();
 
-            initDataStructures();
+            initPageMemoryDataStructures(dbCfg);
         }
     }
 
     /**
-     * @param log Logger.
+     * @param dbCfg Database config.
      */
-    public void dumpStatistics(IgniteLogger log) {
-        if (freeList != null)
-            freeList.dumpStatistics(log);
+    protected void initPageMemoryDataStructures(MemoryConfiguration dbCfg) throws IgniteCheckedException {
+        freeListMap = U.newHashMap(memPlcMap.size());
+
+        String dfltMemPlcName = dbCfg.getDefaultMemoryPolicyName();
+
+        for (MemoryPolicy memPlc : memPlcMap.values()) {
+            MemoryPolicyConfiguration memPlcCfg = memPlc.config();
+
+            FreeListImpl freeList = new FreeListImpl(0,
+                    cctx.igniteInstanceName(),
+                    memPlc.pageMemory(),
+                    null,
+                    cctx.wal(),
+                    0L,
+                    true);
+
+            freeListMap.put(memPlcCfg.getName(), freeList);
+        }
+
+        dfltFreeList = freeListMap.get(dfltMemPlcName);
     }
 
     /**
-     * @throws IgniteCheckedException If failed.
+     * @return Size of page used for PageMemory regions.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+
+    /**
+     *
+     */
+    private void startPageMemoryPools() {
+        for (MemoryPolicy memPlc : memPlcMap.values())
+            memPlc.pageMemory().start();
+    }
+
+    /**
+     * @param dbCfg Database config.
+     */
+    protected void initPageMemoryPolicies(MemoryConfiguration dbCfg) {
+        MemoryPolicyConfiguration[] memPlcsCfgs = dbCfg.getMemoryPolicies();
+
+        if (memPlcsCfgs == null) {
+            //reserve place for default and system memory policies
+            memPlcMap = U.newHashMap(2);
+
+            dfltMemPlc = createDefaultMemoryPolicy(dbCfg);
+            memPlcMap.put(null, dfltMemPlc);
+
+            log.warning("No user-defined default MemoryPolicy found; system default of 1GB size will be used.");
+        }
+        else {
+            String dfltMemPlcName = dbCfg.getDefaultMemoryPolicyName();
+
+            if (dfltMemPlcName == null) {
+                //reserve additional place for default and system memory policies
+                memPlcMap = U.newHashMap(memPlcsCfgs.length + 2);
+
+                dfltMemPlc = createDefaultMemoryPolicy(dbCfg);
+                memPlcMap.put(null, dfltMemPlc);
+
+                log.warning("No user-defined default MemoryPolicy found; system default of 1GB size will be used.");
+            }
+            else
+                //reserve additional place for system memory policy only
+                memPlcMap = U.newHashMap(memPlcsCfgs.length + 1);
+
+            for (MemoryPolicyConfiguration memPlcCfg : memPlcsCfgs) {
+                PageMemory pageMem = initMemory(dbCfg, memPlcCfg);
+
+                MemoryPolicy memPlc = new MemoryPolicy(pageMem, memPlcCfg);
+
+                memPlcMap.put(memPlcCfg.getName(), memPlc);
+
+                if (memPlcCfg.getName().equals(dfltMemPlcName))
+                    dfltMemPlc = memPlc;
+            }
+        }
+
+        MemoryPolicyConfiguration sysPlcCfg = createSystemMemoryPolicy(dbCfg.getSystemCacheMemorySize());
+
+        memPlcMap.put(SYSTEM_MEMORY_POLICY_NAME, new MemoryPolicy(initMemory(dbCfg, sysPlcCfg), sysPlcCfg));
+    }
+
+    /**
+     * @param dbCfg Database config.
+     */
+    private MemoryPolicy createDefaultMemoryPolicy(MemoryConfiguration dbCfg) {
+        MemoryPolicyConfiguration dfltPlc = dbCfg.createDefaultPolicyConfig();
+
+        PageMemory pageMem = initMemory(dbCfg, dfltPlc);
+
+        return new MemoryPolicy(pageMem, dfltPlc);
+    }
+
+    /**
+     * @param sysCacheMemSize size of PageMemory to be created for system cache.
+     */
+    private MemoryPolicyConfiguration createSystemMemoryPolicy(long sysCacheMemSize) {
+        MemoryPolicyConfiguration res = new MemoryPolicyConfiguration();
+
+        res.setName(SYSTEM_MEMORY_POLICY_NAME);
+        res.setSize(sysCacheMemSize);
+
+        return res;
+    }
+
+    /**
+     * @param dbCfg configuration to validate.
+     */
+    private void validateConfiguration(MemoryConfiguration dbCfg) throws IgniteCheckedException {
+        MemoryPolicyConfiguration[] plcCfgs = dbCfg.getMemoryPolicies();
+
+        Set<String> plcNames = (plcCfgs != null) ? U.<String>newHashSet(plcCfgs.length) : new HashSet<String>(0);
+
+        if (plcCfgs != null) {
+            for (MemoryPolicyConfiguration plcCfg : plcCfgs) {
+                assert plcCfg != null;
+
+                checkPolicyName(plcCfg.getName(), plcNames);
+
+                checkPolicySize(plcCfg);
+            }
+        }
+
+        checkDefaultPolicyConfiguration(dbCfg.getDefaultMemoryPolicyName(), plcNames);
+    }
+
+    /**
+     * @param dfltPlcName Default MemoryPolicy name.
+     * @param plcNames All MemoryPolicy names.
+     * @throws IgniteCheckedException In case of validation violation.
+     */
+    private static void checkDefaultPolicyConfiguration(String dfltPlcName, Set<String> plcNames) throws IgniteCheckedException {
+        if (dfltPlcName != null) {
+            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);
+        }
+    }
+
+    /**
+     * @param plcCfg MemoryPolicyConfiguration to validate.
      */
-    protected void initDataStructures() throws IgniteCheckedException {
-        freeList = new FreeListImpl(0, cctx.igniteInstanceName(), pageMem, null, cctx.wal(), 0L, true);
+    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());
+    }
+
+    /**
+     * @param plcName MemoryPolicy name to validate.
+     * @param observedNames Names of MemoryPolicies observed before.
+     */
+    private static void checkPolicyName(String plcName, Set<String> observedNames) throws IgniteCheckedException {
+        if (plcName == null || plcName.isEmpty())
+            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);
+
+        if (SYSTEM_MEMORY_POLICY_NAME.equals(plcName))
+            throw new IgniteCheckedException("'sysMemPlc' policy name is reserved for internal use.");
+
+        observedNames.add(plcName);
+    }
+
+    /**
+     * @param log Logger.
+     */
+    public void dumpStatistics(IgniteLogger log) {
+        if (freeListMap != null) {
+            for (FreeListImpl freeList : freeListMap.values())
+                freeList.dumpStatistics(log);
+        }
     }
 
     /**
@@ -99,27 +288,60 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @return Node-global free list.
+     * @return collection of all configured {@link MemoryPolicy policies}.
      */
-    public FreeList globalFreeList() {
-        assert freeList != null : "Non initialized";
+    public Collection<MemoryPolicy> memoryPolicies() {
+        return memPlcMap != null ? memPlcMap.values() : null;
+    }
+
+    /**
+     * @param memPlcName Memory policy name.
+     * @return {@link MemoryPolicy} instance associated with a given {@link MemoryPolicyConfiguration}.
+     * @throws IgniteCheckedException in case of request for unknown MemoryPolicy.
+     */
+    public MemoryPolicy memoryPolicy(String memPlcName) throws IgniteCheckedException {
+        if (memPlcName == null)
+            return dfltMemPlc;
+
+        if (memPlcMap == null)
+            return null;
 
-        return freeList;
+        MemoryPolicy plc;
+
+        if ((plc = memPlcMap.get(memPlcName)) == null)
+            throw new IgniteCheckedException("Requested MemoryPolicy is not configured: " + memPlcName);
+
+        return plc;
     }
 
     /**
-     * @return Node-global reuse list.
+     * @param memPlcName MemoryPolicyConfiguration name.
+     * @return {@link FreeList} instance associated with a given {@link MemoryPolicyConfiguration}.
      */
-    public ReuseList globalReuseList() {
-        assert freeList != null : "Non initialized";
+    public FreeList freeList(String memPlcName) {
+        if (memPlcName == null)
+            return dfltFreeList;
 
-        return freeList;
+        return freeListMap != null ? freeListMap.get(memPlcName) : null;
+    }
+
+    /**
+     * @param memPlcName MemoryPolicyConfiguration name.
+     * @return {@link ReuseList} instance associated with a given {@link MemoryPolicyConfiguration}.
+     */
+    public ReuseList reuseList(String memPlcName) {
+        if (memPlcName == null)
+            return dfltFreeList;
+
+        return freeListMap != null ? freeListMap.get(memPlcName) : null;
     }
 
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
-        if (pageMem != null)
-            pageMem.stop();
+        if (memPlcMap != null) {
+            for (MemoryPolicy memPlc : memPlcMap.values())
+                memPlc.pageMemory().stop();
+        }
     }
 
     /**
@@ -130,13 +352,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @return Page memory instance.
-     */
-    public PageMemory pageMemory() {
-        return pageMem;
-    }
-
-    /**
      *
      */
     public void lock() throws IgniteCheckedException {
@@ -230,42 +445,76 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param dbCfg Database configuration.
+     * @param dbCfg memory configuration with common parameters.
+     * @param plc memory policy with PageMemory specific parameters.
      * @return Page memory instance.
      */
-    protected PageMemory initMemory(MemoryConfiguration dbCfg) {
-        String path = dbCfg.getFileCacheAllocationPath();
+    private PageMemory initMemory(MemoryConfiguration dbCfg, MemoryPolicyConfiguration plc) {
+        long[] sizes = calculateFragmentSizes(
+                dbCfg.getConcurrencyLevel(),
+                plc.getSize());
 
-        int concLvl = dbCfg.getConcurrencyLevel();
+        File allocPath = buildAllocPath(plc);
 
+        DirectMemoryProvider memProvider = allocPath == null ?
+            new UnsafeMemoryProvider(sizes) :
+            new MappedFileMemoryProvider(
+                log,
+                allocPath,
+                true,
+                sizes);
+
+        return createPageMemory(memProvider, dbCfg.getPageSize());
+    }
+
+    /**
+     * 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 = dbCfg.getPageCacheSize() / concLvl;
+        long fragmentSize = cacheSize / concLvl;
 
         if (fragmentSize < 1024 * 1024)
             fragmentSize = 1024 * 1024;
 
-        String consId = String.valueOf(cctx.discovery().consistentId());
-
-        consId = consId.replaceAll("[:,\\.]", "_");
-
-        File allocPath = path == null ? null : buildPath(path, consId);
-
         long[] sizes = new long[concLvl];
 
         for (int i = 0; i < concLvl; i++)
             sizes[i] = fragmentSize;
 
-        DirectMemoryProvider memProvider = path == null ?
-            new UnsafeMemoryProvider(sizes) :
-            new MappedFileMemoryProvider(
-                log,
-                allocPath,
-                true,
-                sizes);
+        return sizes;
+    }
 
-        return new PageMemoryNoStoreImpl(log, memProvider, cctx, dbCfg.getPageSize(), false);
+    /**
+     * Builds allocation path for memory mapped file to be used with PageMemory.
+     *
+     * @param plc MemoryPolicyConfiguration.
+     */
+    @Nullable protected File buildAllocPath(MemoryPolicyConfiguration plc) {
+        String path = plc.getSwapFilePath();
+
+        if (path == null)
+            return null;
+
+        String consId = String.valueOf(cctx.discovery().consistentId());
+
+        consId = consId.replaceAll("[:,\\.]", "_");
+
+        return buildPath(path, consId);
+    }
+
+    /**
+     * Creates PageMemory with given size and memory provider.
+     *
+     * @param memProvider Memory provider.
+     * @param pageSize Page size.
+     */
+    protected PageMemory createPageMemory(DirectMemoryProvider memProvider, int pageSize) {
+        return new PageMemoryNoStoreImpl(log, memProvider, cctx, pageSize, false);
     }
 
     /**
@@ -292,4 +541,11 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
 
     }
+
+    /**
+     * @return Name of MemoryPolicyConfiguration for internal caches.
+     */
+    public String systemMemoryPolicyName() {
+        return SYSTEM_MEMORY_POLICY_NAME;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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
new file mode 100644
index 0000000..0978d10
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryPolicy.java
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.internal.pagemem.PageMemory;
+
+/**
+ * Memory policy provides access to objects configured with {@link MemoryPolicyConfiguration} configuration.
+ */
+public class MemoryPolicy {
+    /** */
+    private final PageMemory pageMem;
+
+    /** */
+    private final MemoryPolicyConfiguration cfg;
+
+    /**
+     * @param pageMem Page mem.
+     */
+    public MemoryPolicy(PageMemory pageMem, MemoryPolicyConfiguration cfg) {
+        this.pageMem = pageMem;
+        this.cfg = cfg;
+    }
+
+    /**
+     *
+     */
+    public PageMemory pageMemory() {
+        return pageMem;
+    }
+
+    /**
+     * @return Config.
+     */
+    public MemoryPolicyConfiguration config() {
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
index 8d54542..6d1fc0e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
@@ -52,7 +52,7 @@ public class RowStore {
         this.freeList = freeList;
 
         coctx = cctx.cacheObjectContext();
-        pageMem = cctx.shared().database().pageMemory();
+        pageMem = cctx.memoryPolicy().pageMemory();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 750a99d..c257154 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -1567,7 +1567,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements AutoClosea
                 {
                     if (log.isDebugEnabled())
                         log.debug("Acquired transaction lock for remove on keys: " + enlisted);
-                    U.debug(log, "Acquired transaction lock for remove on keys: " + enlisted);
 
                     postLockWrite(cacheCtx,
                         enlisted,

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
index d20e074..0ce13df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
@@ -112,7 +112,7 @@ public class VisorGridConfiguration implements Serializable, LessNamingBean {
         sysProps = IgniteSystemProperties.snapshot();
         atomic = VisorAtomicConfiguration.from(c.getAtomicConfiguration());
         txCfg = VisorTransactionConfiguration.from(c.getTransactionConfiguration());
-        memCfg = VisorMemoryConfiguration.from(c.getMemoryConfiguration());
+        memCfg = new VisorMemoryConfiguration(c.getMemoryConfiguration());
 
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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 489aad2..2080105 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
@@ -18,19 +18,20 @@
 package org.apache.ignite.internal.visor.node;
 
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
 import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.LessNamingBean;
+import org.apache.ignite.internal.util.typedef.F;
 
 /**
  * Data transfer object for memory configuration.
  */
-public class VisorMemoryConfiguration implements Serializable, LessNamingBean {
+public class VisorMemoryConfiguration implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Concurrency level. */
-    private int concLvl;
-
     /** File cache allocation path. */
     private String fileCacheAllocationPath;
 
@@ -40,27 +41,41 @@ public class VisorMemoryConfiguration implements Serializable, LessNamingBean {
     /** Page size. */
     private int pageSize;
 
+    /** Concurrency level. */
+    private int concLvl;
+
+    /** Name of MemoryPolicy to be used as default. */
+    private String dfltMemPlcName;
+
+    /** Memory policies. */
+    private List<VisorMemoryPolicyConfiguration> memPlcs;
+
     /**
      * Create data transfer object.
      *
      * @param memCfg Memory configuration.
-     * @return Data transfer object.
      */
-    public static VisorMemoryConfiguration from(MemoryConfiguration memCfg) {
-        VisorMemoryConfiguration res = new VisorMemoryConfiguration();
+    public VisorMemoryConfiguration(MemoryConfiguration memCfg) {
+        assert memCfg != null;
+
+        pageSize = memCfg.getPageSize();
+        concLvl = memCfg.getConcurrencyLevel();
+        dfltMemPlcName = memCfg.getDefaultMemoryPolicyName();
 
-        res.concLvl = memCfg.getConcurrencyLevel();
-        res.fileCacheAllocationPath = memCfg.getFileCacheAllocationPath();
-        res.pageCacheSize = memCfg.getPageCacheSize();
-        res.pageSize = memCfg.getPageSize();
+        MemoryPolicyConfiguration[] plcs = memCfg.getMemoryPolicies();
 
-        return res;
+        if (!F.isEmpty(plcs)) {
+            memPlcs = new ArrayList<>(plcs.length);
+
+            for (MemoryPolicyConfiguration plc : plcs)
+                memPlcs.add(new VisorMemoryPolicyConfiguration(plc));
+        }
     }
 
     /**
      * @return Concurrency level.
      */
-    public int concurrencyLevel() {
+    public int getConcurrencyLevel() {
         return concLvl;
     }
 
@@ -84,4 +99,18 @@ public class VisorMemoryConfiguration implements Serializable, LessNamingBean {
     public int getPageSize() {
         return pageSize;
     }
+
+    /**
+     * @return Name of MemoryPolicy to be used as default.
+     */
+    public String getDefaultMemoryPolicyName() {
+        return dfltMemPlcName;
+    }
+
+    /**
+     * @return Collection of MemoryPolicyConfiguration objects.
+     */
+    public List<VisorMemoryPolicyConfiguration> getMemoryPolicies() {
+        return memPlcs;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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
new file mode 100644
index 0000000..28eebd2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
@@ -0,0 +1,72 @@
+/*
+ * 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.visor.node;
+
+import java.io.Serializable;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+
+/**
+ * Data transfer object for memory configuration.
+ */
+public class VisorMemoryPolicyConfiguration implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Unique name of MemoryPolicy. */
+    private String name;
+
+    /** Page memory size in bytes. */
+    private long size;
+
+    /** Path for memory mapped file. */
+    private String swapFilePath;
+
+    /**
+     * Constructor.
+     *
+     * @param plc Memory policy configuration.
+     */
+    public VisorMemoryPolicyConfiguration(MemoryPolicyConfiguration plc) {
+        assert plc != null;
+
+        name = plc.getName();
+        size = plc.getSize();
+        swapFilePath = plc.getSwapFilePath();
+    }
+
+    /**
+     * Unique name of MemoryPolicy.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * Page memory size in bytes.
+     */
+    public long getSize() {
+        return size;
+    }
+
+    /**
+     * @return Path for memory mapped file.
+     */
+    public String getSwapFilePath() {
+        return swapFilePath;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 3a26c97..988d889 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1800,6 +1800,7 @@ org.apache.ignite.internal.visor.node.VisorGridConfiguration
 org.apache.ignite.internal.visor.node.VisorIgfsConfiguration
 org.apache.ignite.internal.visor.node.VisorLifecycleConfiguration
 org.apache.ignite.internal.visor.node.VisorMemoryConfiguration
+org.apache.ignite.internal.visor.node.VisorMemoryPolicyConfiguration
 org.apache.ignite.internal.visor.node.VisorMetricsConfiguration
 org.apache.ignite.internal.visor.node.VisorNodeConfigurationCollectorJob
 org.apache.ignite.internal.visor.node.VisorNodeConfigurationCollectorTask

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/test/java/org/apache/ignite/cache/LargeEntryUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/LargeEntryUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/cache/LargeEntryUpdateTest.java
index 18a1654..592ba0f 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/LargeEntryUpdateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/LargeEntryUpdateTest.java
@@ -74,7 +74,6 @@ public class LargeEntryUpdateTest extends GridCommonAbstractTest {
         MemoryConfiguration mem = new MemoryConfiguration();
 
         mem.setPageSize(PAGE_SIZE);
-        mem.setPageCacheSize(PAGE_CACHE_SIZE);
 
         cfg.setMemoryConfiguration(mem);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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 918a3b0..820cff3 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
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 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.lang.IgniteInClosure;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -44,7 +45,11 @@ public class CacheConfigurationLeakTest extends GridCommonAbstractTest {
 
         MemoryConfiguration memCfg = new MemoryConfiguration();
 
-        memCfg.setPageCacheSize(MemoryConfiguration.DFLT_PAGE_CACHE_SIZE * 10);
+        MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
+
+        plc.setSize(MemoryConfiguration.DFLT_MEMORY_POLICY_SIZE * 10);
+
+        memCfg.setMemoryPolicies(plc);
 
         cfg.setMemoryConfiguration(memCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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
new file mode 100644
index 0000000..630d4be
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMemoryPolicyConfigurationTest.java
@@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+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.IgniteEx;
+import org.apache.ignite.internal.mem.OutOfMemoryException;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class CacheMemoryPolicyConfigurationTest extends GridCommonAbstractTest {
+    /** */
+    private volatile CacheConfiguration ccfg;
+
+    /** */
+    private volatile MemoryConfiguration memCfg;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (memCfg != null)
+            cfg.setMemoryConfiguration(memCfg);
+
+        if (ccfg != null)
+            cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Verifies that proper exception is thrown when MemoryPolicy is misconfigured for cache.
+     */
+    public void testMissingMemoryPolicy() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setMemoryPolicyName("nonExistingMemPlc");
+
+        try {
+            startGrid(0);
+        }
+        catch (IgniteCheckedException e) {
+            String msg = e.getMessage();
+
+            assertTrue("Not expected exception was thrown: " + e, msg.contains("Requested MemoryPolicy is not configured"));
+
+            return;
+        }
+
+        fail("Expected exception was not thrown: missing MemoryPolicy");
+    }
+
+    /**
+     * Verifies that {@link OutOfMemoryException} is thrown when cache is configured with too small MemoryPolicy.
+     */
+    public void testTooSmallMemoryPolicy() throws Exception {
+        memCfg = new MemoryConfiguration();
+
+        MemoryPolicyConfiguration dfltPlcCfg = new MemoryPolicyConfiguration();
+        dfltPlcCfg.setName("dfltPlc");
+        dfltPlcCfg.setSize(1024 * 1024);
+
+        MemoryPolicyConfiguration bigPlcCfg = new MemoryPolicyConfiguration();
+        bigPlcCfg.setName("bigPlc");
+        bigPlcCfg.setSize(1024 * 1024 * 1024);
+
+        memCfg.setMemoryPolicies(dfltPlcCfg, bigPlcCfg);
+        memCfg.setDefaultMemoryPolicyName("dfltPlc");
+
+        ccfg = new CacheConfiguration();
+
+        IgniteEx ignite0 = startGrid(0);
+
+        IgniteCache<Object, Object> cache = ignite0.cache(null);
+
+        boolean oomeThrown = false;
+
+        try {
+            for (int i = 0; i < 500_000; i++)
+                cache.put(i, "abc");
+        }
+        catch (Exception e) {
+            Throwable cause = e;
+
+            do {
+                if (cause instanceof OutOfMemoryException) {
+                    oomeThrown = true;
+                    break;
+                }
+
+                if (cause == null)
+                    break;
+
+                if (cause.getSuppressed() == null || cause.getSuppressed().length == 0)
+                    cause = cause.getCause();
+                else
+                    cause = cause.getSuppressed()[0];
+            }
+            while (true);
+        }
+
+        if (!oomeThrown)
+            fail("OutOfMemoryException hasn't been thrown");
+    }
+
+    /**
+     * Verifies that with enough memory allocated adding values to cache doesn't cause any exceptions.
+     */
+    public void testProperlySizedMemoryPolicy() throws Exception {
+        memCfg = new MemoryConfiguration();
+
+        MemoryPolicyConfiguration dfltPlcCfg = new MemoryPolicyConfiguration();
+        dfltPlcCfg.setName("dfltPlc");
+        dfltPlcCfg.setSize(1024 * 1024);
+
+        MemoryPolicyConfiguration bigPlcCfg = new MemoryPolicyConfiguration();
+        bigPlcCfg.setName("bigPlc");
+        bigPlcCfg.setSize(1024 * 1024 * 1024);
+
+        memCfg.setMemoryPolicies(dfltPlcCfg, bigPlcCfg);
+        memCfg.setDefaultMemoryPolicyName("dfltPlc");
+
+        ccfg = new CacheConfiguration();
+        ccfg.setMemoryPolicyName("bigPlc");
+
+        IgniteEx ignite0 = startGrid(0);
+
+        IgniteCache<Object, Object> cache = ignite0.cache(null);
+
+        try {
+            for (int i = 0; i < 500_000; i++)
+                cache.put(i, "abc");
+        }
+        catch (Exception e) {
+            fail("With properly sized MemoryPolicy no exceptions are expected to be thrown.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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
new file mode 100644
index 0000000..6b93936
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
+    /** Configuration violation type to check. */
+    private ValidationViolationType violationType;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        MemoryConfiguration memCfg = new MemoryConfiguration();
+
+        MemoryPolicyConfiguration[] plcs = null;
+
+        switch (violationType) {
+            case NAMES_CONFLICT:
+                plcs = createPlcsWithNamesConflictCfg();
+
+                break;
+
+            case RESERVED_MEMORY_POLICY_MISUSE:
+                plcs = createPlcWithReservedNameMisuseCfg();
+
+                break;
+
+            case TOO_SMALL_MEMORY_SIZE:
+                plcs = createTooSmallMemoryCfg();
+
+                break;
+
+            case NULL_NAME_ON_USER_DEFINED_POLICY:
+                plcs = createPlcWithNullName();
+
+                break;
+
+            case MISSING_USER_DEFINED_DEFAULT:
+                plcs = createMissingUserDefinedDefault();
+
+                memCfg.setDefaultMemoryPolicyName("missingMemoryPolicyName");
+
+                break;
+        }
+
+        memCfg.setMemoryPolicies(plcs);
+
+        cfg.setMemoryConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    private MemoryPolicyConfiguration[] createMissingUserDefinedDefault() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy("presentedPolicyCfg", 10 * 1024 * 1024);
+
+        return res;
+    }
+
+    private MemoryPolicyConfiguration[] createPlcWithNullName() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy(null, 10 * 1024 * 1024);
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private MemoryPolicyConfiguration[] createTooSmallMemoryCfg() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy("dflt", 10);
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private MemoryPolicyConfiguration[] createPlcWithReservedNameMisuseCfg() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy("sysMemPlc", 1024 * 1024);
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private MemoryPolicyConfiguration[] createPlcsWithNamesConflictCfg() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[2];
+
+        res[0] = createMemoryPolicy("cflt0", 1024 * 1024);
+        res[1] = createMemoryPolicy("cflt0", 1024 * 1024);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @param name Name of MemoryPolicyConfiguration.
+     * @param size Size of MemoryPolicyConfiguration in bytes.
+     */
+    private MemoryPolicyConfiguration createMemoryPolicy(String name, long size) {
+        MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
+
+        plc.setName(name);
+        plc.setSize(size);
+
+        return plc;
+    }
+
+    /**
+     * 'sysMemPlc' name is reserved for MemoryPolicyConfiguration for system caches.
+     */
+    public void testReservedMemoryPolicyMisuse() throws Exception {
+        violationType = ValidationViolationType.RESERVED_MEMORY_POLICY_MISUSE;
+
+        doTest(violationType);
+    }
+
+    /**
+     * If user defines default is must be presented among configured memory policies.
+     */
+    public void testMissingUserDefinedDefault() throws Exception {
+        violationType = ValidationViolationType.MISSING_USER_DEFINED_DEFAULT;
+
+        doTest(violationType);
+    }
+
+    /**
+     * Names of all MemoryPolicies must be distinct.
+     */
+    public void testNamesConflict() throws Exception {
+        violationType = ValidationViolationType.NAMES_CONFLICT;
+
+        doTest(violationType);
+    }
+
+    /**
+     * User-defined policy must have a non-null non-empty name.
+     */
+    public void testNullNameOnUserDefinedPolicy() throws Exception {
+        violationType = ValidationViolationType.NULL_NAME_ON_USER_DEFINED_POLICY;
+
+        doTest(violationType);
+    }
+
+    /**
+     * MemoryPolicy must be configured with size of at least 1MB.
+     */
+    public void testMemoryTooSmall() throws Exception {
+        violationType = ValidationViolationType.TOO_SMALL_MEMORY_SIZE;
+
+        doTest(violationType);
+    }
+
+    /**
+     * Tries to start ignite node with invalid configuration and checks that corresponding exception is thrown.
+     *
+     * @param violationType Configuration violation type.
+     */
+    private void doTest(ValidationViolationType violationType) throws Exception {
+        try {
+            startGrid(0);
+        }
+        catch (IgniteCheckedException e) {
+            Throwable c = e.getCause();
+
+            assertTrue(c != null);
+            assertTrue(c.getMessage().contains(violationType.violationMsg));
+
+            return;
+        }
+
+        fail("Expected exception hasn't been thrown");
+    }
+
+    /**
+     *
+     */
+    private enum ValidationViolationType {
+        /** */
+        NAMES_CONFLICT("Two MemoryPolicies have the same name: "),
+
+        /** */
+        RESERVED_MEMORY_POLICY_MISUSE("'sysMemPlc' policy name is reserved for internal use."),
+
+        /** */
+        TOO_SMALL_MEMORY_SIZE("MemoryPolicy must have size more than 1MB: "),
+
+        /** */
+        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 is not presented among configured MemoryPolicies: ");
+
+        /**
+         * @param violationMsg Violation message.
+         */
+        ValidationViolationType(String violationMsg) {
+            this.violationMsg = violationMsg;
+        }
+
+        /** */
+        String violationMsg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
index cf26187..f9c63ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
@@ -68,8 +68,6 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
         else
             dbCfg.setPageSize(1024);
 
-        dbCfg.setPageCacheSize(200 * 1024 * 1024);
-
         configure(dbCfg);
 
         cfg.setMemoryConfiguration(dbCfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/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 a9e0541..b7a8bf3 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
@@ -27,6 +27,7 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -40,7 +41,13 @@ public class IgniteDbDynamicCacheSelfTest extends GridCommonAbstractTest {
 
         MemoryConfiguration dbCfg = new MemoryConfiguration();
 
-        dbCfg.setPageCacheSize(200 * 1024 * 1024);
+        MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
+
+        plc.setName("dfltPlc");
+        plc.setSize(200 * 1024 * 1024);
+
+        dbCfg.setDefaultMemoryPolicyName("dfltPlc");
+        dbCfg.setMemoryPolicies(plc);
 
         cfg.setMemoryConfiguration(dbCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
index 2e9d3f9..5228955 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
@@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.database.DataStructure;
 
@@ -35,9 +36,6 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
     private static final int CONCURRENCY_LEVEL = 8;
 
     /** */
-    private static final int MIN_PAGE_CACHE_SIZE = 1048576 * CONCURRENCY_LEVEL;
-
-    /** */
     private volatile Exception ex;
 
     /** */
@@ -76,10 +74,6 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
     /** {@inheritDoc} */
     @Override protected void configure(MemoryConfiguration mCfg) {
         mCfg.setConcurrencyLevel(CONCURRENCY_LEVEL);
-
-        long size = (1024 * (isLargePage() ? 16 : 1) + 24) * pagesMax();
-
-        mCfg.setPageCacheSize(Math.max(size, MIN_PAGE_CACHE_SIZE));
     }
 
     /**
@@ -231,7 +225,7 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
      * @throws Exception If failed.
      */
     protected void check(IgniteEx ig) throws Exception {
-        long pagesActual = ig.context().cache().context().database().pageMemory().loadedPages();
+        long pagesActual = ig.context().cache().context().database().memoryPolicy(null).pageMemory().loadedPages();
 
         if (loadedPages > 0) {
             delta += pagesActual - loadedPages;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ac0fd0c/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index cec9e30..0f4aa87 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -80,6 +80,9 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
             CacheType.USER,
             true,
             true,
+            null,
+            null,
+            null,
             new GridCacheEventManager(),
             new CacheOsStoreManager(null, new CacheConfiguration()),
             new GridCacheEvictionManager(),