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

ignite git commit: IGNITE-5024 - expandable memory space

Repository: ignite
Updated Branches:
  refs/heads/ignite-5024 [created] 67ba03bc2


IGNITE-5024 - expandable memory space


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

Branch: refs/heads/ignite-5024
Commit: 67ba03bc2c2181f9d61caddd5981d24ef50cc10a
Parents: a826c61
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Apr 20 19:38:24 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Apr 20 19:38:24 2017 +0300

----------------------------------------------------------------------
 .../benchmarks/jmh/tree/BPlusTreeBenchmark.java |   8 +-
 .../configuration/DataPageEvictionMode.java     |   4 +-
 .../configuration/MemoryConfiguration.java      |   7 +-
 .../MemoryPolicyConfiguration.java              |  41 ++-
 .../ignite/internal/mem/DirectMemory.java       |  55 ----
 .../internal/mem/DirectMemoryProvider.java      |  19 +-
 .../mem/file/MappedFileMemoryProvider.java      | 153 +++-------
 .../mem/unsafe/UnsafeMemoryProvider.java        |  69 ++---
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 286 +++++++++++--------
 .../IgniteCacheDatabaseSharedManager.java       |  81 ++----
 .../cache/database/MemoryMetricsImpl.java       |   2 +-
 .../evict/PageAbstractEvictionTracker.java      |   2 +-
 .../evict/Random2LruPageEvictionTracker.java    |   2 +-
 .../evict/RandomLruPageEvictionTracker.java     |   2 +-
 .../node/VisorMemoryPolicyConfiguration.java    |   2 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |   3 +-
 .../cache/CacheConfigurationLeakTest.java       |   2 +-
 .../CacheMemoryPolicyConfigurationTest.java     |   8 +-
 .../cache/MemoryPolicyConfigValidationTest.java |   2 +-
 .../paged/PageEvictionAbstractTest.java         |   2 +-
 .../TxPessimisticDeadlockDetectionTest.java     |   2 +-
 .../processors/database/BPlusTreeSelfTest.java  |  12 +-
 .../database/FreeListImplSelfTest.java          |   4 +-
 .../database/IgniteDbDynamicCacheSelfTest.java  |   2 +-
 .../database/MetadataStorageSelfTest.java       |   2 +-
 .../index/DynamicIndexAbstractSelfTest.java     |   2 +-
 .../h2/database/InlineIndexHelperTest.java      |  18 +-
 27 files changed, 363 insertions(+), 429 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
index 5833e1f..d4e2c9f 100644
--- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java
@@ -210,7 +210,7 @@ public class BPlusTreeBenchmark extends JmhAbstractBenchmark {
             sizes[i] = 1024 * MB / CPUS;
 
         PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+            new UnsafeMemoryProvider(new JavaLogger()),
             null,
             PAGE_SIZE,
             null,
@@ -281,8 +281,7 @@ public class BPlusTreeBenchmark extends JmhAbstractBenchmark {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             return PageUtils.getLong(pageAddr, offset(idx));
         }
     }
@@ -318,8 +317,7 @@ public class BPlusTreeBenchmark extends JmhAbstractBenchmark {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             return PageUtils.getLong(pageAddr, offset(idx));
         }
     }

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/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 cd7c1fe..a353eef 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
@@ -62,8 +62,11 @@ public class MemoryConfiguration implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Default memory policy start size (256 MB). */
+    public static final long DFLT_MEMORY_POLICY_INITIAL_SIZE = 256 * 1024 * 1024;
+
     /** Default memory policy's size (1 GB). */
-    public static final long DFLT_MEMORY_POLICY_SIZE = 1024 * 1024 * 1024;
+    public static final long DFLT_MEMORY_POLICY_MAX_SIZE = 1024 * 1024 * 1024;
 
     /** Default size of a memory chunk for the system cache (100 MB). */
     public static final long DFLT_SYS_CACHE_MEM_SIZE = 100 * 1024 * 1024;
@@ -165,7 +168,7 @@ public class MemoryConfiguration implements Serializable {
         MemoryPolicyConfiguration memPlc = new MemoryPolicyConfiguration();
 
         memPlc.setName(null);
-        memPlc.setSize(DFLT_MEMORY_POLICY_SIZE);
+        memPlc.setMaxSize(DFLT_MEMORY_POLICY_MAX_SIZE);
 
         return memPlc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
index 6d5fe8c..2f52fe1 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -63,8 +63,11 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /** Memory policy name. */
     private String name;
 
+    /** Memory policy start size. */
+    private long initialSize = MemoryConfiguration.DFLT_MEMORY_POLICY_INITIAL_SIZE;
+
     /** Memory policy maximum size. */
-    private long size;
+    private long maxSize = MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE;
 
     /** An optional path to a memory mapped file for this memory policy. */
     private String swapFilePath;
@@ -107,16 +110,42 @@ public final class MemoryPolicyConfiguration implements Serializable {
      *
      * @return Size in bytes.
      */
-    public long getSize() {
-        return size;
+    public long getMaxSize() {
+        return maxSize;
     }
 
     /**
      * Sets maximum memory region size defined by this memory policy. The total size should not be less than 10 MB
      * due to the internal data structures overhead.
+     *
+     * @param maxSize Maxumum memory policy size in bytes.
+     * @return {@code this} for chaining.
+     */
+    public MemoryPolicyConfiguration setMaxSize(long maxSize) {
+        this.maxSize = maxSize;
+
+        return this;
+    }
+
+    /**
+     * Gets initial memory region size defined by this memory policy. When the used memory size exceeds this value,
+     * new chunks of memory will be allocated.
+     *
+     * @return Memory policy start size.
+     */
+    public long getInitialSize() {
+        return initialSize;
+    }
+
+    /**
+     * Sets initial memory region size defined by this memory policy. When the used memory size exceeds this value,
+     * new chunks of memory will be allocated.
+     *
+     * @param initialSize Memory policy initial size.
+     * @return {@code this} for chaining.
      */
-    public MemoryPolicyConfiguration setSize(long size) {
-        this.size = size;
+    public MemoryPolicyConfiguration setInitialSize(long initialSize) {
+        this.initialSize = initialSize;
 
         return this;
     }
@@ -146,7 +175,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /**
      * Gets memory pages eviction mode. If {@link DataPageEvictionMode#DISABLED} is used (default) then an out of
      * memory exception will be thrown if the memory region usage, defined by this memory policy, goes beyond its
-     * capacity which is {@link #getSize()}.
+     * capacity which is {@link #getMaxSize()}.
      *
      * @return Memory pages eviction algorithm. {@link DataPageEvictionMode#DISABLED} used by default.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java
deleted file mode 100644
index 2211a4c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/mem/DirectMemory.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.mem;
-
-import java.util.List;
-
-/**
- *
- */
-public class DirectMemory {
-    /** Will be set if  */
-    private boolean restored;
-
-    /** */
-    private List<DirectMemoryRegion> regions;
-
-    /**
-     * @param restored Restored flag.
-     * @param regions Memory fragments.
-     */
-    public DirectMemory(boolean restored, List<DirectMemoryRegion> regions) {
-        this.restored = restored;
-        this.regions = regions;
-    }
-
-    /**
-     * @return Restored flag. If {@code true}, the memory fragments were successfully restored since the previous
-     *      usage and can be reused.
-     */
-    public boolean restored() {
-        return restored;
-    }
-
-    /**
-     * @return Memory fragments.
-     */
-    public List<DirectMemoryRegion> regions() {
-        return regions;
-    }
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index 872e496..d983790 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
@@ -21,11 +21,11 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
-import org.apache.ignite.internal.mem.DirectMemory;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.IgniteOutOfMemoryException;
@@ -38,7 +38,6 @@ import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lifecycle.LifecycleAware;
 
 import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
 
@@ -98,6 +97,21 @@ public class PageMemoryNoStoreImpl implements PageMemory {
      */
     public static final int PAGE_OVERHEAD = LOCK_OFFSET + OffheapReadWriteLock.LOCK_SIZE;
 
+    /** Number of bits required to store segment index. */
+    private static final int SEG_BITS = 4;
+
+    /** Number of bits required to store segment index. */
+    private static final int SEG_CNT = (1 << SEG_BITS);
+
+    /** Number of bits left to store page index. */
+    private static final int IDX_BITS = PageIdUtils.PAGE_IDX_SIZE - SEG_BITS;
+
+    /** Segment mask. */
+    private static final int SEG_MASK = ~(-1 << SEG_BITS);
+
+    /** Index mask. */
+    private static final int IDX_MASK = ~(-1 << IDX_BITS);
+
     /** Page size. */
     private int sysPageSize;
 
@@ -113,20 +127,14 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     /** Object to collect memory usage metrics. */
     private final MemoryMetricsImpl memMetrics;
 
-    /** Segments array. */
-    private Segment[] segments;
-
-    /** Number of bits required to store segment index. */
-    private int segBits;
-
-    /** Number of bits left to store page index. */
-    private int idxBits;
-
     /** */
-    private int segMask;
+    private AtomicLong freePageListHead = new AtomicLong(INVALID_REL_PTR);
+
+    /** Segments array. */
+    private volatile Segment[] segments;
 
     /** */
-    private int idxMask;
+    private final AtomicInteger allocatedPages = new AtomicInteger();
 
     /** */
     private AtomicInteger selector = new AtomicInteger();
@@ -174,29 +182,28 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteException {
-        if (directMemoryProvider instanceof LifecycleAware)
-            ((LifecycleAware)directMemoryProvider).start();
+        long startSize = memoryPolicyCfg.getInitialSize();
+        long maxSize = memoryPolicyCfg.getMaxSize();
 
-        DirectMemory memory = directMemoryProvider.memory();
+        long[] chunks = new long[SEG_CNT];
 
-        segments = new Segment[memory.regions().size()];
+        chunks[0] = startSize;
 
-        for (int i = 0; i < segments.length; i++) {
-            segments[i] = new Segment(i, memory.regions().get(i));
+        long total = startSize;
 
-            segments[i].init();
-        }
+        long allocChunkSize = Math.max((maxSize - startSize) / (SEG_CNT - 1), 256 * 1024 * 1024);
 
-        segBits = Integer.SIZE - Integer.numberOfLeadingZeros(segments.length - 1);
+        for (int i = 1; i < SEG_CNT; i++) {
+            long allocSize = Math.min(allocChunkSize, maxSize - total);
 
-        // Reserve at least one bit for segment.
-        if (segBits == 0)
-            segBits = 1;
+            chunks[i] = allocSize;
+
+            total += allocSize;
+        }
 
-        idxBits = PageIdUtils.PAGE_IDX_SIZE - segBits;
+        directMemoryProvider.initialize(chunks);
 
-        segMask = ~(-1 << segBits);
-        idxMask = ~(-1 << idxBits);
+        addSegment(null);
     }
 
     /** {@inheritDoc} */
@@ -205,8 +212,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         if (log.isDebugEnabled())
             log.debug("Stopping page memory.");
 
-        if (directMemoryProvider instanceof LifecycleAware)
-            ((LifecycleAware)directMemoryProvider).stop();
+        directMemoryProvider.shutdown();
 
         if (directMemoryProvider instanceof Closeable) {
             try {
@@ -227,35 +233,34 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     @Override public long allocatePage(int cacheId, int partId, byte flags) {
         memMetrics.incrementTotalAllocatedPages();
 
-        long relPtr = INVALID_REL_PTR;
+        long relPtr = borrowFreePage();
         long absPtr = 0;
 
-        for (Segment seg : segments) {
-            relPtr = seg.borrowFreePage();
+        if (relPtr != INVALID_REL_PTR) {
+            int pageIdx = PageIdUtils.pageIndex(relPtr);
 
-            if (relPtr != INVALID_REL_PTR) {
-                absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
+            Segment seg = segment(pageIdx);
 
-                break;
-            }
+            absPtr = seg.absolute(pageIdx);
         }
 
         // No segments contained a free page.
         if (relPtr == INVALID_REL_PTR) {
-            int segAllocIdx = nextRoundRobinIndex();
-
-            for (int i = 0; i < segments.length; i++) {
-                int idx = (segAllocIdx + i) % segments.length;
+            Segment[] seg0 = segments;
+            Segment allocSeg = seg0[seg0.length - 1];
 
-                Segment seg = segments[idx];
-
-                relPtr = seg.allocateFreePage(flags);
+            while (allocSeg != null) {
+                relPtr = allocSeg.allocateFreePage(flags);
 
                 if (relPtr != INVALID_REL_PTR) {
-                    absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
+                    if (relPtr != INVALID_REL_PTR) {
+                        absPtr = allocSeg.absolute(PageIdUtils.pageIndex(relPtr));
 
-                    break;
+                        break;
+                    }
                 }
+                else
+                    allocSeg = addSegment(seg0);
             }
         }
 
@@ -263,7 +268,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             throw new IgniteOutOfMemoryException("Not enough memory allocated " +
                 "(consider increasing memory policy size or enabling evictions) " +
                 "[policyName=" + memoryPolicyCfg.getName() +
-                ", size=" + U.readableSize(memoryPolicyCfg.getSize(), true) + "]"
+                ", size=" + U.readableSize(memoryPolicyCfg.getMaxSize(), true) + "]"
             );
 
         assert (relPtr & ~PageIdUtils.PAGE_IDX_MASK) == 0;
@@ -281,9 +286,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
     /** {@inheritDoc} */
     @Override public boolean freePage(int cacheId, long pageId) {
-        Segment seg = segment(PageIdUtils.pageIndex(pageId));
-
-        seg.releaseFreePage(pageId);
+        releaseFreePage(pageId);
 
         return true;
     }
@@ -382,7 +385,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
      * @return Segment index.
      */
     private int segmentIndex(long pageIdx) {
-        return (int)((pageIdx >> idxBits) & segMask);
+        return (int)((pageIdx >> IDX_BITS) & SEG_MASK);
     }
 
     /**
@@ -393,8 +396,8 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     private long fromSegmentIndex(int segIdx, long pageIdx) {
         long res = 0;
 
-        res = (res << segBits) | (segIdx & segMask);
-        res = (res << idxBits) | (pageIdx & idxMask);
+        res = (res << SEG_BITS) | (segIdx & SEG_MASK);
+        res = (res << IDX_BITS) | (pageIdx & IDX_MASK);
 
         return res;
     }
@@ -428,7 +431,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /** {@inheritDoc} */
-    public long readLockForce(int cacheId, long pageId, long page) {
+    @Override public long readLockForce(int cacheId, long pageId, long page) {
         if (rwLock.readLock(page + LOCK_OFFSET, -1))
             return page + PAGE_OVERHEAD;
 
@@ -470,6 +473,109 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /**
+     * @param pageId Page ID to release.
+     */
+    private void releaseFreePage(long pageId) {
+        int pageIdx = PageIdUtils.pageIndex(pageId);
+
+        // Clear out flags and file ID.
+        long relPtr = PageIdUtils.pageId(0, (byte)0, pageIdx);
+
+        Segment seg = segment(pageIdx);
+
+        long absPtr = seg.absolute(pageIdx);
+
+        // Second, write clean relative pointer instead of page ID.
+        writePageId(absPtr, relPtr);
+
+        // Third, link the free page.
+        while (true) {
+            long freePageRelPtrMasked = freePageListHead.get();
+
+            long freePageRelPtr = freePageRelPtrMasked & RELATIVE_PTR_MASK;
+
+            GridUnsafe.putLong(absPtr, freePageRelPtr);
+
+            if (freePageListHead.compareAndSet(freePageRelPtrMasked, relPtr)) {
+                allocatedPages.decrementAndGet();
+
+                return;
+            }
+        }
+    }
+
+    /**
+     * @return Relative pointer to a free page that was borrowed from the allocated pool.
+     */
+    private long borrowFreePage() {
+        while (true) {
+            long freePageRelPtrMasked = freePageListHead.get();
+
+            long freePageRelPtr = freePageRelPtrMasked & ADDRESS_MASK;
+
+            if (freePageRelPtr != INVALID_REL_PTR) {
+                int pageIdx = PageIdUtils.pageIndex(freePageRelPtr);
+
+                Segment seg = segment(pageIdx);
+
+                long freePageAbsPtr = seg.absolute(pageIdx);
+                long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
+                long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
+
+                if (freePageListHead.compareAndSet(freePageRelPtrMasked, nextFreePageRelPtr | cnt)) {
+                    GridUnsafe.putLong(freePageAbsPtr, PAGE_MARKER);
+
+                    allocatedPages.incrementAndGet();
+
+                    return freePageRelPtr;
+                }
+            }
+            else
+                return INVALID_REL_PTR;
+        }
+    }
+
+    /**
+     * Attempts to add a new memory segment.
+     *
+     * @param oldRef Old segments array. If this method observes another segments array, it will allocate a new
+     *      segment (if possible). If the array has already been updated, it will return the last element in the
+     *      new array.
+     * @return Added segment, if successfull, {@code null} if failed to add.
+     */
+    private synchronized Segment addSegment(Segment[] oldRef) {
+        if (segments == oldRef) {
+            DirectMemoryRegion region = directMemoryProvider.nextRegion();
+
+            // No more memory is available.
+            if (region == null)
+                return null;
+
+            if (oldRef != null) {
+                if (log.isInfoEnabled())
+                    log.info("Allocated next memory segment [plcName=" + memoryPolicyCfg.getName() +
+                        ", chunkSize=" + U.readableSize(region.size(), true) + ']');
+            }
+
+            Segment[] newRef = new Segment[oldRef == null ? 1 : oldRef.length + 1];
+
+            if (oldRef != null)
+                System.arraycopy(oldRef, 0, newRef, 0, oldRef.length);
+
+            Segment allocated = new Segment(newRef.length - 1, region);
+
+            allocated.init();
+
+            newRef[newRef.length - 1] = allocated;
+
+            segments = newRef;
+        }
+
+        // Only this synchronized method writes to segments, so it is safe to read twice.
+        return segments[segments.length - 1];
+    }
+
+    /**
      *
      */
     private class Segment extends ReentrantReadWriteLock {
@@ -482,9 +588,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         /** Direct memory chunk. */
         private DirectMemoryRegion region;
 
-        /** Pointer to the address of the free page list. */
-        private long freePageListPtr;
-
         /** Last allocated page index. */
         private long lastAllocatedIdxPtr;
 
@@ -492,9 +595,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         private long pagesBase;
 
         /** */
-        private final AtomicInteger allocatedPages;
-
-        /** */
         private final AtomicInteger acquiredPages;
 
         /**
@@ -505,7 +605,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             this.idx = idx;
             this.region = region;
 
-            allocatedPages = new AtomicInteger();
             acquiredPages = new AtomicInteger();
         }
 
@@ -515,10 +614,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         private void init() {
             long base = region.address();
 
-            freePageListPtr = base;
-
-            base += 8;
-
             lastAllocatedIdxPtr = base;
 
             base += 8;
@@ -526,7 +621,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             // Align by 8 bytes.
             pagesBase = (base + 7) & ~0x7;
 
-            GridUnsafe.putLong(freePageListPtr, INVALID_REL_PTR);
             GridUnsafe.putLong(lastAllocatedIdxPtr, 0);
         }
 
@@ -556,7 +650,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
          * @return Absolute pointer.
          */
         private long absolute(int pageIdx) {
-            pageIdx &= idxMask;
+            pageIdx &= IDX_MASK;
 
             long off = ((long)pageIdx) * sysPageSize;
 
@@ -578,64 +672,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         }
 
         /**
-         * @param pageId Page ID to release.
-         */
-        private void releaseFreePage(long pageId) {
-            int pageIdx = PageIdUtils.pageIndex(pageId);
-
-            // Clear out flags and file ID.
-            long relPtr = PageIdUtils.pageId(0, (byte)0, pageIdx);
-
-            long absPtr = absolute(pageIdx);
-
-            // Second, write clean relative pointer instead of page ID.
-            writePageId(absPtr, relPtr);
-
-            // Third, link the free page.
-            while (true) {
-                long freePageRelPtrMasked = GridUnsafe.getLong(freePageListPtr);
-
-                long freePageRelPtr = freePageRelPtrMasked & RELATIVE_PTR_MASK;
-
-                GridUnsafe.putLong(absPtr, freePageRelPtr);
-
-                if (GridUnsafe.compareAndSwapLong(null, freePageListPtr, freePageRelPtrMasked, relPtr)) {
-                    allocatedPages.decrementAndGet();
-
-                    return;
-                }
-            }
-        }
-
-        /**
-         * @return Relative pointer to a free page that was borrowed from the allocated pool.
-         */
-        private long borrowFreePage() {
-            while (true) {
-                long freePageRelPtrMasked = GridUnsafe.getLong(freePageListPtr);
-
-                long freePageRelPtr = freePageRelPtrMasked & ADDRESS_MASK;
-                long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
-
-                if (freePageRelPtr != INVALID_REL_PTR) {
-                    long freePageAbsPtr = absolute(PageIdUtils.pageIndex(freePageRelPtr));
-
-                    long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
-
-                    if (GridUnsafe.compareAndSwapLong(null, freePageListPtr, freePageRelPtrMasked, nextFreePageRelPtr | cnt)) {
-                        GridUnsafe.putLong(freePageAbsPtr, PAGE_MARKER);
-
-                        allocatedPages.incrementAndGet();
-
-                        return freePageRelPtr;
-                    }
-                }
-                else
-                    return INVALID_REL_PTR;
-            }
-        }
-
-        /**
          * @param tag Tag to initialize RW lock.
          * @return Relative pointer of the allocated page.
          * @throws GridOffHeapOutOfMemoryException If failed to allocate.

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/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 e6fe7cd..d7e1e17 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
@@ -209,7 +209,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
             else {
                 //reserve additional place for system memory policy only
                 memPlcMap = U.newHashMap(memPlcsCfgs.length + 1);
-                memMetricsMap = U.newHashMap(memPlcsCfgs.length + 1);;
+                memMetricsMap = U.newHashMap(memPlcsCfgs.length + 1);
             }
 
             for (MemoryPolicyConfiguration memPlcCfg : memPlcsCfgs) {
@@ -238,7 +238,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     /**
      * @param memMetrics Mem metrics.
      */
-    private void registerMetricsMBean(MemoryMetricsImpl memMetrics) {
+    private void registerMetricsMBean(MemoryMetricsMXBean memMetrics) {
         IgniteConfiguration cfg = cctx.gridConfig();
 
         try {
@@ -251,7 +251,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                     MemoryMetricsMXBean.class);
         }
         catch (JMException e) {
-            log.warning("Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'");
+            U.error(log, "Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'", e);
         }
     }
 
@@ -271,7 +271,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         MemoryPolicyConfiguration res = new MemoryPolicyConfiguration();
 
         res.setName(SYSTEM_MEMORY_POLICY_NAME);
-        res.setSize(sysCacheMemSize);
+        res.setInitialSize(sysCacheMemSize);
+        res.setMaxSize(sysCacheMemSize);
 
         return res;
     }
@@ -304,12 +305,14 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param plcNames All MemoryPolicy names.
      * @throws IgniteCheckedException In case of validation violation.
      */
-    private static void checkDefaultPolicyConfiguration(String dfltPlcName, Set<String> plcNames) throws IgniteCheckedException {
+    private static void checkDefaultPolicyConfiguration(String dfltPlcName, Collection<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);
+                throw new IgniteCheckedException("User-defined default MemoryPolicy name must be presented " +
+                    "among configured MemoryPolices: " + dfltPlcName);
         }
     }
 
@@ -318,8 +321,12 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @throws IgniteCheckedException If config is invalid.
      */
     private static void checkPolicySize(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
-        if (plcCfg.getSize() < MIN_PAGE_MEMORY_SIZE)
+        if (plcCfg.getMaxSize() < MIN_PAGE_MEMORY_SIZE)
             throw new IgniteCheckedException("MemoryPolicy must have size more than 1MB: " + plcCfg.getName());
+
+        if (plcCfg.getMaxSize() > plcCfg.getInitialSize())
+            throw new IgniteCheckedException("MemoryPolicy maxSize must not be smaller than " +
+                "initialSize: " + plcCfg.getName());
     }
 
     /**
@@ -340,7 +347,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         if (plcCfg.getEmptyPagesPoolSize() <= 10)
             throw new IgniteCheckedException("Evicted pages pool size should be greater than 10: " + plcCfg.getName());
 
-        long maxPoolSize = plcCfg.getSize() / dbCfg.getPageSize() / 10;
+        long maxPoolSize = plcCfg.getMaxSize() / dbCfg.getPageSize() / 10;
 
         if (plcCfg.getEmptyPagesPoolSize() >= maxPoolSize) {
             throw new IgniteCheckedException("Evicted pages pool size should be lesser than " + maxPoolSize +
@@ -353,9 +360,11 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param observedNames Names of MemoryPolicies observed before.
      * @throws IgniteCheckedException If config is invalid.
      */
-    private static void checkPolicyName(String plcName, Set<String> observedNames) throws IgniteCheckedException {
+    private static void checkPolicyName(String plcName, Collection<String> observedNames)
+        throws IgniteCheckedException {
         if (plcName == null || plcName.isEmpty())
-            throw new IgniteCheckedException("User-defined MemoryPolicyConfiguration must have non-null and non-empty name.");
+            throw new IgniteCheckedException("User-defined MemoryPolicyConfiguration must have non-null and " +
+                "non-empty name.");
 
         if (observedNames.contains(plcName))
             throw new IgniteCheckedException("Two MemoryPolicies have the same name: " + plcName);
@@ -564,7 +573,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         if (plcCfg.getPageEvictionMode() == DataPageEvictionMode.DISABLED)
             return;
 
-        long memorySize = plcCfg.getSize();
+        long memorySize = plcCfg.getMaxSize();
 
         PageMemory pageMem = memPlc.pageMemory();
 
@@ -588,29 +597,23 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param dbCfg memory configuration with common parameters.
-     * @param plc memory policy with PageMemory specific parameters.
+     * @param memCfg memory configuration with common parameters.
+     * @param plcCfg memory policy with PageMemory specific parameters.
      * @param memMetrics {@link MemoryMetrics} object to collect memory usage metrics.
      * @return Memory policy instance.
      */
-    private MemoryPolicy initMemory(MemoryConfiguration dbCfg, MemoryPolicyConfiguration plc, MemoryMetricsImpl memMetrics) {
-        long[] sizes = calculateFragmentSizes(
-                dbCfg.getConcurrencyLevel(),
-                plc.getSize());
-
-        File allocPath = buildAllocPath(plc);
+    private MemoryPolicy initMemory(MemoryConfiguration memCfg, MemoryPolicyConfiguration plcCfg, MemoryMetricsImpl memMetrics) {
+        File allocPath = buildAllocPath(plcCfg);
 
         DirectMemoryProvider memProvider = allocPath == null ?
-            new UnsafeMemoryProvider(sizes) :
+            new UnsafeMemoryProvider(log) :
             new MappedFileMemoryProvider(
                 log,
-                allocPath,
-                true,
-                sizes);
+                allocPath);
 
-        PageMemory pageMem = createPageMemory(memProvider, dbCfg.getPageSize(), plc, memMetrics);
+        PageMemory pageMem = createPageMemory(memProvider, memCfg, plcCfg, memMetrics);
 
-        return new MemoryPolicy(pageMem, plc, memMetrics, createPageEvictionTracker(plc, pageMem));
+        return new MemoryPolicy(pageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg, pageMem));
     }
 
     /**
@@ -632,28 +635,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * Calculate fragment sizes for a cache with given size and concurrency level.
-     * @param concLvl Concurrency level.
-     * @param cacheSize Cache size.
-     */
-    protected long[] calculateFragmentSizes(int concLvl, long cacheSize) {
-        if (concLvl < 1)
-            concLvl = Runtime.getRuntime().availableProcessors();
-
-        long fragmentSize = cacheSize / concLvl;
-
-        if (fragmentSize < 1024 * 1024)
-            fragmentSize = 1024 * 1024;
-
-        long[] sizes = new long[concLvl];
-
-        for (int i = 0; i < concLvl; i++)
-            sizes[i] = fragmentSize;
-
-        return sizes;
-    }
-
-    /**
      * Builds allocation path for memory mapped file to be used with PageMemory.
      *
      * @param plc MemoryPolicyConfiguration.
@@ -675,18 +656,18 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * Creates PageMemory with given size and memory provider.
      *
      * @param memProvider Memory provider.
-     * @param pageSize Page size.
+     * @param memCfg Memory configuartion.
      * @param memPlcCfg Memory policy configuration.
      * @param memMetrics MemoryMetrics to collect memory usage metrics.
      * @return PageMemory instance.
      */
     protected PageMemory createPageMemory(
         DirectMemoryProvider memProvider,
-        int pageSize,
+        MemoryConfiguration memCfg,
         MemoryPolicyConfiguration memPlcCfg,
         MemoryMetricsImpl memMetrics
     ) {
-        return new PageMemoryNoStoreImpl(log, memProvider, cctx, pageSize, memPlcCfg, memMetrics, false);
+        return new PageMemoryNoStoreImpl(log, memProvider, cctx, memCfg.getPageSize(), memPlcCfg, memMetrics, false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
index ed4cae0..ad12983 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
@@ -79,7 +79,7 @@ public class MemoryMetricsImpl implements MemoryMetricsMXBean {
 
     /** {@inheritDoc} */
     @Override public int getSize() {
-        return (int) (memPlcCfg.getSize() / (1024 * 1024));
+        return (int) (memPlcCfg.getMaxSize() / (1024 * 1024));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
index 88de545..510ffae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/PageAbstractEvictionTracker.java
@@ -85,7 +85,7 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
 
         int pageSize = memCfg.getPageSize();
 
-        long segSize = plcCfg.getSize() / concurrencyLevel;
+        long segSize = plcCfg.getMaxSize() / concurrencyLevel;
 
         if (segSize < 1024 * 1024)
             segSize = 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
index f0ad813..870b4b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/Random2LruPageEvictionTracker.java
@@ -62,7 +62,7 @@ public class Random2LruPageEvictionTracker extends PageAbstractEvictionTracker {
 
         MemoryConfiguration memCfg = sharedCtx.kernalContext().config().getMemoryConfiguration();
 
-        assert plcCfg.getSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
+        assert plcCfg.getMaxSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
 
         log = sharedCtx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
index 8818b1c..169ec7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/evict/RandomLruPageEvictionTracker.java
@@ -62,7 +62,7 @@ public class RandomLruPageEvictionTracker extends PageAbstractEvictionTracker {
 
         MemoryConfiguration memCfg = sharedCtx.kernalContext().config().getMemoryConfiguration();
 
-        assert plcCfg.getSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
+        assert plcCfg.getMaxSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
 
         log = sharedCtx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
index 509aa48..66d7c01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
@@ -58,7 +58,7 @@ public class VisorMemoryPolicyConfiguration extends VisorDataTransferObject {
         assert plc != null;
 
         name = plc.getName();
-        size = plc.getSize();
+        size = plc.getMaxSize();
         swapFilePath = plc.getSwapFilePath();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
index 0fe90cd..2d6839d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
@@ -284,8 +284,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * 1024;
 
-        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), memDir, true,
-            sizes);
+        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), memDir);
 
         return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE, null, new MemoryMetricsImpl(null), true);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/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 df9fbd5..69eda15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
@@ -48,7 +48,7 @@ public class CacheConfigurationLeakTest extends GridCommonAbstractTest {
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
         plc.setName("dfltPlc");
-        plc.setSize(MemoryConfiguration.DFLT_MEMORY_POLICY_SIZE * 10);
+        plc.setMaxSize(MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE * 10);
 
         memCfg.setDefaultMemoryPolicyName("dfltPlc");
         memCfg.setMemoryPolicies(plc);

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
index c0f74d0..27c09c9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
@@ -138,7 +138,7 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
         plc.setName(name);
-        plc.setSize(size);
+        plc.setMaxSize(size);
 
         return plc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
index 20edd4e..6fb4397 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
@@ -79,7 +79,7 @@ public class PageEvictionAbstractTest extends GridCommonAbstractTest {
 
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
-        plc.setSize(SIZE);
+        plc.setMaxSize(SIZE);
         plc.setEmptyPagesPoolSize(EMPTY_PAGES_POOL_SIZE);
         plc.setEvictionThreshold(EVICTION_THRESHOLD);
         plc.setName(DEFAULT_POLICY_NAME);

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 32c6675..bf25349 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -1272,7 +1272,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
                             tree.invoke(x, null, new IgniteTree.InvokeClosure<Long>() {
                                 IgniteTree.OperationType opType;
 
-                                @Override public void call(@Nullable Long row) throws IgniteCheckedException {
+                                @Override public void call(@Nullable Long row) {
                                     opType = PUT;
 
                                     if (row != null)
@@ -1294,7 +1294,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
                             tree.invoke(x, null, new IgniteTree.InvokeClosure<Long>() {
                                 IgniteTree.OperationType opType;
 
-                                @Override public void call(@Nullable Long row) throws IgniteCheckedException {
+                                @Override public void call(@Nullable Long row) {
                                     if (row != null) {
                                         assertEquals(x, row);
                                         opType = REMOVE;
@@ -1685,8 +1685,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             Long row = PageUtils.getLong(pageAddr, offset(idx));
 
             checkNotRemoved(row);
@@ -1705,7 +1704,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
             sizes[i] = 1024 * MB / CPUS;
 
         PageMemory pageMem = new PageMemoryNoStoreImpl(log,
-            new UnsafeMemoryProvider(sizes),
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
             null,
@@ -1754,8 +1753,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
-            throws IgniteCheckedException {
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx) {
             return PageUtils.getLong(pageAddr, offset(idx));
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 1cede9b..95771b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -145,7 +145,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param pageSize Page size.
-     * @throws Exception
+     * @throws Exception If failed.
      */
     protected void checkInsertDeleteMultiThreaded(final int pageSize) throws Exception {
         final FreeList list = createFreeList(pageSize);
@@ -320,7 +320,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
             sizes[i] = 1024 * MB / CPUS;
 
         PageMemory pageMem = new PageMemoryNoStoreImpl(log,
-            new UnsafeMemoryProvider(sizes),
+            new UnsafeMemoryProvider(log),
             null,
             pageSize,
             null,

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/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 b7a8bf3..9f556d3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
@@ -44,7 +44,7 @@ public class IgniteDbDynamicCacheSelfTest extends GridCommonAbstractTest {
         MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
 
         plc.setName("dfltPlc");
-        plc.setSize(200 * 1024 * 1024);
+        plc.setMaxSize(200 * 1024 * 1024);
 
         dbCfg.setDefaultMemoryPolicyName("dfltPlc");
         dbCfg.setMemoryPolicies(plc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
index 61c8ad9..d15938d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
@@ -159,7 +159,7 @@ public class MetadataStorageSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * 1024;
 
-        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath, clean, sizes);
+        DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath);
 
         return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE, null, new MemoryMetricsImpl(null), true);
     }

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/67ba03bc/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
index f34deae..c4ba3c3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
@@ -136,8 +136,10 @@ public class InlineIndexHelperTest extends TestCase {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+        JavaLogger log = new JavaLogger();
+
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
             null,
@@ -183,8 +185,10 @@ public class InlineIndexHelperTest extends TestCase {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+        JavaLogger log = new JavaLogger();
+
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
             null,
@@ -306,8 +310,10 @@ public class InlineIndexHelperTest extends TestCase {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(new JavaLogger(),
-            new UnsafeMemoryProvider(sizes),
+        JavaLogger log = new JavaLogger();
+
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
+            new UnsafeMemoryProvider(log),
             null,
             PAGE_SIZE,
             null,