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/10/20 07:32:17 UTC

[08/11] ignite git commit: IGNITE-6030 Allow enabling persistence per data region

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 9a2e028..de3b60a 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -57,18 +57,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import javax.management.ObjectName;
+import org.apache.ignite.DataStorageMetrics;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.PersistenceMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CheckpointWriteOrder;
 import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
-import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridKernalContext;
@@ -139,7 +138,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteOutClosure;
-import org.apache.ignite.mxbean.PersistenceMetricsMXBean;
+import org.apache.ignite.mxbean.DataStorageMetricsMXBean;
 import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.NotNull;
@@ -229,7 +228,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     };
 
     /** */
-    private static final String MBEAN_NAME = "PersistenceMetrics";
+    private static final String MBEAN_NAME = "DataStorageMetrics";
 
     /** */
     private static final String MBEAN_GROUP = "Persistent Store";
@@ -262,7 +261,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private volatile boolean printCheckpointStats = true;
 
     /** Database configuration. */
-    private final PersistentStoreConfiguration persistenceCfg;
+    private final DataStorageConfiguration persistenceCfg;
 
     /** */
     private final Collection<DbCheckpointListener> lsnrs = new CopyOnWriteArrayList<>();
@@ -301,7 +300,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private IgniteCacheSnapshotManager snapshotMgr;
 
     /** */
-    private PersistenceMetricsImpl persStoreMetrics;
+    private DataStorageMetricsImpl persStoreMetrics;
 
     /** */
     private ObjectName persistenceMetricsMbeanName;
@@ -318,18 +317,18 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     public GridCacheDatabaseSharedManager(GridKernalContext ctx) {
         IgniteConfiguration cfg = ctx.config();
 
-        persistenceCfg = cfg.getPersistentStoreConfiguration();
+        persistenceCfg = cfg.getDataStorageConfiguration();
 
-        assert persistenceCfg != null : "PageStore should not be created if persistence is disabled.";
+        assert persistenceCfg != null;
 
-        checkpointFreq = persistenceCfg.getCheckpointingFrequency();
+        checkpointFreq = persistenceCfg.getCheckpointFrequency();
 
         lockWaitTime = persistenceCfg.getLockWaitTime();
 
-        persStoreMetrics = new PersistenceMetricsImpl(
+        persStoreMetrics = new DataStorageMetricsImpl(
             persistenceCfg.isMetricsEnabled(),
-            persistenceCfg.getRateTimeInterval(),
-            persistenceCfg.getSubIntervals()
+            persistenceCfg.getMetricsRateTimeInterval(),
+            persistenceCfg.getMetricsSubIntervalCount()
         );
     }
 
@@ -400,12 +399,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      *
      */
     private void initDataBase() {
-        if (persistenceCfg.getCheckpointingThreads() > 1)
+        if (persistenceCfg.getCheckpointThreads() > 1)
             asyncRunner = new IgniteThreadPoolExecutor(
                 "checkpoint-runner",
                 cctx.igniteInstanceName(),
-                persistenceCfg.getCheckpointingThreads(),
-                persistenceCfg.getCheckpointingThreads(),
+                persistenceCfg.getCheckpointThreads(),
+                persistenceCfg.getCheckpointThreads(),
                 30_000,
                 new LinkedBlockingQueue<Runnable>()
             );
@@ -420,26 +419,26 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @return Checkpoint buffer size.
      */
     public static long checkpointBufferSize(IgniteConfiguration cfg) {
-        PersistentStoreConfiguration persistenceCfg = cfg.getPersistentStoreConfiguration();
+        DataStorageConfiguration persistenceCfg = cfg.getDataStorageConfiguration();
 
         if (persistenceCfg == null)
             return 0L;
 
-        long res = persistenceCfg.getCheckpointingPageBufferSize();
+        long res = persistenceCfg.getCheckpointPageBufferSize();
 
         if (res == 0L) {
             res = DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
 
-            MemoryConfiguration memCfg = cfg.getMemoryConfiguration();
+            DataStorageConfiguration memCfg = cfg.getDataStorageConfiguration();
 
             assert memCfg != null;
 
-            long totalSize = memCfg.getSystemCacheMaxSize();
+            long totalSize = memCfg.getSystemRegionMaxSize();
 
-            if (memCfg.getMemoryPolicies() == null)
-                totalSize += MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE;
+            if (memCfg.getDataRegionConfigurations() == null)
+                totalSize += DataStorageConfiguration.DFLT_DATA_REGION_MAX_SIZE;
             else {
-                for (MemoryPolicyConfiguration memPlc : memCfg.getMemoryPolicies()) {
+                for (DataRegionConfiguration memPlc : memCfg.getDataRegionConfigurations()) {
                     if (Long.MAX_VALUE - memPlc.getMaxSize() > totalSize)
                         totalSize += memPlc.getMaxSize();
                     else {
@@ -465,11 +464,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /** {@inheritDoc} */
-    @Override protected void initPageMemoryDataStructures(MemoryConfiguration dbCfg) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext ctx) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Activate database manager [id=" + cctx.localNodeId() +
@@ -522,7 +516,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 MBEAN_GROUP,
                 MBEAN_NAME,
                 persStoreMetrics,
-                PersistenceMetricsMXBean.class);
+                DataStorageMetricsMXBean.class);
         }
         catch (Throwable e) {
             throw new IgniteCheckedException("Failed to register " + MBEAN_NAME + " MBean.", e);
@@ -549,14 +543,19 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteOutClosure<Float> fillFactorProvider(final String memPlcName) {
+    @Override protected IgniteOutClosure<Float> fillFactorProvider(final DataRegionConfiguration dataRegCfg) {
+        if (!dataRegCfg.isPersistenceEnabled())
+            return super.fillFactorProvider(dataRegCfg);
+
+        final String dataRegName = dataRegCfg.getName();
+
         return new IgniteOutClosure<Float>() {
             @Override public Float apply() {
                 long loadSize = 0L;
                 long totalSize = 0L;
 
                 for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) {
-                    if (!grpCtx.memoryPolicy().config().getName().equals(memPlcName))
+                    if (!grpCtx.dataRegion().config().getName().equals(dataRegName))
                         continue;
 
                     assert grpCtx.offheap() instanceof GridCacheOffheapManager;
@@ -678,10 +677,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** {@inheritDoc} */
     @Override protected PageMemory createPageMemory(
         DirectMemoryProvider memProvider,
-        MemoryConfiguration memCfg,
-        MemoryPolicyConfiguration plcCfg,
-        MemoryMetricsImpl memMetrics
+        DataStorageConfiguration memCfg,
+        DataRegionConfiguration plcCfg,
+        DataRegionMetricsImpl memMetrics
     ) {
+        if (!plcCfg.isPersistenceEnabled())
+            return super.createPageMemory(memProvider, memCfg, plcCfg, memMetrics);
+
         memMetrics.persistenceEnabled(true);
 
         long cacheSize = plcCfg.getMaxSize();
@@ -741,15 +743,18 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /** {@inheritDoc} */
-    @Override protected void checkPolicyEvictionProperties(MemoryPolicyConfiguration plcCfg, MemoryConfiguration dbCfg)
+    @Override protected void checkRegionEvictionProperties(DataRegionConfiguration regCfg, DataStorageConfiguration dbCfg)
         throws IgniteCheckedException {
-        if (plcCfg.getPageEvictionMode() != DataPageEvictionMode.DISABLED)
-            U.warn(log, "Page eviction mode for [" + plcCfg.getName() + "] memory region is ignored " +
+        if (!regCfg.isPersistenceEnabled())
+            super.checkRegionEvictionProperties(regCfg, dbCfg);
+
+        if (regCfg.getPageEvictionMode() != DataPageEvictionMode.DISABLED)
+            U.warn(log, "Page eviction mode for [" + regCfg.getName() + "] memory region is ignored " +
                 "because Ignite Native Persistence is enabled");
     }
 
     /** {@inheritDoc} */
-    @Override protected void checkPageSize(MemoryConfiguration memCfg) {
+    @Override protected void checkPageSize(DataStorageConfiguration memCfg) {
         if (memCfg.getPageSize() == 0) {
             try {
                 assert cctx.pageStore() instanceof FilePageStoreManager :
@@ -767,10 +772,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             catch (IgniteCheckedException | IOException | IllegalArgumentException e) {
                 U.quietAndWarn(log, "Attempt to resolve pageSize from store files failed: " + e.getMessage());
 
-                U.quietAndWarn(log, "Default page size will be used: " + MemoryConfiguration.DFLT_PAGE_SIZE + " bytes");
+                U.quietAndWarn(log, "Default page size will be used: " + DataStorageConfiguration.DFLT_PAGE_SIZE + " bytes");
             }
 
-            memCfg.setPageSize(MemoryConfiguration.DFLT_PAGE_SIZE);
+            memCfg.setPageSize(DataStorageConfiguration.DFLT_PAGE_SIZE);
         }
     }
 
@@ -800,7 +805,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             int pageSize = hdr.getInt();
 
             if (pageSize == 2048) {
-                U.quietAndWarn(log, "You are currently using persistent store with 2K pages (MemoryConfiguration#" +
+                U.quietAndWarn(log, "You are currently using persistent store with 2K pages (DataStorageConfiguration#" +
                     "pageSize). If you use SSD disk, consider migrating to 4K pages for better IO performance.");
             }
 
@@ -923,9 +928,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         for (IgniteBiTuple<CacheGroupContext, Boolean> tup : stoppedGrps) {
             CacheGroupContext gctx = tup.get1();
 
+            if (!gctx.persistenceEnabled())
+                continue;
+
             snapshotMgr.onCacheGroupStop(gctx);
 
-            PageMemoryEx pageMem = (PageMemoryEx)gctx.memoryPolicy().pageMemory();
+            PageMemoryEx pageMem = (PageMemoryEx)gctx.dataRegion().pageMemory();
 
             Collection<Integer> grpIds = destroyed.get(pageMem);
 
@@ -1024,12 +1032,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @return {@code true} if all PageMemory instances are safe to update.
      */
     private boolean safeToUpdatePageMemories() {
-        Collection<MemoryPolicy> memPlcs = context().database().memoryPolicies();
+        Collection<DataRegion> memPlcs = context().database().dataRegions();
 
         if (memPlcs == null)
             return true;
 
-        for (MemoryPolicy memPlc : memPlcs) {
+        for (DataRegion memPlc : memPlcs) {
+            if (!memPlc.config().isPersistenceEnabled())
+                continue;
+
             PageMemoryEx pageMemEx = (PageMemoryEx)memPlc.pageMemory();
 
             if (!pageMemEx.safeToUpdate())
@@ -1049,11 +1060,14 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         checkpointLock.readLock().unlock();
 
         if (checkpointer != null) {
-            Collection<MemoryPolicy> memPlcs = context().database().memoryPolicies();
+            Collection<DataRegion> dataRegs = context().database().dataRegions();
+
+            if (dataRegs != null) {
+                for (DataRegion dataReg : dataRegs) {
+                    if (!dataReg.config().isPersistenceEnabled())
+                        continue;
 
-            if (memPlcs != null) {
-                for (MemoryPolicy memPlc : memPlcs) {
-                    PageMemoryEx mem = (PageMemoryEx)memPlc.pageMemory();
+                    PageMemoryEx mem = (PageMemoryEx)dataReg.pageMemory();
 
                     if (mem != null && !mem.safeToUpdate()) {
                         checkpointer.wakeupForCheckpoint(0, "too many dirty pages");
@@ -1595,7 +1609,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      *
      * @param grpId Cache group id.
      * @return PageMemoryEx instance.
-     * @throws IgniteCheckedException if no MemoryPolicy is configured for a name obtained from cache descriptor.
+     * @throws IgniteCheckedException if no DataRegion is configured for a name obtained from cache descriptor.
      */
     private PageMemoryEx getPageMemoryForCacheGroup(int grpId) throws IgniteCheckedException {
         // TODO IGNITE-5075: cache descriptor can be removed.
@@ -1606,9 +1620,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         if (desc == null)
             throw new IgniteCheckedException("Failed to find cache group descriptor [grpId=" + grpId + ']');
 
-        String memPlcName = desc.config().getMemoryPolicyName();
+        String memPlcName = desc.config().getDataRegionName();
 
-        return (PageMemoryEx)sharedCtx.database().memoryPolicy(memPlcName).pageMemory();
+        return (PageMemoryEx)sharedCtx.database().dataRegion(memPlcName).pageMemory();
     }
 
     /**
@@ -1687,9 +1701,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 continue;
             }
 
+            if (!grp.dataRegion().config().isPersistenceEnabled())
+                continue;
+
             int grpId = grp.groupId();
 
-            PageMemoryEx pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+            PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
 
             for (int i = 0; i < grp.affinity().partitions(); i++) {
                 if (storeMgr.exists(grpId, i)) {
@@ -1822,14 +1839,17 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         long start = System.currentTimeMillis();
 
-        Collection<MemoryPolicy> memPolicies = context().database().memoryPolicies();
+        Collection<DataRegion> memPolicies = context().database().dataRegions();
 
         List<IgniteBiTuple<PageMemory, Collection<FullPageId>>> cpEntities = new ArrayList<>(memPolicies.size());
 
-        for (MemoryPolicy memPlc : memPolicies) {
-            PageMemoryEx pageMem = (PageMemoryEx)memPlc.pageMemory();
-            cpEntities.add(new IgniteBiTuple<PageMemory, Collection<FullPageId>>(pageMem,
-                (pageMem).beginCheckpoint()));
+        for (DataRegion memPlc : memPolicies) {
+            if (memPlc.config().isPersistenceEnabled()) {
+                PageMemoryEx pageMem = (PageMemoryEx)memPlc.pageMemory();
+
+                cpEntities.add(new IgniteBiTuple<PageMemory, Collection<FullPageId>>(
+                    pageMem, (pageMem).beginCheckpoint()));
+            }
         }
 
         tmpWriteBuf.order(ByteOrder.nativeOrder());
@@ -2426,11 +2446,14 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
          * pages.
          */
         private IgniteBiTuple<Collection<GridMultiCollectionWrapper<FullPageId>>, Integer> beginAllCheckpoints() {
-            Collection<GridMultiCollectionWrapper<FullPageId>> res = new ArrayList(memoryPolicies().size());
+            Collection<GridMultiCollectionWrapper<FullPageId>> res = new ArrayList(dataRegions().size());
 
             int pagesNum = 0;
 
-            for (MemoryPolicy memPlc : memoryPolicies()) {
+            for (DataRegion memPlc : dataRegions()) {
+                if (!memPlc.config().isPersistenceEnabled())
+                    continue;
+
                 GridMultiCollectionWrapper<FullPageId> nextCpPagesCol = ((PageMemoryEx)memPlc.pageMemory()).beginCheckpoint();
 
                 pagesNum += nextCpPagesCol.size();
@@ -2446,8 +2469,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
          */
         private void markCheckpointEnd(Checkpoint chp) throws IgniteCheckedException {
             synchronized (this) {
-                for (MemoryPolicy memPlc : memoryPolicies())
+                for (DataRegion memPlc : dataRegions()) {
+                    if (!memPlc.config().isPersistenceEnabled())
+                        continue;
+
                     ((PageMemoryEx)memPlc.pageMemory()).finishCheckpoint();
+                }
 
                 if (chp.hasDelta())
                     writeCheckpointEntry(
@@ -2495,8 +2522,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /**
      * Reorders list of checkpoint pages and splits them into needed number of sublists according to
-     * {@link PersistentStoreConfiguration#getCheckpointingThreads()} and
-     * {@link PersistentStoreConfiguration#getCheckpointWriteOrder()}.
+     * {@link DataStorageConfiguration#getCheckpointThreads()} and
+     * {@link DataStorageConfiguration#getCheckpointWriteOrder()}.
      *
      * @param cpPagesTuple Checkpoint pages tuple.
      */
@@ -2523,7 +2550,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             });
         }
 
-        int cpThreads = persistenceCfg.getCheckpointingThreads();
+        int cpThreads = persistenceCfg.getCheckpointThreads();
 
         int pagesSubLists = cpThreads == 1 ? 1 : cpThreads * 4;
         // Splitting pages to (threads * 4) subtasks. If any thread will be faster, it will help slower threads.
@@ -2606,7 +2633,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     if (grp == null)
                         continue;
 
-                    PageMemoryEx pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+                    if (!grp.dataRegion().config().isPersistenceEnabled())
+                        continue;
+
+                    PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
 
                     Integer tag = pageMem.getForCheckpoint(
                         fullId, tmpWriteBuf, persStoreMetrics.metricsEnabled() ? tracker : null);
@@ -3313,14 +3343,14 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /** {@inheritDoc} */
-    @Override public PersistenceMetrics persistentStoreMetrics() {
-        return new PersistenceMetricsSnapshot(persStoreMetrics);
+    @Override public DataStorageMetrics persistentStoreMetrics() {
+        return new DataStorageMetricsSnapshot(persStoreMetrics);
     }
 
     /**
      *
      */
-    public PersistenceMetricsImpl persistentStoreMetricsImpl() {
+    public DataStorageMetricsImpl persistentStoreMetricsImpl() {
         return persStoreMetrics;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index 5c91a4f..6ed62f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -93,14 +93,14 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
         reuseList = new ReuseListImpl(grp.groupId(),
             grp.cacheOrGroupName(),
-            grp.memoryPolicy().pageMemory(),
+            grp.dataRegion().pageMemory(),
             ctx.wal(),
             reuseListRoot.pageId().pageId(),
             reuseListRoot.isAllocated());
 
         RootPage metastoreRoot = metas.treeRoot;
 
-        metaStore = new MetadataStorage(grp.memoryPolicy().pageMemory(),
+        metaStore = new MetadataStorage(grp.dataRegion().pageMemory(),
             ctx.wal(),
             globalRemoveId(),
             grp.groupId(),
@@ -126,7 +126,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                 pendingEntries = new PendingEntriesTree(
                     grp,
                     name,
-                    grp.memoryPolicy().pageMemory(),
+                    grp.dataRegion().pageMemory(),
                     pendingRootPage.pageId().pageId(),
                     reuseList,
                     pendingRootPage.isAllocated()
@@ -148,7 +148,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
     /** {@inheritDoc} */
     @Override public void onCheckpointBegin(Context ctx) throws IgniteCheckedException {
-        assert grp.memoryPolicy().pageMemory() instanceof PageMemoryEx;
+        assert grp.dataRegion().pageMemory() instanceof PageMemoryEx;
 
         reuseList.saveMetadata();
 
@@ -185,7 +185,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
             int size = store.fullSize();
             long rmvId = globalRemoveId().get();
 
-            PageMemoryEx pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+            PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
             IgniteWriteAheadLogManager wal = this.ctx.wal();
 
             if (size > 0 || updCntr > 0) {
@@ -437,7 +437,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
             saveStoreMetadata(store, null, false, true);
 
-            PageMemoryEx pageMemory = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+            PageMemoryEx pageMemory = (PageMemoryEx)grp.dataRegion().pageMemory();
 
             int tag = pageMemory.invalidate(grp.groupId(), p);
 
@@ -511,7 +511,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
      * @throws IgniteCheckedException If failed.
      */
     private Metas getOrAllocateCacheMetas() throws IgniteCheckedException {
-        PageMemoryEx pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+        PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
         IgniteWriteAheadLogManager wal = ctx.wal();
 
         int grpId = grp.groupId();
@@ -915,8 +915,8 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                     freeList = new FreeListImpl(
                         grp.groupId(),
                         grp.cacheOrGroupName() + "-" + partId,
-                        grp.memoryPolicy().memoryMetrics(),
-                        grp.memoryPolicy(),
+                        grp.dataRegion().memoryMetrics(),
+                        grp.dataRegion(),
                         null,
                         ctx.wal(),
                         reuseRoot.pageId().pageId(),
@@ -942,7 +942,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                         }
                     };
 
-                    PageMemoryEx pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+                    PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
 
                     delegate0 = new CacheDataStoreImpl(partId, name, rowStore, dataTree);
 
@@ -1036,7 +1036,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
          * @return Partition metas.
          */
         private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException {
-            PageMemoryEx pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+            PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
             IgniteWriteAheadLogManager wal = ctx.wal();
 
             int grpId = grp.groupId();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index d7682f0..da598d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -25,14 +25,14 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.ignite.DataRegionMetrics;
+import org.apache.ignite.DataStorageMetrics;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.MemoryMetrics;
-import org.apache.ignite.PersistenceMetrics;
 import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
@@ -57,24 +57,24 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseL
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
+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.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteOutClosure;
-import org.apache.ignite.mxbean.MemoryMetricsMXBean;
+import org.apache.ignite.mxbean.DataRegionMetricsMXBean;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_MEMORY_POLICY_INITIAL_SIZE;
-import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_MEM_PLC_DEFAULT_NAME;
-import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_PAGE_SIZE;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE;
 
 /**
  *
  */
 public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdapter
     implements IgniteChangeGlobalStateSupport, CheckpointLockStateChecker {
-    /** MemoryPolicyConfiguration name reserved for internal caches. */
-    static final String SYSTEM_MEMORY_POLICY_NAME = "sysMemPlc";
+    /** DataRegionConfiguration name reserved for internal caches. */
+    static final String SYSTEM_DATA_REGION_NAME = "sysMemPlc";
 
     /** Minimum size of memory chunk */
     private static final long MIN_PAGE_MEMORY_SIZE = 10 * 1024 * 1024;
@@ -83,16 +83,16 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     private static final long MAX_PAGE_MEMORY_INIT_SIZE_32_BIT = 2L * 1024 * 1024 * 1024;
 
     /** */
-    protected Map<String, MemoryPolicy> memPlcMap;
+    protected Map<String, DataRegion> dataRegionMap;
 
     /** */
-    protected Map<String, MemoryMetrics> memMetricsMap;
+    protected Map<String, DataRegionMetrics> memMetricsMap;
 
     /** */
-    protected MemoryPolicy dfltMemPlc;
+    protected DataRegion dfltDataRegion;
 
     /** */
-    private Map<String, FreeListImpl> freeListMap;
+    protected Map<String, FreeListImpl> freeListMap;
 
     /** */
     private FreeListImpl dfltFreeList;
@@ -102,10 +102,10 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
-        if (cctx.kernalContext().clientNode() && cctx.kernalContext().config().getMemoryConfiguration() == null)
+        if (cctx.kernalContext().clientNode() && cctx.kernalContext().config().getDataStorageConfiguration() == null)
             return;
 
-        MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();
+        DataStorageConfiguration memCfg = cctx.kernalContext().config().getDataStorageConfiguration();
 
         assert memCfg != null;
 
@@ -115,7 +115,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * Registers MBeans for all MemoryMetrics configured in this instance.
+     * Registers MBeans for all DataRegionMetrics configured in this instance.
      */
     private void registerMetricsMBeans() {
         if(U.IGNITE_MBEANS_DISABLED)
@@ -123,21 +123,21 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
         IgniteConfiguration cfg = cctx.gridConfig();
 
-        for (MemoryMetrics memMetrics : memMetricsMap.values()) {
-            MemoryPolicyConfiguration memPlcCfg = memPlcMap.get(memMetrics.getName()).config();
+        for (DataRegionMetrics memMetrics : memMetricsMap.values()) {
+            DataRegionConfiguration memPlcCfg = dataRegionMap.get(memMetrics.getName()).config();
 
-            registerMetricsMBean((MemoryMetricsImpl)memMetrics, memPlcCfg, cfg);
+            registerMetricsMBean((DataRegionMetricsImpl)memMetrics, memPlcCfg, cfg);
         }
     }
 
     /**
      * @param memMetrics Memory metrics.
-     * @param memPlcCfg Memory policy configuration.
+     * @param dataRegionCfg Data region configuration.
      * @param cfg Ignite configuration.
      */
     private void registerMetricsMBean(
-        MemoryMetricsImpl memMetrics,
-        MemoryPolicyConfiguration memPlcCfg,
+        DataRegionMetricsImpl memMetrics,
+        DataRegionConfiguration dataRegionCfg,
         IgniteConfiguration cfg
     ) {
         assert !U.IGNITE_MBEANS_DISABLED;
@@ -146,13 +146,13 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
             U.registerMBean(
                 cfg.getMBeanServer(),
                 cfg.getIgniteInstanceName(),
-                "MemoryMetrics",
-                memPlcCfg.getName(),
-                new MemoryMetricsMXBeanImpl(memMetrics, memPlcCfg),
-                MemoryMetricsMXBean.class);
+                "DataRegionMetrics",
+                dataRegionCfg.getName(),
+                new DataRegionMetricsMXBeanImpl(memMetrics, dataRegionCfg),
+                DataRegionMetricsMXBean.class);
         }
         catch (Throwable e) {
-            U.error(log, "Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'", e);
+            U.error(log, "Failed to register MBean for DataRegionMetrics with name: '" + memMetrics.getName() + "'", e);
         }
     }
 
@@ -160,22 +160,24 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param dbCfg Database config.
      * @throws IgniteCheckedException If failed.
      */
-    protected void initPageMemoryDataStructures(MemoryConfiguration dbCfg) throws IgniteCheckedException {
-        freeListMap = U.newHashMap(memPlcMap.size());
+    protected void initPageMemoryDataStructures(DataStorageConfiguration dbCfg) throws IgniteCheckedException {
+        freeListMap = U.newHashMap(dataRegionMap.size());
 
-        String dfltMemPlcName = dbCfg.getDefaultMemoryPolicyName();
+        String dfltMemPlcName = dbCfg.getDefaultDataRegionConfiguration().getName();
 
-        for (MemoryPolicy memPlc : memPlcMap.values()) {
-            MemoryPolicyConfiguration memPlcCfg = memPlc.config();
+        for (DataRegion memPlc : dataRegionMap.values()) {
+            DataRegionConfiguration memPlcCfg = memPlc.config();
 
-            MemoryMetricsImpl memMetrics = (MemoryMetricsImpl) memMetricsMap.get(memPlcCfg.getName());
+            DataRegionMetricsImpl memMetrics = (DataRegionMetricsImpl) memMetricsMap.get(memPlcCfg.getName());
+
+            boolean persistenceEnabled = memPlcCfg.isPersistenceEnabled();
 
             FreeListImpl freeList = new FreeListImpl(0,
                     cctx.igniteInstanceName(),
                     memMetrics,
                     memPlc,
                     null,
-                    cctx.wal(),
+                    persistenceEnabled ? cctx.wal() : null,
                     0L,
                     true);
 
@@ -196,7 +198,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      *
      */
     private void startMemoryPolicies() {
-        for (MemoryPolicy memPlc : memPlcMap.values()) {
+        for (DataRegion memPlc : dataRegionMap.values()) {
             memPlc.pageMemory().start();
 
             memPlc.evictionTracker().start();
@@ -207,102 +209,81 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param memCfg Database config.
      * @throws IgniteCheckedException If failed to initialize swap path.
      */
-    protected void initPageMemoryPolicies(MemoryConfiguration memCfg) throws IgniteCheckedException {
-        MemoryPolicyConfiguration[] memPlcsCfgs = memCfg.getMemoryPolicies();
-
-        if (memPlcsCfgs == null) {
-            //reserve place for default and system memory policies
-            memPlcMap = U.newHashMap(2);
-            memMetricsMap = U.newHashMap(2);
-
-            addMemoryPolicy(
-                memCfg,
-                memCfg.createDefaultPolicyConfig(),
-                DFLT_MEM_PLC_DEFAULT_NAME
-            );
-
-            U.warn(log, "No user-defined default MemoryPolicy found; system default of 1GB size will be used.");
-        }
-        else {
-            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);
+    protected void initDataRegions(DataStorageConfiguration memCfg) throws IgniteCheckedException {
+        DataRegionConfiguration[] dataRegionCfgs = memCfg.getDataRegionConfigurations();
 
-                addMemoryPolicy(
-                    memCfg,
-                    memCfg.createDefaultPolicyConfig(),
-                    DFLT_MEM_PLC_DEFAULT_NAME
-                );
+        int dataRegions = dataRegionCfgs == null ? 0 : dataRegionCfgs.length;
 
-                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
-                memPlcMap = U.newHashMap(memPlcsCfgs.length + 1);
-                memMetricsMap = U.newHashMap(memPlcsCfgs.length + 1);
-            }
+        dataRegionMap = U.newHashMap(2 + dataRegions);
+        memMetricsMap = U.newHashMap(2 + dataRegions);
 
-            for (MemoryPolicyConfiguration memPlcCfg : memPlcsCfgs)
-                addMemoryPolicy(memCfg, memPlcCfg, memPlcCfg.getName());
+        if (dataRegionCfgs != null) {
+            for (DataRegionConfiguration dataRegionCfg : dataRegionCfgs)
+                addDataRegion(memCfg, dataRegionCfg, dataRegionCfg.getName());
         }
 
-        addMemoryPolicy(
+        addDataRegion(
+            memCfg,
+            memCfg.getDefaultDataRegionConfiguration(),
+            memCfg.getDefaultDataRegionConfiguration().getName()
+        );
+
+        addDataRegion(
             memCfg,
-            createSystemMemoryPolicy(
-                memCfg.getSystemCacheInitialSize(),
-                memCfg.getSystemCacheMaxSize()
+            createSystemDataRegion(
+                memCfg.getSystemRegionInitialSize(),
+                memCfg.getSystemRegionMaxSize()
             ),
-            SYSTEM_MEMORY_POLICY_NAME
+            SYSTEM_DATA_REGION_NAME
         );
     }
 
     /**
-     * @param memCfg Database config.
-     * @param memPlcCfg Memory policy config.
-     * @param memPlcName Memory policy name.
+     * @param dataStorageCfg Database config.
+     * @param dataRegionCfg Data region config.
+     * @param dataRegionName Data region name.
      * @throws IgniteCheckedException If failed to initialize swap path.
      */
-    private void addMemoryPolicy(
-        MemoryConfiguration memCfg,
-        MemoryPolicyConfiguration memPlcCfg,
-        String memPlcName
+    private void addDataRegion(
+        DataStorageConfiguration dataStorageCfg,
+        DataRegionConfiguration dataRegionCfg,
+        String dataRegionName
     ) throws IgniteCheckedException {
-        String dfltMemPlcName = memCfg.getDefaultMemoryPolicyName();
+        String dfltMemPlcName = dataStorageCfg.getDefaultDataRegionConfiguration().getName();
 
         if (dfltMemPlcName == null)
-            dfltMemPlcName = DFLT_MEM_PLC_DEFAULT_NAME;
+            dfltMemPlcName = DFLT_DATA_REG_DEFAULT_NAME;
 
-        MemoryMetricsImpl memMetrics = new MemoryMetricsImpl(memPlcCfg, fillFactorProvider(memPlcName));
+        DataRegionMetricsImpl memMetrics = new DataRegionMetricsImpl(dataRegionCfg, fillFactorProvider(dataRegionCfg));
 
-        MemoryPolicy memPlc = initMemory(memCfg, memPlcCfg, memMetrics);
+        DataRegion memPlc = initMemory(dataStorageCfg, dataRegionCfg, memMetrics);
 
-        memPlcMap.put(memPlcName, memPlc);
+        dataRegionMap.put(dataRegionName, memPlc);
 
-        memMetricsMap.put(memPlcName, memMetrics);
+        memMetricsMap.put(dataRegionName, memMetrics);
 
-        if (memPlcName.equals(dfltMemPlcName))
-            dfltMemPlc = memPlc;
-        else if (memPlcName.equals(DFLT_MEM_PLC_DEFAULT_NAME))
-            U.warn(log, "Memory Policy with name 'default' isn't used as a default. " +
+        if (dataRegionName.equals(dfltMemPlcName))
+            dfltDataRegion = memPlc;
+        else if (dataRegionName.equals(DFLT_DATA_REG_DEFAULT_NAME))
+            U.warn(log, "Data Region with name 'default' isn't used as a default. " +
                     "Please check Memory Policies configuration.");
     }
 
     /**
-     * Closure that can be used to compute fill factor for provided memory policy.
+     * Closure that can be used to compute fill factor for provided data region.
      *
-     * @param memPlcName Memory policy name.
+     * @param dataRegCfg Data region configuration.
      * @return Closure.
      */
-    protected IgniteOutClosure<Float> fillFactorProvider(final String memPlcName) {
+    protected IgniteOutClosure<Float> fillFactorProvider(final DataRegionConfiguration dataRegCfg) {
+        final String dataRegName = dataRegCfg.getName();
+
         return new IgniteOutClosure<Float>() {
             private FreeListImpl freeList;
 
             @Override public Float apply() {
                 if (freeList == null) {
-                    FreeListImpl freeList0 = freeListMap.get(memPlcName);
+                    FreeListImpl freeList0 = freeListMap.get(dataRegName);
 
                     if (freeList0 == null)
                         return (float) 0;
@@ -321,11 +302,11 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param memPlcsCfgs User-defined memory policy configurations.
+     * @param memPlcsCfgs User-defined data region configurations.
      */
-    private boolean hasCustomDefaultMemoryPolicy(MemoryPolicyConfiguration[] memPlcsCfgs) {
-        for (MemoryPolicyConfiguration memPlcsCfg : memPlcsCfgs) {
-            if (DFLT_MEM_PLC_DEFAULT_NAME.equals(memPlcsCfg.getName()))
+    private boolean hasCustomDefaultDataRegion(DataRegionConfiguration[] memPlcsCfgs) {
+        for (DataRegionConfiguration memPlcsCfg : memPlcsCfgs) {
+            if (DFLT_DATA_REG_DEFAULT_NAME.equals(memPlcsCfg.getName()))
                 return true;
         }
 
@@ -336,12 +317,12 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @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.
+     * @return {@link DataRegionConfiguration configuration} of DataRegion for system cache.
      */
-    private MemoryPolicyConfiguration createSystemMemoryPolicy(long sysCacheInitSize, long sysCacheMaxSize) {
-        MemoryPolicyConfiguration res = new MemoryPolicyConfiguration();
+    private DataRegionConfiguration createSystemDataRegion(long sysCacheInitSize, long sysCacheMaxSize) {
+        DataRegionConfiguration res = new DataRegionConfiguration();
 
-        res.setName(SYSTEM_MEMORY_POLICY_NAME);
+        res.setName(SYSTEM_DATA_REGION_NAME);
         res.setInitialSize(sysCacheInitSize);
         res.setMaxSize(sysCacheMaxSize);
 
@@ -351,71 +332,76 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     /**
      * @param memCfg configuration to validate.
      */
-    private void validateConfiguration(MemoryConfiguration memCfg) throws IgniteCheckedException {
+    private void validateConfiguration(DataStorageConfiguration memCfg) throws IgniteCheckedException {
         checkPageSize(memCfg);
 
-        MemoryPolicyConfiguration[] plcCfgs = memCfg.getMemoryPolicies();
+        DataRegionConfiguration[] regCfgs = memCfg.getDataRegionConfigurations();
 
-        Set<String> plcNames = (plcCfgs != null) ? U.<String>newHashSet(plcCfgs.length) : new HashSet<String>(0);
+        Set<String> regNames = (regCfgs != null) ? U.<String>newHashSet(regCfgs.length) : new HashSet<String>(0);
 
-        checkSystemMemoryPolicySizeConfiguration(
-            memCfg.getSystemCacheInitialSize(),
-            memCfg.getSystemCacheMaxSize()
+        checkSystemDataRegionSizeConfiguration(
+            memCfg.getSystemRegionInitialSize(),
+            memCfg.getSystemRegionMaxSize()
         );
 
-        if (plcCfgs != null) {
-            for (MemoryPolicyConfiguration plcCfg : plcCfgs) {
-                assert plcCfg != null;
+        if (regCfgs != null) {
+            for (DataRegionConfiguration regCfg : regCfgs)
+                checkDataRegionConfiguration(memCfg, regNames, regCfg);
+        }
+
+        checkDataRegionConfiguration(memCfg, regNames, memCfg.getDefaultDataRegionConfiguration());
+    }
 
-                checkPolicyName(plcCfg.getName(), plcNames);
+    /**
+     * @param memCfg Mem config.
+     * @param regNames Region names.
+     * @param regCfg Reg config.
+     */
+    private void checkDataRegionConfiguration(DataStorageConfiguration memCfg, Set<String> regNames,
+        DataRegionConfiguration regCfg) throws IgniteCheckedException {
+        assert regCfg != null;
 
-                checkPolicySize(plcCfg);
+        checkDataRegionName(regCfg.getName(), regNames);
 
-                checkMetricsProperties(plcCfg);
+        checkDataRegionSize(regCfg);
 
-                checkPolicyEvictionProperties(plcCfg, memCfg);
-            }
-        }
+        checkMetricsProperties(regCfg);
 
-        checkDefaultPolicyConfiguration(
-            memCfg.getDefaultMemoryPolicyName(),
-            memCfg.getDefaultMemoryPolicySize(),
-            plcNames
-        );
+        checkRegionEvictionProperties(regCfg, memCfg);
     }
 
     /**
      * @param memCfg Memory config.
      */
-    protected void checkPageSize(MemoryConfiguration memCfg) {
+    protected void checkPageSize(DataStorageConfiguration memCfg) {
         if (memCfg.getPageSize() == 0)
             memCfg.setPageSize(DFLT_PAGE_SIZE);
     }
 
     /**
-     * @param plcCfg Memory policy config.
+     * @param regCfg data region config.
      *
      * @throws IgniteCheckedException if validation of memory metrics properties fails.
      */
-    private static void checkMetricsProperties(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
-        if (plcCfg.getRateTimeInterval() <= 0)
+    private static void checkMetricsProperties(DataRegionConfiguration regCfg) throws IgniteCheckedException {
+        if (regCfg.getMetricsRateTimeInterval() <= 0)
             throw new IgniteCheckedException("Rate time interval must be greater than zero " +
-                "(use MemoryPolicyConfiguration.rateTimeInterval property to adjust the interval) " +
-                "[name=" + plcCfg.getName() +
-                ", rateTimeInterval=" + plcCfg.getRateTimeInterval() + "]"
+                "(use DataRegionConfiguration.rateTimeInterval property to adjust the interval) " +
+                "[name=" + regCfg.getName() +
+                ", rateTimeInterval=" + regCfg.getMetricsRateTimeInterval() + "]"
             );
-        if (plcCfg.getSubIntervals() <= 0)
+        if (regCfg.getMetricsSubIntervalCount() <= 0)
             throw new IgniteCheckedException("Sub intervals must be greater than zero " +
-                "(use MemoryPolicyConfiguration.subIntervals property to adjust the sub intervals) " +
-                "[name=" + plcCfg.getName() +
-                ", subIntervals=" + plcCfg.getSubIntervals() + "]"
+                "(use DataRegionConfiguration.subIntervals property to adjust the sub intervals) " +
+                "[name=" + regCfg.getName() +
+                ", subIntervals=" + regCfg.getMetricsSubIntervalCount() + "]"
             );
 
-        if (plcCfg.getRateTimeInterval() < 1_000)
+        if (regCfg.getMetricsRateTimeInterval() < 1_000)
             throw new IgniteCheckedException("Rate time interval must be longer that 1 second (1_000 milliseconds) " +
-                "(use MemoryPolicyConfiguration.rateTimeInterval property to adjust the interval) " +
-                "[name=" + plcCfg.getName() +
-                ", rateTimeInterval=" + plcCfg.getRateTimeInterval() + "]");
+                "(use DataRegionConfiguration.rateTimeInterval property to adjust the interval) " +
+                "[name=" + regCfg.getName() +
+                ", rateTimeInterval=" + regCfg.getMetricsRateTimeInterval() + "]");
     }
 
     /**
@@ -424,19 +410,19 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      *
      * @throws IgniteCheckedException In case of validation violation.
      */
-    private static void checkSystemMemoryPolicySizeConfiguration(
+    private static void checkSystemDataRegionSizeConfiguration(
         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) " +
+                "DataStorageConfiguration.systemCacheInitialSize property to set correct size in bytes) " +
                 "[size=" + U.readableSize(sysCacheInitSize, true) + ']'
             );
 
         if (U.jvm32Bit() && sysCacheInitSize > MAX_PAGE_MEMORY_INIT_SIZE_32_BIT)
             throw new IgniteCheckedException("Initial size for system cache exceeds 2GB on 32-bit JVM (use " +
-                "MemoryPolicyConfiguration.systemCacheInitialSize property to set correct size in bytes " +
+                "DataRegionConfiguration.systemCacheInitialSize property to set correct size in bytes " +
                 "or use 64-bit JVM) [size=" + U.readableSize(sysCacheInitSize, true) + ']'
             );
 
@@ -444,138 +430,90 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
             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 " +
+                "Use DataStorageConfiguration.systemCacheInitialSize/DataStorageConfiguration.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,
-        long dfltPlcSize,
-        Collection<String> plcNames
-    ) throws IgniteCheckedException {
-        if (dfltPlcSize != MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE) {
-            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 (U.jvm32Bit() && dfltPlcSize > MAX_PAGE_MEMORY_INIT_SIZE_32_BIT)
-                throw new IgniteCheckedException("User-defined default MemoryPolicy size exceeds 2GB on 32-bit JVM " +
-                    "(use MemoryConfiguration.defaultMemoryPolicySize property to set correct size in bytes " +
-                    "or use 64-bit JVM) [size=" + U.readableSize(dfltPlcSize, true) + ']'
-                );
-        }
-
-        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);
-        }
-    }
-
-    /**
-     * @param plcCfg MemoryPolicyConfiguration to validate.
+     * @param regCfg DataRegionConfiguration to validate.
      * @throws IgniteCheckedException If config is invalid.
      */
-    private void checkPolicySize(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
-        boolean dfltInitSize = false;
-
-        if (plcCfg.getInitialSize() == 0) {
-            plcCfg.setInitialSize(DFLT_MEMORY_POLICY_INITIAL_SIZE);
-
-            dfltInitSize = true;
-        }
-
-        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) + "]"
+    private void checkDataRegionSize(DataRegionConfiguration regCfg) throws IgniteCheckedException {
+        if (regCfg.getInitialSize() < MIN_PAGE_MEMORY_SIZE || regCfg.getMaxSize() < MIN_PAGE_MEMORY_SIZE)
+            throw new IgniteCheckedException("DataRegion must have size more than 10MB (use " +
+                "DataRegionConfiguration.initialSize and .maxSize properties to set correct size in bytes) " +
+                "[name=" + regCfg.getName() + ", initialSize=" + U.readableSize(regCfg.getInitialSize(), true) +
+                ", maxSize=" + U.readableSize(regCfg.getMaxSize(), true) + "]"
             );
 
-        if (plcCfg.getMaxSize() < plcCfg.getInitialSize()) {
-            // If initial size was not set, use the max size.
-            if (dfltInitSize) {
-                plcCfg.setInitialSize(plcCfg.getMaxSize());
-
-                LT.warn(log, "MemoryPolicy maxSize=" + U.readableSize(plcCfg.getMaxSize(), true) +
-                    " is smaller than defaultInitialSize=" +
-                    U.readableSize(MemoryConfiguration.DFLT_MEMORY_POLICY_INITIAL_SIZE, true) +
-                    ", setting initialSize to " + U.readableSize(plcCfg.getMaxSize(), true));
-            }
-            else {
-                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) + ']');
+        if (regCfg.getMaxSize() < regCfg.getInitialSize()) {
+            if (regCfg.getInitialSize() != Math.min(DataStorageConfiguration.DFLT_DATA_REGION_MAX_SIZE,
+                DataStorageConfiguration.DFLT_DATA_REGION_INITIAL_SIZE)) {
+                throw new IgniteCheckedException("DataRegion maxSize must not be smaller than initialSize" +
+                    "[name=" + regCfg.getName() + ", initialSize=" + U.readableSize(regCfg.getInitialSize(), true) +
+                    ", maxSize=" + U.readableSize(regCfg.getMaxSize(), true) + "]");
             }
+
+            regCfg.setInitialSize(regCfg.getMaxSize());
+
+            LT.warn(log, "DataRegion maxSize=" + U.readableSize(regCfg.getMaxSize(), true) +
+                " is smaller than defaultInitialSize=" +
+                U.readableSize(DataStorageConfiguration.DFLT_DATA_REGION_INITIAL_SIZE, true) +
+                ", setting initialSize to " + U.readableSize(regCfg.getMaxSize(), true));
         }
 
-        if (U.jvm32Bit() && plcCfg.getInitialSize() > MAX_PAGE_MEMORY_INIT_SIZE_32_BIT)
-            throw new IgniteCheckedException("MemoryPolicy initialSize exceeds 2GB on 32-bit JVM (use " +
-                "MemoryPolicyConfiguration.initialSize property to set correct size in bytes or use 64-bit JVM) " +
-                "[name=" + plcCfg.getName() +
-                ", size=" + U.readableSize(plcCfg.getInitialSize(), true) + "]");
+        if (U.jvm32Bit() && regCfg.getInitialSize() > MAX_PAGE_MEMORY_INIT_SIZE_32_BIT)
+            throw new IgniteCheckedException("DataRegion initialSize exceeds 2GB on 32-bit JVM (use " +
+                "DataRegionConfiguration.initialSize property to set correct size in bytes or use 64-bit JVM) " +
+                "[name=" + regCfg.getName() +
+                ", size=" + U.readableSize(regCfg.getInitialSize(), true) + "]");
     }
 
     /**
-     * @param plcCfg MemoryPolicyConfiguration to validate.
+     * @param regCfg DataRegionConfiguration to validate.
      * @param dbCfg Memory configuration.
      * @throws IgniteCheckedException If config is invalid.
      */
-    protected void checkPolicyEvictionProperties(MemoryPolicyConfiguration plcCfg, MemoryConfiguration dbCfg)
+    protected void checkRegionEvictionProperties(DataRegionConfiguration regCfg, DataStorageConfiguration dbCfg)
         throws IgniteCheckedException {
-        if (plcCfg.getPageEvictionMode() == DataPageEvictionMode.DISABLED)
+        if (regCfg.getPageEvictionMode() == DataPageEvictionMode.DISABLED)
             return;
 
-        if (plcCfg.getEvictionThreshold() < 0.5 || plcCfg.getEvictionThreshold() > 0.999) {
+        if (regCfg.getEvictionThreshold() < 0.5 || regCfg.getEvictionThreshold() > 0.999) {
             throw new IgniteCheckedException("Page eviction threshold must be between 0.5 and 0.999: " +
-                plcCfg.getName());
+                regCfg.getName());
         }
 
-        if (plcCfg.getEmptyPagesPoolSize() <= 10)
-            throw new IgniteCheckedException("Evicted pages pool size should be greater than 10: " + plcCfg.getName());
+        if (regCfg.getEmptyPagesPoolSize() <= 10)
+            throw new IgniteCheckedException("Evicted pages pool size should be greater than 10: " + regCfg.getName());
 
-        long maxPoolSize = plcCfg.getMaxSize() / dbCfg.getPageSize() / 10;
+        long maxPoolSize = regCfg.getMaxSize() / dbCfg.getPageSize() / 10;
 
-        if (plcCfg.getEmptyPagesPoolSize() >= maxPoolSize) {
+        if (regCfg.getEmptyPagesPoolSize() >= maxPoolSize) {
             throw new IgniteCheckedException("Evicted pages pool size should be lesser than " + maxPoolSize +
-                ": " + plcCfg.getName());
+                ": " + regCfg.getName());
         }
     }
 
     /**
-     * @param plcName MemoryPolicy name to validate.
+     * @param regName DataRegion name to validate.
      * @param observedNames Names of MemoryPolicies observed before.
      * @throws IgniteCheckedException If config is invalid.
      */
-    private static void checkPolicyName(String plcName, Collection<String> observedNames)
+    private static void checkDataRegionName(String regName, Collection<String> observedNames)
         throws IgniteCheckedException {
-        if (plcName == null || plcName.isEmpty())
-            throw new IgniteCheckedException("User-defined MemoryPolicyConfiguration must have non-null and " +
+        if (regName == null || regName.isEmpty())
+            throw new IgniteCheckedException("User-defined DataRegionConfiguration must have non-null and " +
                 "non-empty name.");
 
-        if (observedNames.contains(plcName))
-            throw new IgniteCheckedException("Two MemoryPolicies have the same name: " + plcName);
+        if (observedNames.contains(regName))
+            throw new IgniteCheckedException("Two MemoryPolicies have the same name: " + regName);
 
-        if (SYSTEM_MEMORY_POLICY_NAME.equals(plcName))
-            throw new IgniteCheckedException("'sysMemPlc' policy name is reserved for internal use.");
+        if (SYSTEM_DATA_REGION_NAME.equals(regName))
+            throw new IgniteCheckedException("'" + SYSTEM_DATA_REGION_NAME + "' policy name is reserved for internal use.");
 
-        observedNames.add(plcName);
+        observedNames.add(regName);
     }
 
     /**
@@ -589,22 +527,22 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @return collection of all configured {@link MemoryPolicy policies}.
+     * @return collection of all configured {@link DataRegion policies}.
      */
-    public Collection<MemoryPolicy> memoryPolicies() {
-        return memPlcMap != null ? memPlcMap.values() : null;
+    public Collection<DataRegion> dataRegions() {
+        return dataRegionMap != null ? dataRegionMap.values() : null;
     }
 
     /**
-     * @return MemoryMetrics for all MemoryPolicies configured in Ignite instance.
+     * @return DataRegionMetrics for all MemoryPolicies configured in Ignite instance.
      */
-    public Collection<MemoryMetrics> memoryMetrics() {
+    public Collection<DataRegionMetrics> memoryMetrics() {
         if (!F.isEmpty(memMetricsMap)) {
             // Intentionally return a collection copy to make it explicitly serializable.
-            Collection<MemoryMetrics> res = new ArrayList<>(memMetricsMap.size());
+            Collection<DataRegionMetrics> res = new ArrayList<>(memMetricsMap.size());
 
-            for (MemoryMetrics metrics : memMetricsMap.values())
-                res.add(new MemoryMetricsSnapshot(metrics));
+            for (DataRegionMetrics metrics : memMetricsMap.values())
+                res.add(new DataRegionMetricsSnapshot(metrics));
 
             return res;
         }
@@ -613,9 +551,9 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @return PersistenceMetrics if persistence is enabled or {@code null} otherwise.
+     * @return DataStorageMetrics if persistence is enabled or {@code null} otherwise.
      */
-    public PersistenceMetrics persistentStoreMetrics() {
+    public DataStorageMetrics persistentStoreMetrics() {
         return null;
     }
 
@@ -628,46 +566,46 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param memPlcName Name of {@link MemoryPolicy} to obtain {@link MemoryMetrics} for.
-     * @return {@link MemoryMetrics} snapshot for specified {@link MemoryPolicy} or {@code null} if
-     * no {@link MemoryPolicy} is configured for specified name.
+     * @param memPlcName Name of {@link DataRegion} to obtain {@link DataRegionMetrics} for.
+     * @return {@link DataRegionMetrics} snapshot for specified {@link DataRegion} or {@code null} if
+     * no {@link DataRegion} is configured for specified name.
      */
-    @Nullable public MemoryMetrics memoryMetrics(String memPlcName) {
+    @Nullable public DataRegionMetrics memoryMetrics(String memPlcName) {
         if (!F.isEmpty(memMetricsMap)) {
-            MemoryMetrics memMetrics = memMetricsMap.get(memPlcName);
+            DataRegionMetrics memMetrics = memMetricsMap.get(memPlcName);
 
             if (memMetrics == null)
                 return null;
             else
-                return new MemoryMetricsSnapshot(memMetrics);
+                return new DataRegionMetricsSnapshot(memMetrics);
         }
         else
             return 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.
+     * @param memPlcName data region name.
+     * @return {@link DataRegion} instance associated with a given {@link DataRegionConfiguration}.
+     * @throws IgniteCheckedException in case of request for unknown DataRegion.
      */
-    public MemoryPolicy memoryPolicy(String memPlcName) throws IgniteCheckedException {
+    public DataRegion dataRegion(String memPlcName) throws IgniteCheckedException {
         if (memPlcName == null)
-            return dfltMemPlc;
+            return dfltDataRegion;
 
-        if (memPlcMap == null)
+        if (dataRegionMap == null)
             return null;
 
-        MemoryPolicy plc;
+        DataRegion plc;
 
-        if ((plc = memPlcMap.get(memPlcName)) == null)
-            throw new IgniteCheckedException("Requested MemoryPolicy is not configured: " + memPlcName);
+        if ((plc = dataRegionMap.get(memPlcName)) == null)
+            throw new IgniteCheckedException("Requested DataRegion is not configured: " + memPlcName);
 
         return plc;
     }
 
     /**
-     * @param memPlcName MemoryPolicyConfiguration name.
-     * @return {@link FreeList} instance associated with a given {@link MemoryPolicyConfiguration}.
+     * @param memPlcName DataRegionConfiguration name.
+     * @return {@link FreeList} instance associated with a given {@link DataRegionConfiguration}.
      */
     public FreeList freeList(String memPlcName) {
         if (memPlcName == null)
@@ -677,8 +615,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param memPlcName MemoryPolicyConfiguration name.
-     * @return {@link ReuseList} instance associated with a given {@link MemoryPolicyConfiguration}.
+     * @param memPlcName DataRegionConfiguration name.
+     * @return {@link ReuseList} instance associated with a given {@link DataRegionConfiguration}.
      */
     public ReuseList reuseList(String memPlcName) {
         if (memPlcName == null)
@@ -689,8 +627,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
-        if (memPlcMap != null) {
-            for (MemoryPolicy memPlc : memPlcMap.values()) {
+        if (dataRegionMap != null) {
+            for (DataRegion memPlc : dataRegionMap.values()) {
                 memPlc.pageMemory().stop();
 
                 memPlc.evictionTracker().stop();
@@ -698,9 +636,9 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 unregisterMBean(memPlc.memoryMetrics().getName());
             }
 
-            memPlcMap.clear();
+            dataRegionMap.clear();
 
-            memPlcMap = null;
+            dataRegionMap = null;
         }
     }
 
@@ -718,7 +656,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
             cfg.getMBeanServer().unregisterMBean(
                 U.makeMBeanName(
                     cfg.getIgniteInstanceName(),
-                    "MemoryMetrics", name
+                    "DataRegionMetrics", name
                     ));
         }
         catch (Throwable e) {
@@ -848,13 +786,13 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     /**
      * See {@link GridCacheMapEntry#ensureFreeSpace()}
      *
-     * @param memPlc Memory policy.
+     * @param memPlc data region.
      */
-    public void ensureFreeSpace(MemoryPolicy memPlc) throws IgniteCheckedException {
+    public void ensureFreeSpace(DataRegion memPlc) throws IgniteCheckedException {
         if (memPlc == null)
             return;
 
-        MemoryPolicyConfiguration plcCfg = memPlc.config();
+        DataRegionConfiguration plcCfg = memPlc.config();
 
         if (plcCfg.getPageEvictionMode() == DataPageEvictionMode.DISABLED)
             return;
@@ -884,16 +822,16 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /**
      * @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.
+     * @param plcCfg data region with PageMemory specific parameters.
+     * @param memMetrics {@link DataRegionMetrics} object to collect memory usage metrics.
+     * @return data region instance.
      *
      * @throws IgniteCheckedException If failed to initialize swap path.
      */
-    private MemoryPolicy initMemory(
-        MemoryConfiguration memCfg,
-        MemoryPolicyConfiguration plcCfg,
-        MemoryMetricsImpl memMetrics
+    private DataRegion initMemory(
+        DataStorageConfiguration memCfg,
+        DataRegionConfiguration plcCfg,
+        DataRegionMetricsImpl memMetrics
     ) throws IgniteCheckedException {
         File allocPath = buildAllocPath(plcCfg);
 
@@ -905,15 +843,15 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
         PageMemory pageMem = createPageMemory(memProvider, memCfg, plcCfg, memMetrics);
 
-        return new MemoryPolicy(pageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg, pageMem));
+        return new DataRegion(pageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg, pageMem));
     }
 
     /**
-     * @param plc Memory Policy Configuration.
+     * @param plc data region Configuration.
      * @param pageMem Page memory.
      */
-    private PageEvictionTracker createPageEvictionTracker(MemoryPolicyConfiguration plc, PageMemory pageMem) {
-        if (plc.getPageEvictionMode() == DataPageEvictionMode.DISABLED || cctx.gridConfig().isPersistentStoreEnabled())
+    private PageEvictionTracker createPageEvictionTracker(DataRegionConfiguration plc, PageMemory pageMem) {
+        if (plc.getPageEvictionMode() == DataPageEvictionMode.DISABLED || CU.isPersistenceEnabled(cctx.gridConfig()))
             return new NoOpPageEvictionTracker();
 
         assert pageMem instanceof PageMemoryNoStoreImpl : pageMem.getClass();
@@ -936,12 +874,12 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     /**
      * Builds allocation path for memory mapped file to be used with PageMemory.
      *
-     * @param plc MemoryPolicyConfiguration.
+     * @param plc DataRegionConfiguration.
      *
      * @throws IgniteCheckedException If resolving swap directory fails.
      */
-    @Nullable private File buildAllocPath(MemoryPolicyConfiguration plc) throws IgniteCheckedException {
-        String path = plc.getSwapFilePath();
+    @Nullable private File buildAllocPath(DataRegionConfiguration plc) throws IgniteCheckedException {
+        String path = plc.getSwapPath();
 
         if (path == null)
             return null;
@@ -962,15 +900,15 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      *
      * @param memProvider Memory provider.
      * @param memCfg Memory configuartion.
-     * @param memPlcCfg Memory policy configuration.
-     * @param memMetrics MemoryMetrics to collect memory usage metrics.
+     * @param memPlcCfg data region configuration.
+     * @param memMetrics DataRegionMetrics to collect memory usage metrics.
      * @return PageMemory instance.
      */
     protected PageMemory createPageMemory(
         DirectMemoryProvider memProvider,
-        MemoryConfiguration memCfg,
-        MemoryPolicyConfiguration memPlcCfg,
-        MemoryMetricsImpl memMetrics
+        DataStorageConfiguration memCfg,
+        DataRegionConfiguration memPlcCfg,
+        DataRegionMetricsImpl memMetrics
     ) {
         memMetrics.persistenceEnabled(false);
 
@@ -1003,14 +941,14 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        if (cctx.kernalContext().clientNode() && cctx.kernalContext().config().getMemoryConfiguration() == null)
+        if (cctx.kernalContext().clientNode() && cctx.kernalContext().config().getDataStorageConfiguration() == null)
             return;
 
-        MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();
+        DataStorageConfiguration memCfg = cctx.kernalContext().config().getDataStorageConfiguration();
 
         assert memCfg != null;
 
-        initPageMemoryPolicies(memCfg);
+        initDataRegions(memCfg);
 
         registerMetricsMBeans();
 
@@ -1025,10 +963,10 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @return Name of MemoryPolicyConfiguration for internal caches.
+     * @return Name of DataRegionConfiguration for internal caches.
      */
-    public String systemMemoryPolicyName() {
-        return SYSTEM_MEMORY_POLICY_NAME;
+    public String systemDateRegionName() {
+        return SYSTEM_DATA_REGION_NAME;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java
deleted file mode 100644
index 3261874..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java
+++ /dev/null
@@ -1,286 +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.processors.cache.persistence;
-
-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.ratemetrics.HitRateMetrics;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteOutClosure;
-import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
-
-/**
- *
- */
-public class MemoryMetricsImpl implements MemoryMetrics {
-    /** */
-    private final IgniteOutClosure<Float> fillFactorProvider;
-
-    /** */
-    private final LongAdder8 totalAllocatedPages = new LongAdder8();
-
-    /**
-     * Counter for number of pages occupied by large entries (one entry is larger than one page).
-     */
-    private final LongAdder8 largeEntriesPages = new LongAdder8();
-
-    /** Counter for number of dirty pages. */
-    private LongAdder8 dirtyPages = new LongAdder8();
-
-    /** */
-    private volatile boolean metricsEnabled;
-
-    /** */
-    private boolean persistenceEnabled;
-
-    /** */
-    private volatile int subInts;
-
-    /** Allocation rate calculator. */
-    private volatile HitRateMetrics allocRate = new HitRateMetrics(60_000, 5);
-
-    /** */
-    private volatile HitRateMetrics pageReplaceRate = new HitRateMetrics(60_000, 5);
-
-    /** */
-    private final MemoryPolicyConfiguration memPlcCfg;
-
-    /** */
-    private PageMemory pageMem;
-
-    /** Time interval (in milliseconds) when allocations/evictions are counted to calculate rate. */
-    private volatile long rateTimeInterval;
-
-    /**
-     * @param memPlcCfg MemoryPolicyConfiguration.
-    */
-    public MemoryMetricsImpl(MemoryPolicyConfiguration memPlcCfg) {
-        this(memPlcCfg, null);
-    }
-
-    /**
-     * @param memPlcCfg MemoryPolicyConfiguration.
-     */
-    public MemoryMetricsImpl(MemoryPolicyConfiguration memPlcCfg, @Nullable IgniteOutClosure<Float> fillFactorProvider) {
-        this.memPlcCfg = memPlcCfg;
-        this.fillFactorProvider = fillFactorProvider;
-
-        metricsEnabled = memPlcCfg.isMetricsEnabled();
-
-        rateTimeInterval = memPlcCfg.getRateTimeInterval();
-
-        subInts = memPlcCfg.getSubIntervals();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getName() {
-        return U.maskName(memPlcCfg.getName());
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getTotalAllocatedPages() {
-        return metricsEnabled ? totalAllocatedPages.longValue() : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getAllocationRate() {
-        if (!metricsEnabled)
-            return 0;
-
-        return ((float) allocRate.getRate()) / rateTimeInterval;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getEvictionRate() {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getLargeEntriesPagesPercentage() {
-        if (!metricsEnabled)
-            return 0;
-
-        return totalAllocatedPages.longValue() != 0 ?
-                (float) largeEntriesPages.doubleValue() / totalAllocatedPages.longValue()
-                : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getPagesFillFactor() {
-        if (!metricsEnabled || fillFactorProvider == null)
-            return 0;
-
-        return fillFactorProvider.apply();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getDirtyPages() {
-        if (!metricsEnabled || !persistenceEnabled)
-            return 0;
-
-        return dirtyPages.longValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getPagesReplaceRate() {
-        if (!metricsEnabled || !persistenceEnabled)
-            return 0;
-
-        return ((float) pageReplaceRate.getRate()) / rateTimeInterval;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getPhysicalMemoryPages() {
-        if (!metricsEnabled || !persistenceEnabled)
-            return 0;
-
-        assert pageMem != null;
-
-        return pageMem.loadedPages();
-    }
-
-    /**
-     * Updates pageReplaceRate metric.
-     */
-    public void updatePageReplaceRate() {
-        if (metricsEnabled)
-            pageReplaceRate.onHit();
-    }
-
-    /**
-     * Increments dirtyPages counter.
-     */
-    public void incrementDirtyPages() {
-        if (metricsEnabled)
-            dirtyPages.increment();
-    }
-
-    /**
-     * Decrements dirtyPages counter.
-     */
-    public void decrementDirtyPages() {
-        if (metricsEnabled)
-            dirtyPages.decrement();
-    }
-
-    /**
-     * Resets dirtyPages counter to zero.
-     */
-    public void resetDirtyPages() {
-        if (metricsEnabled)
-            dirtyPages.reset();
-    }
-
-    /**
-     * Increments totalAllocatedPages counter.
-     */
-    public void incrementTotalAllocatedPages() {
-        if (metricsEnabled) {
-            totalAllocatedPages.increment();
-
-            updateAllocationRateMetrics();
-        }
-    }
-
-    /**
-     *
-     */
-    private void updateAllocationRateMetrics() {
-        allocRate.onHit();
-    }
-
-    /**
-     * @param intervalNum Interval number.
-     */
-    private long subInt(int intervalNum) {
-        return (rateTimeInterval * intervalNum) / subInts;
-    }
-
-    /**
-     *
-     */
-    public void incrementLargeEntriesPages() {
-        if (metricsEnabled)
-            largeEntriesPages.increment();
-    }
-
-    /**
-     *
-     */
-    public void decrementLargeEntriesPages() {
-        if (metricsEnabled)
-            largeEntriesPages.decrement();
-    }
-
-    /**
-     * Enable metrics.
-     */
-    public void enableMetrics() {
-        metricsEnabled = true;
-    }
-
-    /**
-     * Disable metrics.
-     */
-    public void disableMetrics() {
-        metricsEnabled = false;
-    }
-
-    /**
-     * @param persistenceEnabled Persistence enabled.
-     */
-    public void persistenceEnabled(boolean persistenceEnabled) {
-        this.persistenceEnabled = persistenceEnabled;
-    }
-
-    /**
-     * @param pageMem Page mem.
-     */
-    public void pageMemory(PageMemory pageMem) {
-        this.pageMem = pageMem;
-    }
-
-    /**
-     * @param rateTimeInterval Time interval (in milliseconds) used to calculate allocation/eviction rate.
-     */
-    public void rateTimeInterval(long rateTimeInterval) {
-        this.rateTimeInterval = rateTimeInterval;
-
-        allocRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-        pageReplaceRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-    }
-
-    /**
-     * Sets number of subintervals the whole rateTimeInterval will be split into to calculate allocation rate.
-     *
-     * @param subInts Number of subintervals.
-     */
-    public void subIntervals(int subInts) {
-        assert subInts > 0;
-
-        if (this.subInts == subInts)
-            return;
-
-        if (rateTimeInterval / subInts < 10)
-            subInts = (int) rateTimeInterval / 10;
-
-        allocRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-        pageReplaceRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsMXBeanImpl.java
deleted file mode 100644
index 392f83f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsMXBeanImpl.java
+++ /dev/null
@@ -1,131 +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.processors.cache.persistence;
-
-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 long getDirtyPages() {
-        return memMetrics.getDirtyPages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getPagesReplaceRate() {
-        return memMetrics.getPagesReplaceRate();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getPhysicalMemoryPages() {
-        return memMetrics.getPhysicalMemoryPages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void rateTimeInterval(long rateTimeInterval) {
-        if (rateTimeInterval < 1000)
-            throw new IllegalArgumentException("rateTimeInterval property must be positive " +
-                "and greater than 1_000 milliseconds (one second)");
-
-        memMetrics.rateTimeInterval(rateTimeInterval);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void subIntervals(int subInts) {
-        if (subInts <= 1)
-            throw new IllegalArgumentException("subIntervals property must be positive " +
-                "and greater than one");
-
-        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();
-    }
-}