You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/04/05 14:59:07 UTC

[1/8] ignite git commit: GC pressure

Repository: ignite
Updated Branches:
  refs/heads/ignite-4811-no-start-ver [created] 64959753f


http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
index 06999cb..2586696 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
@@ -62,7 +61,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.util.PageLockLi
  *
  * 7. It is almost always preferable to read or write (especially write) page contents using
  *    static methods on {@link PageHandler}. To just initialize new page use
- *    {@link PageHandler#initPage(PageMemory, Page, PageLockListener, PageIO, IgniteWriteAheadLogManager)}
+ *    {@link PageHandler#initPage(PageMemory, int, long, PageIO, IgniteWriteAheadLogManager, PageLockListener)}
  *    method with needed IO instance.
  */
 public abstract class PageIO {

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
index ce8266a..1c6c8d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
@@ -65,7 +65,7 @@ public class ReuseListImpl extends PagesList implements ReuseList {
 
     /** {@inheritDoc} */
     @Override public void addForRecycle(ReuseBag bag) throws IgniteCheckedException {
-        put(bag, null, 0, 0);
+        put(bag, 0, 0, 0, 0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
index 97b5a04..f660686 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.database.tree.util;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
@@ -34,38 +33,62 @@ import static java.lang.Boolean.TRUE;
  */
 public abstract class PageHandler<X, R> {
     /** */
-    private static final PageHandler<Void, Boolean> NOOP = new PageHandler<Void, Boolean>() {
-        @Override public Boolean run(Page page, PageIO io, long pageAddr, Void arg, int intArg)
-            throws IgniteCheckedException {
-            return TRUE;
+    private static final PageHandler<Void, Boolean> NO_OP = new PageHandler<Void, Boolean>() {
+        @Override
+        public Boolean run(int cacheId, long pageId, long page, long pageAddr, PageIO io, Boolean walPlc,
+            Void arg,
+            int intArg) throws IgniteCheckedException {
+            return Boolean.TRUE;
         }
     };
 
     /**
-     * @param page Page.
-     * @param io IO.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page absolute pointer.
      * @param pageAddr Page address.
+     * @param io IO.
+     * @param walPlc Full page WAL record policy.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract R run(Page page, PageIO io, long pageAddr, X arg, int intArg)
+    public abstract R run(
+        int cacheId,
+        long pageId,
+        long page,
+        long pageAddr,
+        PageIO io,
+        Boolean walPlc,
+        X arg,
+        int intArg)
         throws IgniteCheckedException;
 
     /**
-     * @param page Page.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
      * @return {@code true} If release.
      */
-    public boolean releaseAfterWrite(Page page, X arg, int intArg) {
+    public boolean releaseAfterWrite(
+        int cacheId,
+        long pageId,
+        long page,
+        long pageAddr,
+        X arg,
+        int intArg) {
         return true;
     }
 
     /**
-     * @param page Page.
-     * @param lockLsnr Lock listener.
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param lsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -74,32 +97,40 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static <X, R> R readPage(
-        Page page,
-        PageLockListener lockLsnr,
+        PageMemory pageMem,
+        int cacheId,
+        long pageId,
+        PageLockListener lsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        long pageAddr = readLock(page, lockLsnr);
-
-        if (pageAddr == 0L)
-            return lockFailed;
-
+        long page = pageMem.acquirePage(cacheId, pageId);
         try {
-            PageIO io = PageIO.getPageIO(pageAddr);
-
-            return h.run(page, io, pageAddr, arg, intArg);
+            long pageAddr = readLock(pageMem, cacheId, pageId, page, lsnr);
+
+            if (pageAddr == 0L)
+                return lockFailed;
+            try {
+                PageIO io = PageIO.getPageIO(pageAddr);
+                return h.run(cacheId, pageId, page, pageAddr, io, null, arg, intArg);
+            }
+            finally {
+                readUnlock(pageMem, cacheId, pageId, page, pageAddr, lsnr);
+            }
         }
         finally {
-            readUnlock(page, pageAddr, lockLsnr);
+            pageMem.releasePage(cacheId, pageId, page);
         }
     }
 
     /**
      * @param pageMem Page memory.
-     * @param page Page.
-     * @param lockLsnr Lock listener.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param lsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -107,98 +138,177 @@ public abstract class PageHandler<X, R> {
      * @return Handler result.
      * @throws IgniteCheckedException If failed.
      */
-    public static <X, R> R writePage(
+    public static <X, R> R readPage(
         PageMemory pageMem,
-        Page page,
-        PageLockListener lockLsnr,
+        int cacheId,
+        long pageId,
+        long page,
+        PageLockListener lsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        return writePage(pageMem, page, lockLsnr, h, null, null, arg, intArg, lockFailed);
+        long pageAddr = readLock(pageMem, cacheId, pageId, page, lsnr);
+
+        if (pageAddr == 0L)
+            return lockFailed;
+        try {
+            PageIO io = PageIO.getPageIO(pageAddr);
+            return h.run(cacheId, pageId, page, pageAddr, io, null, arg, intArg);
+        }
+        finally {
+            readUnlock(pageMem, cacheId, pageId, page, pageAddr, lsnr);
+        }
     }
 
     /**
      * @param pageMem Page memory.
-     * @param page Page.
-     * @param lockLsnr Lock listener.
-     * @param init IO for new page initialization or {@code null} if it is an existing page.
-     * @param wal WAL manager.
-     * @throws IgniteCheckedException If failed.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param lsnr Lock listener.
+     * @return Page address.
      */
-    public static void initPage(
+    public static long readLock(
         PageMemory pageMem,
-        Page page,
-        PageLockListener lockLsnr,
-        PageIO init,
-        IgniteWriteAheadLogManager wal
-    ) throws IgniteCheckedException {
-        Boolean res = writePage(pageMem, page, lockLsnr, NOOP, init, wal, null, 0, FALSE);
+        int cacheId,
+        long pageId,
+        long page,
+        PageLockListener lsnr) {
+        boolean notifyLsnr = lsnr != null;
 
-        assert res == TRUE : res; // It must be newly allocated page, can't be recycled.
-    }
+        if(notifyLsnr)
+            lsnr.onBeforeReadLock(cacheId, pageId, page);
 
-    /**
-     * @param page Page.
-     * @param lockLsnr Lock listener.
-     * @return Page address or {@code 0} if failed to lock due to recycling.
-     */
-    public static long readLock(Page page, PageLockListener lockLsnr) {
-        lockLsnr.onBeforeReadLock(page);
-
-        long pageAddr = page.getForReadPointer();
+        long pageAddr = pageMem.readLock(cacheId, pageId, page);
 
-        lockLsnr.onReadLock(page, pageAddr);
+        if(notifyLsnr)
+            lsnr.onReadLock(cacheId, pageId, page, pageAddr);
 
         return pageAddr;
     }
 
     /**
-     * @param page Page.
-     * @param pageAddr Page address.
-     * @param lockLsnr Lock listener.
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address (for-write pointer)
+     * @param lsnr Lock listener.
      */
-    public static void readUnlock(Page page, long pageAddr, PageLockListener lockLsnr) {
-        lockLsnr.onReadUnlock(page, pageAddr);
+    public static void readUnlock(
+        PageMemory pageMem,
+        int cacheId,
+        long pageId,
+        long page,
+        long pageAddr,
+        PageLockListener lsnr) {
+        if(lsnr != null)
+            lsnr.onReadUnlock(cacheId, pageId, page, pageAddr);
 
-        page.releaseRead();
+        pageMem.readUnlock(cacheId, pageId, page);
     }
 
     /**
-     * @param page Page.
-     * @param lockLsnr Lock listener.
-     * @param tryLock Only try to lock without waiting.
-     * @return Page address or {@code 0} if failed to lock due to recycling.
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param init IO for new page initialization.
+     * @param wal Write ahead log.
+     * @param lsnr Lock listener.
+     * @throws IgniteCheckedException If failed.
      */
-    public static long writeLock(Page page, PageLockListener lockLsnr, boolean tryLock) {
-        lockLsnr.onBeforeWriteLock(page);
-
-        long pageAddr = tryLock ? page.tryGetForWritePointer() : page.getForWritePointer();
-
-        lockLsnr.onWriteLock(page, pageAddr);
+    public static void initPage(
+        PageMemory pageMem,
+        int cacheId,
+        long pageId,
+        PageIO init,
+        IgniteWriteAheadLogManager wal,
+        PageLockListener lsnr
+    ) throws IgniteCheckedException {
+        Boolean res = writePage(pageMem, cacheId, pageId, lsnr, PageHandler.NO_OP, init, wal, null, null, 0, FALSE);
 
-        return pageAddr;
+        assert res != FALSE;
     }
 
     /**
-     * @param page Page.
-     * @param buf Page buffer.
-     * @param lockLsnr Lock listener.
-     * @param dirty Page is dirty.
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param lsnr Lock listener.
+     * @param h Handler.
+     * @param init IO for new page initialization or {@code null} if it is an existing page.
+     * @param wal Write ahead log.
+     * @param walPlc Full page WAL record policy.
+     * @param arg Argument.
+     * @param intArg Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @return Handler result.
+     * @throws IgniteCheckedException If failed.
      */
-    public static void writeUnlock(Page page, long buf, PageLockListener lockLsnr, boolean dirty) {
-        lockLsnr.onWriteUnlock(page, buf);
+    public static <X, R> R writePage(
+        PageMemory pageMem,
+        int cacheId,
+        long pageId,
+        PageLockListener lsnr,
+        PageHandler<X, R> h,
+        PageIO init,
+        IgniteWriteAheadLogManager wal,
+        Boolean walPlc,
+        X arg,
+        int intArg,
+        R lockFailed
+    ) throws IgniteCheckedException {
+        boolean releaseAfterWrite = true;
+        long page = pageMem.acquirePage(cacheId, pageId);
+        try {
+            long pageAddr = writeLock(pageMem, cacheId, pageId, page, lsnr, false);
 
-        page.releaseWrite(dirty);
+            if (pageAddr == 0L)
+                return lockFailed;
+
+            boolean ok = false;
+
+            try {
+                if (init != null) {
+                    // It is a new page and we have to initialize it.
+                    doInitPage(pageMem, cacheId, pageId, page, pageAddr, init, wal);
+                    walPlc = FALSE;
+                }
+                else {
+                    init = PageIO.getPageIO(pageAddr);
+                }
+
+                R res = h.run(cacheId, pageId, page, pageAddr, init, walPlc, arg, intArg);
+
+                ok = true;
+
+                return res;
+            }
+            finally {
+                assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
+
+                if (releaseAfterWrite = h.releaseAfterWrite(cacheId, pageId, page, pageAddr, arg, intArg))
+                    writeUnlock(pageMem, cacheId, pageId, page, pageAddr, lsnr, walPlc, ok);
+            }
+        }
+        finally {
+            if(releaseAfterWrite)
+                pageMem.releasePage(cacheId, pageId, page);
+        }
     }
 
     /**
-     * @param page Page.
-     * @param lockLsnr Lock listener.
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param lsnr Lock listener.
      * @param h Handler.
      * @param init IO for new page initialization or {@code null} if it is an existing page.
-     * @param wal WAL manager.
+     * @param wal Write ahead log.
+     * @param walPlc Full page WAL record policy.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
      * @param lockFailed Result in case of lock failure due to page recycling.
@@ -207,47 +317,108 @@ public abstract class PageHandler<X, R> {
      */
     public static <X, R> R writePage(
         PageMemory pageMem,
-        Page page,
-        PageLockListener lockLsnr,
+        int cacheId,
+        long pageId,
+        long page,
+        PageLockListener lsnr,
         PageHandler<X, R> h,
         PageIO init,
         IgniteWriteAheadLogManager wal,
+        Boolean walPlc,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        long pageAddr = writeLock(page, lockLsnr, false);
+        long pageAddr = writeLock(pageMem, cacheId, pageId, page, lsnr, false);
 
         if (pageAddr == 0L)
             return lockFailed;
 
-        R res;
-
         boolean ok = false;
 
         try {
-            if (init != null) // It is a new page and we have to initialize it.
-                doInitPage(pageMem, page, pageAddr, init, wal);
-            else
+            if (init != null) {
+                // It is a new page and we have to initialize it.
+                doInitPage(pageMem, cacheId, pageId, page, pageAddr, init, wal);
+                walPlc = FALSE;
+            }
+            else {
                 init = PageIO.getPageIO(pageAddr);
+            }
 
-            res = h.run(page, init, pageAddr, arg, intArg);
+            R res = h.run(cacheId, pageId, page, pageAddr, init, walPlc, arg, intArg);
 
             ok = true;
+
+            return res;
         }
         finally {
             assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
-            if (h.releaseAfterWrite(page, arg, intArg))
-                writeUnlock(page, pageAddr, lockLsnr, ok);
+            if (h.releaseAfterWrite(cacheId, pageId, page, pageAddr, arg, intArg))
+                writeUnlock(pageMem, cacheId, pageId, page, pageAddr, lsnr, walPlc, ok);
         }
+    }
 
-        return res;
+    /**
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address.
+     * @param lsnr Lock listener.
+     * @param walPlc Full page WAL record policy.
+     * @param dirty Page is dirty.
+     */
+    public static void writeUnlock(
+        PageMemory pageMem,
+        int cacheId,
+        long pageId,
+        long page,
+        long pageAddr,
+        PageLockListener lsnr,
+        Boolean walPlc,
+        boolean dirty) {
+        if(lsnr != null)
+            lsnr.onWriteUnlock(cacheId, pageId, page, pageAddr);
+
+        pageMem.writeUnlock(cacheId, pageId, page, walPlc, dirty);
+    }
+
+    /**
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param lsnr Lock listener.
+     * @param tryLock Only try to lock without waiting.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
+     */
+    public static long writeLock(
+        PageMemory pageMem,
+        int cacheId,
+        long pageId,
+        long page,
+        PageLockListener lsnr,
+        boolean tryLock) {
+        boolean notifyLsnr = lsnr != null;
+
+        if(notifyLsnr)
+            lsnr.onBeforeWriteLock(cacheId, pageId, page);
+
+        long pageAddr = tryLock ? pageMem.tryWriteLock(cacheId, pageId, page) : pageMem.writeLock(cacheId, pageId, page);
+
+        if(notifyLsnr)
+            lsnr.onWriteLock(cacheId, pageId, page, pageAddr);
+
+        return pageAddr;
     }
 
     /**
      * @param pageMem Page memory.
-     * @param page Page.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      * @param pageAddr Page address.
      * @param init Initial IO.
      * @param wal Write ahead log.
@@ -255,35 +426,43 @@ public abstract class PageHandler<X, R> {
      */
     private static void doInitPage(
         PageMemory pageMem,
-        Page page,
+        int cacheId,
+        long pageId,
+        long page,
         long pageAddr,
         PageIO init,
-        IgniteWriteAheadLogManager wal
-    ) throws IgniteCheckedException {
-        assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
+        IgniteWriteAheadLogManager wal) throws IgniteCheckedException {
 
-        long pageId = page.id();
+        assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
         init.initNewPage(pageAddr, pageId, pageMem.pageSize());
 
         // Here we should never write full page, because it is known to be new.
-        page.fullPageWalRecordPolicy(FALSE);
-
-        if (isWalDeltaRecordNeeded(wal, page))
-            wal.log(new InitNewPageRecord(page.fullId().cacheId(), page.id(),
+        if (isWalDeltaRecordNeeded(pageMem, cacheId, pageId, page, wal, FALSE))
+            wal.log(new InitNewPageRecord(cacheId, pageId,
                 init.getType(), init.getVersion(), pageId));
     }
 
     /**
+     * @param pageMem Page memory.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      * @param wal Write ahead log.
-     * @param page Page.
+     * @param walPlc Full page WAL record policy.
      * @return {@code true} If we need to make a delta WAL record for the change in this page.
      */
-    public static boolean isWalDeltaRecordNeeded(IgniteWriteAheadLogManager wal, Page page) {
+    public static boolean isWalDeltaRecordNeeded(
+        PageMemory pageMem,
+        int cacheId,
+        long pageId,
+        long page,
+        IgniteWriteAheadLogManager wal,
+        Boolean walPlc) {
         // If the page is clean, then it is either newly allocated or just after checkpoint.
         // In both cases we have to write full page contents to WAL.
-        return wal != null && !wal.isAlwaysWriteFullPages() && page.fullPageWalRecordPolicy() != TRUE &&
-            (page.fullPageWalRecordPolicy() == FALSE || page.isDirty());
+        return wal != null && !wal.isAlwaysWriteFullPages() && walPlc != TRUE &&
+            (walPlc == FALSE || pageMem.isDirty(cacheId, pageId, page));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
index 07cdd43..904d217 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
@@ -17,43 +17,53 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.util;
 
-import org.apache.ignite.internal.pagemem.Page;
-
 /**
  * Page lock listener.
  */
 public interface PageLockListener {
     /**
-     * @param page Page.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      */
-    public void onBeforeWriteLock(Page page);
+    public void onBeforeWriteLock(int cacheId, long pageId, long page);
 
     /**
-     * @param page Page.
-     * @param pageAddr Page address or {@code 0} if attempt to lock failed.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address.
      */
-    public void onWriteLock(Page page, long pageAddr);
+    public void onWriteLock(int cacheId, long pageId, long page, long pageAddr);
 
     /**
-     * @param page Page.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      * @param pageAddr Page address.
      */
-    public void onWriteUnlock(Page page, long pageAddr);
+    public void onWriteUnlock(int cacheId, long pageId, long page, long pageAddr);
 
     /**
-     * @param page Page.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      */
-    public void onBeforeReadLock(Page page);
+    public void onBeforeReadLock(int cacheId, long pageId, long page);
 
     /**
-     * @param page Page.
-     * @param pageAddr Page address or {@code 0} if attempt to lock failed.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address.
      */
-    public void onReadLock(Page page, long pageAddr);
+    public void onReadLock(int cacheId, long pageId, long page, long pageAddr);
 
     /**
-     * @param page Page.
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      * @param pageAddr Page address.
      */
-    public void onReadUnlock(Page page, long pageAddr);
+    public void onReadUnlock(int cacheId, long pageId, long page, long pageAddr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index bc1c584..a63cf1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -2882,7 +2882,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param locked Locked entries.
      * @param topVer Topology version.
      */
-    private void unlockEntries(Collection<GridDhtCacheEntry> locked, AffinityTopologyVersion topVer) {
+    private void unlockEntries(List<GridDhtCacheEntry> locked, AffinityTopologyVersion topVer) {
         // Process deleted entries before locks release.
         assert ctx.deferredDelete() : this;
 
@@ -2890,8 +2890,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         // Enqueue entries while holding locks.
         Collection<KeyCacheObject> skip = null;
 
+        int size = locked.size();
+
         try {
-            for (GridCacheMapEntry entry : locked) {
+            for (int i = 0; i < size; i++) {
+                GridCacheMapEntry entry = locked.get(i);
                 if (entry != null && entry.deleted()) {
                     if (skip == null)
                         skip = U.newHashSet(locked.size());
@@ -2904,27 +2907,30 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             // At least RuntimeException can be thrown by the code above when GridCacheContext is cleaned and there is
             // an attempt to use cleaned resources.
             // That's why releasing locks in the finally block..
-            for (GridCacheMapEntry entry : locked) {
+            for (int i = 0; i < size; i++) {
+                GridCacheMapEntry entry = locked.get(i);
                 if (entry != null)
                     GridUnsafe.monitorExit(entry);
             }
         }
 
         // Try evict partitions.
-        for (GridDhtCacheEntry entry : locked) {
+        for (int i = 0; i < size; i++) {
+            GridDhtCacheEntry entry = locked.get(i);
             if (entry != null)
                 entry.onUnlock();
         }
 
         ctx.shared().database().checkpointReadUnlock();
 
-        if (skip != null && skip.size() == locked.size())
+        if (skip != null && skip.size() == size)
             // Optimization.
             return;
 
         // Must touch all entries since update may have deleted entries.
         // Eviction manager will remove empty entries.
-        for (GridCacheMapEntry entry : locked) {
+        for (int i = 0; i < size; i++) {
+            GridCacheMapEntry entry = locked.get(i);
             if (entry != null && (skip == null || !skip.contains(entry.key())))
                 ctx.evicts().touch(entry, topVer);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
index 5a8904f..fe5f562 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
@@ -38,6 +38,9 @@ import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED;
  * caches.
  */
 public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
+    /** */
+    public static final GridCacheVersion EVICT_VER = new GridCacheVersion(Integer.MAX_VALUE, 0, 0, 0);
+
     /** Timestamp used as base time for cache topology version (January 1, 2014). */
     public static final long TOP_VER_BASE_TIME = 1388520000000L;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
index 1e675b6..36f2902 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshot.java
@@ -91,7 +91,7 @@ public class GridClockDeltaSnapshot {
      * @return Map of collected deltas.
      */
     public Map<UUID, Long> deltas() {
-        return Collections.unmodifiableMap(deltas);
+        return deltas;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/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 ec62c9b..8362ace 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
@@ -26,7 +26,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -59,30 +58,31 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             FullPageId fullId1 = allocatePage(mem);
             FullPageId fullId2 = allocatePage(mem);
 
-            Page page1 = mem.page(fullId1.cacheId(), fullId1.pageId());
+            long page1 = mem.acquirePage(fullId1.cacheId(), fullId1.pageId());
 
             try {
-                Page page2 = mem.page(fullId2.cacheId(), fullId2.pageId());
+                long page2 = mem.acquirePage(fullId2.cacheId(), fullId2.pageId());
 
-                info("Allocated pages [page1=" + page1 + ", page2=" + page2 + ']');
+                info("Allocated pages [page1Id=" + fullId1.pageId() + ", page1=" + page1 +
+                    ", page2Id=" + fullId2.pageId() + ", page2=" + page2 + ']');
 
                 try {
-                    writePage(page1, 1);
-                    writePage(page2, 2);
+                    writePage(mem, fullId1.pageId(), page1, 1);
+                    writePage(mem, fullId2.pageId(), page2, 2);
 
-                    readPage(page1, 1);
-                    readPage(page2, 2);
+                    readPage(mem, fullId1.pageId(), page1, 1);
+                    readPage(mem, fullId2.pageId(), page2, 2);
 
                     // Check read after read.
-                    readPage(page1, 1);
-                    readPage(page2, 2);
+                    readPage(mem, fullId1.pageId(), page1, 1);
+                    readPage(mem, fullId2.pageId(), page2, 2);
                 }
                 finally {
-                    mem.releasePage(page2);
+                    mem.releasePage(fullId2.cacheId(), fullId2.pageId(), page2);
                 }
             }
             finally {
-                mem.releasePage(page1);
+                mem.releasePage(fullId1.cacheId(), fullId1.pageId(), page1);
             }
         }
         finally {
@@ -108,32 +108,32 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
 
                 pages.add(fullId);
 
-                Page page = mem.page(fullId.cacheId(), fullId.pageId());
+                long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
 
                 try {
                     if (i % 64 == 0)
-                        info("Writing page [idx=" + i + ", page=" + page + ']');
+                        info("Writing page [idx=" + i + ", pageId=" + fullId.pageId() + ", page=" + page + ']');
 
-                    writePage(page, i + 1);
+                    writePage(mem, fullId.pageId(), page, i + 1);
                 }
                 finally {
-                    mem.releasePage(page);
+                    mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
                 }
             }
 
             for (int i = 0; i < pagesCnt; i++) {
                 FullPageId fullId = pages.get(i);
 
-                Page page = mem.page(fullId.cacheId(), fullId.pageId());
+                long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
 
                 try {
                     if (i % 64 == 0)
-                        info("Reading page [idx=" + i + ", page=" + page + ']');
+                        info("Reading page [idx=" + i + ", pageId=" + fullId.pageId() + ", page=" + page + ']');
 
-                    readPage(page, i + 1);
+                    readPage(mem, fullId.pageId(), page, i + 1);
                 }
                 finally {
-                    mem.releasePage(page);
+                    mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
                 }
             }
         }
@@ -188,70 +188,82 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
 
             // Check that initial pages are accessible.
             for (FullPageId id : old) {
-                try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    long buf = page.getForWritePointer();
+                long pageApsPtr = mem.acquirePage(id.cacheId(), id.pageId());
+                try {
+                    long pageAddr = mem.writeLock(id.cacheId(), id.pageId(), pageApsPtr);
 
-                    assertNotNull(buf);
+                    assertNotNull(pageAddr);
 
                     try {
                         long updId = PageIdUtils.rotatePageId(id.pageId());
 
-                        PageIO.setPageId(buf, updId);
+                        PageIO.setPageId(pageAddr, updId);
 
                         updated.add(new FullPageId(updId, id.cacheId()));
                     }
                     finally {
-                        page.releaseWrite(true);
+                        mem.writeUnlock(id.cacheId(), id.pageId(), pageApsPtr, null, true);
                     }
                 }
+                finally {
+                    mem.releasePage(id.cacheId(), id.pageId(), pageApsPtr);
+                }
             }
 
             // Check that updated pages are inaccessible using old IDs.
             for (FullPageId id : old) {
-                try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    long pageAddr = page.getForWritePointer();
+                long pageApsPtr = mem.acquirePage(id.cacheId(), id.pageId());
+                try {
+                    long pageAddr = mem.writeLock(id.cacheId(), id.pageId(), pageApsPtr);
 
                     if (pageAddr != 0L) {
-                        page.releaseWrite(false);
+                        mem.writeUnlock(id.cacheId(), id.pageId(), pageApsPtr, null, false);
 
                         fail("Was able to acquire page write lock.");
                     }
 
-                    pageAddr = page.getForReadPointer();
+                    mem.readLock(id.cacheId(), id.pageId(), pageApsPtr);
 
                     if (pageAddr != 0) {
-                        page.releaseRead();
+                        mem.readUnlock(id.cacheId(), id.pageId(), pageApsPtr);
 
                         fail("Was able to acquire page read lock.");
                     }
                 }
+                finally {
+                    mem.releasePage(id.cacheId(), id.pageId(), pageApsPtr);
+                }
             }
 
             // Check that updated pages are accessible using new IDs.
             for (FullPageId id : updated) {
-                try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    long buf = page.getForWritePointer();
+                long pageApsPtr = mem.acquirePage(id.cacheId(), id.pageId());
+                try {
+                    long pageAddr = mem.writeLock(id.cacheId(), id.pageId(), pageApsPtr);
 
-                    assertNotNull(buf);
+                    assertNotSame(0L, pageAddr);
 
                     try {
-                        assertEquals(id.pageId(), PageIO.getPageId(buf));
+                        assertEquals(id.pageId(), PageIO.getPageId(pageAddr));
                     }
                     finally {
-                        page.releaseWrite(false);
+                        mem.writeUnlock(id.cacheId(), id.pageId(), pageApsPtr, null, false);
                     }
 
-                    buf = page.getForReadPointer();
+                    pageAddr = mem.readLock(id.cacheId(), id.pageId(), pageApsPtr);
 
-                    assertNotNull(buf);
+                    assertNotSame(0L, pageAddr);
 
                     try {
-                        assertEquals(id.pageId(), PageIO.getPageId(buf));
+                        assertEquals(id.pageId(), PageIO.getPageId(pageAddr));
                     }
                     finally {
-                        page.releaseRead();
+                        mem.readUnlock(id.cacheId(), id.pageId(), pageApsPtr);
                     }
                 }
+                finally {
+                    mem.releasePage(id.cacheId(), id.pageId(), pageApsPtr);
+                }
             }
         }
         finally {
@@ -261,6 +273,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
 
     /**
      * @return Page memory implementation.
+     * @throws Exception If failed.
      */
     protected PageMemory memory() throws Exception {
         File memDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "pagemem", false);
@@ -277,31 +290,35 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @param page Page to write.
+     * @param mem Page memory.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      * @param val Value to write.
      */
-    private void writePage(Page page, int val) {
-        long bytes = page.getForWritePointer();
+    private void writePage(PageMemory mem, long pageId, long page, int val) {
+        long pageAddr = mem.writeLock(-1, pageId, page);
 
         try {
-            PageIO.setPageId(bytes, page.id());
+            PageIO.setPageId(pageAddr, pageId);
 
             for (int i = PageIO.COMMON_HEADER_END; i < PAGE_SIZE; i++)
-                PageUtils.putByte(bytes, i, (byte)val);
+                PageUtils.putByte(pageAddr, i, (byte)val);
         }
         finally {
-            page.releaseWrite(true);
+            mem.writeUnlock(-1, pageId, page, null, true);
         }
     }
 
     /**
-     * @param page Page to read.
+     * @param mem Page memory.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      * @param expVal Expected value.
      */
-    private void readPage(Page page, int expVal) {
+    private void readPage(PageMemory mem, long pageId, long page, int expVal) {
         expVal &= 0xFF;
 
-        long pageAddr = page.getForReadPointer();
+        long pageAddr = mem.readLock(-1, pageId, page);
 
         assert(pageAddr != 0);
 
@@ -313,13 +330,14 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             }
         }
         finally {
-            page.releaseRead();
+            mem.readUnlock(-1, pageId, page);
         }
     }
 
     /**
      * @param mem Memory.
      * @return Page.
+     * @throws IgniteCheckedException If failed.
      */
     public static FullPageId allocatePage(PageIdAllocator mem) throws IgniteCheckedException {
         return new FullPageId(mem.allocatePage(-1, 1, PageIdAllocator.FLAG_DATA), -1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
index 5f38bb0..e5375f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.database;
 import java.util.HashSet;
 import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -85,44 +84,44 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onBeforeReadLock(Page page) {
+        @Override public void onBeforeReadLock(int cacheId, long pageId, long page) {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, long pageAddr) {
-            checkPageId(page, pageAddr);
+        @Override public void onReadLock(int cacheId, long pageId, long page, long pageAddr) {
+            checkPageId(pageId, pageAddr);
 
-            assertTrue(readLocks.get().add(page.id()));
+            assertTrue(readLocks.get().add(pageId));
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, long pageAddr) {
-            checkPageId(page, pageAddr);
+        @Override public void onReadUnlock(int cacheId, long pageId, long page, long pageAddr) {
+            checkPageId(pageId, pageAddr);
 
-            assertTrue(readLocks.get().remove(page.id()));
+            assertTrue(readLocks.get().remove(pageId));
         }
 
         /** {@inheritDoc} */
-        @Override public void onBeforeWriteLock(Page page) {
+        @Override public void onBeforeWriteLock(int cacheId, long pageId, long page) {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, long pageAddr) {
+        @Override public void onWriteLock(int cacheId, long pageId, long page, long pageAddr) {
             if (pageAddr == 0L)
                 return; // Failed to lock.
 
-            checkPageId(page, pageAddr);
+            checkPageId(pageId, pageAddr);
 
-            assertTrue(writeLocks.get().add(page.id()));
+            assertTrue(writeLocks.get().add(pageId));
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, long pageAddr) {
-            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(pageAddr)));
+        @Override public void onWriteUnlock(int cacheId, long pageId, long page, long pageAddr) {
+            assertEquals(effectivePageId(pageId), effectivePageId(PageIO.getPageId(pageAddr)));
 
-            assertTrue(writeLocks.get().remove(page.id()));
+            assertTrue(writeLocks.get().remove(pageId));
         }
 
         static boolean checkNoLocks() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/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 9a90c1d..4c6fe4c 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
@@ -34,7 +34,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
@@ -152,27 +151,30 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     @Override protected void afterTest() throws Exception {
         rnd = null;
 
-        if (reuseList != null) {
-            long size = reuseList.recycledPagesCount();
-
-            assertTrue("Reuse size: " + size, size < 7000);
-        }
+        try {
+            if (reuseList != null) {
+                long size = reuseList.recycledPagesCount();
 
-        for (int i = 0; i < 10; i++) {
-            if (acquiredPages() != 0) {
-                System.out.println("!!!");
-                U.sleep(10);
+                assertTrue("Reuse size: " + size, size < 7000);
             }
-        }
 
-        assertEquals(0, acquiredPages());
+            for (int i = 0; i < 10; i++) {
+                if (acquiredPages() != 0) {
+                    System.out.println("!!!");
+                    U.sleep(10);
+                }
+            }
 
-        pageMem.stop();
+            assertEquals(0, acquiredPages());
+        }
+        finally {
+            pageMem.stop();
 
-        MAX_PER_PAGE = 0;
-        PUT_INC = 1;
-        RMV_INC = -1;
-        CNT = 10;
+            MAX_PER_PAGE = 0;
+            PUT_INC = 1;
+            RMV_INC = -1;
+            CNT = 10;
+        }
     }
 
     /**
@@ -1381,15 +1383,15 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @param page Page.
+     * @param pageId Page ID.
      * @param pageAddr Page address.
      */
-    public static void checkPageId(Page page, long pageAddr) {
-        long pageId = PageIO.getPageId(pageAddr);
+    public static void checkPageId(long pageId, long pageAddr) {
+        long actual = PageIO.getPageId(pageAddr);
 
         // Page ID must be 0L for newly allocated page, for reused page effective ID must remain the same.
-        if (pageId != 0L && page.id() != pageId)
-            throw new IllegalStateException("Page ID: " + U.hexLong(pageId));
+        if (actual != 0L && pageId != actual)
+            throw new IllegalStateException("Page ID: " + U.hexLong(actual));
     }
 
     /**
@@ -1493,74 +1495,74 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onBeforeReadLock(Page page) {
+        @Override public void onBeforeReadLock(int cacheId, long pageId, long page) {
 //            X.println("  onBeforeReadLock: " + U.hexLong(page.id()));
 //
 //            U.dumpStack();
 
-            assertNull(beforeReadLock.put(threadId(), page.id()));
+            assertNull(beforeReadLock.put(threadId(), pageId));
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, long pageAddr) {
+        @Override public void onReadLock(int cacheId, long pageId, long page, long pageAddr) {
 //            X.println("  onReadLock: " + U.hexLong(page.id()));
 
             if (pageAddr != 0L) {
-                long pageId = PageIO.getPageId(pageAddr);
+                long actual = PageIO.getPageId(pageAddr);
 
-                checkPageId(page, pageAddr);
+                checkPageId(pageId, pageAddr);
 
-                assertNull(locks(true).put(page.id(), pageId));
+                assertNull(locks(true).put(pageId, actual));
             }
 
-            assertEquals(Long.valueOf(page.id()), beforeReadLock.remove(threadId()));
+            assertEquals(Long.valueOf(pageId), beforeReadLock.remove(threadId()));
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, long pageAddr) {
+        @Override public void onReadUnlock(int cacheId, long pageId, long page, long pageAddr) {
 //            X.println("  onReadUnlock: " + U.hexLong(page.id()));
 
-            checkPageId(page, pageAddr);
+            checkPageId(pageId, pageAddr);
 
-            long pageId = PageIO.getPageId(pageAddr);
+            long actual = PageIO.getPageId(pageAddr);
 
-            assertEquals(Long.valueOf(pageId), locks(true).remove(page.id()));
+            assertEquals(Long.valueOf(actual), locks(true).remove(pageId));
         }
 
         /** {@inheritDoc} */
-        @Override public void onBeforeWriteLock(Page page) {
+        @Override public void onBeforeWriteLock(int cacheId, long pageId, long page) {
 //            X.println("  onBeforeWriteLock: " + U.hexLong(page.id()));
 
-            assertNull(beforeWriteLock.put(threadId(), page.id()));
+            assertNull(beforeWriteLock.put(threadId(), pageId));
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, long pageAddr) {
+        @Override public void onWriteLock(int cacheId, long pageId, long page, long pageAddr) {
 //            X.println("  onWriteLock: " + U.hexLong(page.id()));
 //
 //            U.dumpStack();
 
             if (pageAddr != 0L) {
-                checkPageId(page, pageAddr);
+                checkPageId(pageId, pageAddr);
 
-                long pageId = PageIO.getPageId(pageAddr);
+                long actual = PageIO.getPageId(pageAddr);
 
-                if (pageId == 0L)
-                    pageId = page.id(); // It is a newly allocated page.
+                if (actual == 0L)
+                    actual = pageId; // It is a newly allocated page.
 
-                assertNull(locks(false).put(page.id(), pageId));
+                assertNull(locks(false).put(pageId, actual));
             }
 
-            assertEquals(Long.valueOf(page.id()), beforeWriteLock.remove(threadId()));
+            assertEquals(Long.valueOf(pageId), beforeWriteLock.remove(threadId()));
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, long pageAddr) {
+        @Override public void onWriteUnlock(int cacheId, long pageId, long page, long pageAddr) {
 //            X.println("  onWriteUnlock: " + U.hexLong(page.id()));
 
-            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(pageAddr)));
+            assertEquals(effectivePageId(pageId), effectivePageId(PageIO.getPageId(pageAddr)));
 
-            assertEquals(Long.valueOf(page.id()), locks(false).remove(page.id()));
+            assertEquals(Long.valueOf(pageId), locks(false).remove(pageId));
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
index 842f035..d7f5b29 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
@@ -21,7 +21,6 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
@@ -136,10 +135,11 @@ public abstract class H2Tree extends BPlusTree<SearchRow, GridH2Row> {
      * @throws IgniteCheckedException If failed.
      */
     private int getMetaInlineSize() throws IgniteCheckedException {
-        try (Page meta = page(metaPageId)) {
-            long pageAddr = readLock(meta); // Meta can't be removed.
+        final long metaPage = acquirePage(metaPageId);
+        try {
+            long pageAddr = readLock(metaPageId, metaPage); // Meta can't be removed.
 
-            assert pageAddr != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert pageAddr != 0 : "Failed to read lock meta page [metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
@@ -148,9 +148,12 @@ public abstract class H2Tree extends BPlusTree<SearchRow, GridH2Row> {
                 return io.getInlineSize(pageAddr);
             }
             finally {
-                readUnlock(meta, pageAddr);
+                readUnlock(metaPageId, metaPage, pageAddr);
             }
         }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/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 bcbca1a..2e6ec34 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
@@ -25,8 +25,6 @@ import java.util.UUID;
 import junit.framework.TestCase;
 import org.apache.commons.io.Charsets;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
-import org.apache.ignite.internal.pagemem.FullPageId;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
@@ -144,12 +142,14 @@ public class InlineIndexHelperTest extends TestCase {
             false);
 
         pageMem.start();
-        Page page = null;
+
+        long pageId = 0L;
+        long page = 0L;
 
         try {
-            FullPageId fullId = new FullPageId(pageMem.allocatePage(CACHE_ID, 1, PageIdAllocator.FLAG_DATA), CACHE_ID);
-            page = pageMem.page(fullId.cacheId(), fullId.pageId());
-            long pageAddr = page.getForReadPointer();
+            pageId = pageMem.allocatePage(CACHE_ID, 1, PageIdAllocator.FLAG_DATA);
+            page = pageMem.acquirePage(CACHE_ID, pageId);
+            long pageAddr = pageMem.readLock(CACHE_ID, pageId, page);
 
             int off = 0;
 
@@ -167,8 +167,8 @@ public class InlineIndexHelperTest extends TestCase {
             assertEquals("aaa", ih.get(pageAddr, off, 3 + 5).getString());
         }
         finally {
-            if (page != null)
-                pageMem.releasePage(page);
+            if (page != 0L)
+                pageMem.releasePage(CACHE_ID, pageId, page);
             pageMem.stop();
         }
     }
@@ -187,12 +187,14 @@ public class InlineIndexHelperTest extends TestCase {
             false);
 
         pageMem.start();
-        Page page = null;
+
+        long pageId = 0L;
+        long page = 0L;
 
         try {
-            FullPageId fullId = new FullPageId(pageMem.allocatePage(CACHE_ID, 1, PageIdAllocator.FLAG_DATA), CACHE_ID);
-            page = pageMem.page(fullId.cacheId(), fullId.pageId());
-            long pageAddr = page.getForReadPointer();
+            pageId = pageMem.allocatePage(CACHE_ID, 1, PageIdAllocator.FLAG_DATA);
+            page = pageMem.acquirePage(CACHE_ID, pageId);
+            long pageAddr = pageMem.readLock(CACHE_ID, pageId, page);
 
             int off = 0;
 
@@ -211,8 +213,8 @@ public class InlineIndexHelperTest extends TestCase {
             assertTrue(Arrays.equals(new byte[] {1, 2, 3, 4, 5}, ih.get(pageAddr, off, 3 + 5).getBytes()));
         }
         finally {
-            if (page != null)
-                pageMem.releasePage(page);
+            if (page != 0L)
+                pageMem.releasePage(CACHE_ID, pageId, page);
             pageMem.stop();
         }
     }
@@ -306,12 +308,14 @@ public class InlineIndexHelperTest extends TestCase {
             false);
 
         pageMem.start();
-        Page page = null;
+
+        long pageId = 0L;
+        long page = 0L;
 
         try {
-            FullPageId fullId = new FullPageId(pageMem.allocatePage(CACHE_ID, 1, PageIdAllocator.FLAG_DATA), CACHE_ID);
-            page = pageMem.page(fullId.cacheId(), fullId.pageId());
-            long pageAddr = page.getForReadPointer();
+            pageId = pageMem.allocatePage(CACHE_ID, 1, PageIdAllocator.FLAG_DATA);
+            page = pageMem.acquirePage(CACHE_ID, pageId);
+            long pageAddr = pageMem.readLock(CACHE_ID, pageId, page);
 
             int off = 0;
             int max = 255;
@@ -329,8 +333,8 @@ public class InlineIndexHelperTest extends TestCase {
             assertEquals(v2.getObject(), v22.getObject());
         }
         finally {
-            if (page != null)
-                pageMem.releasePage(page);
+            if (page != 0L)
+                pageMem.releasePage(CACHE_ID, pageId, page);
             pageMem.stop();
         }
     }


[6/8] ignite git commit: Merge branch 'ignite-3477-master' into ignite-4811

Posted by sb...@apache.org.
Merge branch 'ignite-3477-master' into ignite-4811


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

Branch: refs/heads/ignite-4811-no-start-ver
Commit: e34222b3b88fb85a3dabdcf87763d55235554c4f
Parents: 682f3be 6107992
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Thu Mar 30 11:01:08 2017 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Thu Mar 30 11:37:21 2017 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     |   3 +-
 .../configuration/CacheConfiguration.java       |  25 ++
 .../configuration/MemoryConfiguration.java      | 104 ++++--
 .../MemoryPolicyConfiguration.java              |  80 +++++
 .../apache/ignite/internal/IgniteKernal.java    |  50 ++-
 .../processors/cache/GridCacheContext.java      |  43 +++
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  35 +-
 .../cache/database/CacheDataRowAdapter.java     |   2 +-
 .../IgniteCacheDatabaseSharedManager.java       | 354 ++++++++++++++++---
 .../processors/cache/database/MemoryPolicy.java |  53 +++
 .../processors/cache/database/RowStore.java     |   2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   5 +
 .../cache/distributed/near/GridNearTxLocal.java |   1 -
 .../cache/query/GridCacheQueryManager.java      |   3 +-
 .../cache/transactions/IgniteTxManager.java     |   2 +
 .../visor/node/VisorGridConfiguration.java      |   2 +-
 .../visor/node/VisorMemoryConfiguration.java    |  55 ++-
 .../node/VisorMemoryPolicyConfiguration.java    |  72 ++++
 .../resources/META-INF/classnames.properties    |   1 +
 .../ignite/cache/LargeEntryUpdateTest.java      |   1 -
 .../binary/BinaryObjectExceptionSelfTest.java   | 209 -----------
 .../cache/CacheConfigurationLeakTest.java       |   7 +-
 .../CacheMemoryPolicyConfigurationTest.java     | 164 +++++++++
 .../GridCacheAbstractFailoverSelfTest.java      |   4 +
 .../cache/MemoryPolicyConfigValidationTest.java | 241 +++++++++++++
 .../database/IgniteDbAbstractTest.java          |   2 -
 .../database/IgniteDbDynamicCacheSelfTest.java  |   9 +-
 .../IgniteDbMemoryLeakAbstractTest.java         |  10 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   3 +
 .../IgniteBinaryObjectsTestSuite.java           |   2 -
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 .../query/h2/database/H2TreeIndex.java          |  23 +-
 .../processors/query/h2/opt/GridH2Table.java    |  16 +-
 .../query/h2/opt/GridH2TreeIndex.java           |  16 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  13 +-
 .../IgniteCachePartitionedQuerySelfTest.java    |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 196 +++++-----
 .../query/h2/opt/GridH2TableSelfTest.java       |   3 +-
 .../query/h2/sql/GridQueryParsingTest.java      | 112 +++---
 .../cpp/core-test/config/cache-query-32.xml     |  50 +++
 .../config/cache-query-continuous-32.xml        |  46 +++
 .../config/cache-query-continuous-default.xml   |  87 +++++
 .../core-test/config/cache-query-continuous.xml |  61 +---
 .../core-test/config/cache-query-default.xml    | 131 +++++++
 .../cpp/core-test/config/cache-query.xml        | 114 +-----
 .../cpp/core-test/config/cache-test-32.xml      |  50 +++
 .../cpp/core-test/config/cache-test-default.xml | 135 +++++++
 .../cpp/core-test/config/cache-test.xml         | 105 +-----
 .../cpp/core-test/project/vs/core-test.vcxproj  |   7 +
 .../project/vs/core-test.vcxproj.filters        |  21 ++
 .../cpp/core-test/src/cache_query_test.cpp      |   4 +
 .../platforms/cpp/core-test/src/cache_test.cpp  |   5 +
 .../cpp/core-test/src/continuous_query_test.cpp |   4 +
 .../cpp/core-test/src/ignition_test.cpp         |   4 +
 .../cpp/core-test/src/interop_test.cpp          |   4 +
 .../cpp/core-test/src/transactions_test.cpp     |   4 +
 .../cpp/odbc-test/config/queries-test-32.xml    |  47 +++
 .../odbc-test/config/queries-test-default.xml   |  38 ++
 .../odbc-test/config/queries-test-noodbc-32.xml |  47 +++
 .../cpp/odbc-test/config/queries-test.xml       |  11 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   3 +
 .../project/vs/odbc-test.vcxproj.filters        |   6 +
 .../cpp/odbc-test/src/api_robustness_test.cpp   |  20 +-
 .../cpp/odbc-test/src/queries_test.cpp          |   8 +
 .../odbc-test/src/sql_test_suite_fixture.cpp    |   6 +-
 66 files changed, 2159 insertions(+), 809 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e34222b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e34222b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
index b751274,72b7d02..eca59d6
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
@@@ -97,9 -98,6 +97,9 @@@ public class CacheDataRowAdapter implem
          assert key == null : "key";
  
          final CacheObjectContext coctx = cctx.cacheObjectContext();
-         final PageMemory pageMem = cctx.shared().database().pageMemory();
++        final PageMemory pageMem = cctx.memoryPolicy().pageMemory();
 +
 +        final int cacheId = cctx.cacheId();
  
          long nextLink = link;
          IncompleteObject<?> incomplete = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e34222b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e34222b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------


[8/8] ignite git commit: Merge branch 'ignite-3477-master' into ignite-4811

Posted by sb...@apache.org.
Merge branch 'ignite-3477-master' into ignite-4811


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

Branch: refs/heads/ignite-4811-no-start-ver
Commit: 64959753f47e53c9406b5fc68a4e626d6992856c
Parents: 7215c25 b6c1761
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Mon Apr 3 10:38:20 2017 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Mon Apr 3 10:38:20 2017 +0300

----------------------------------------------------------------------
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |  72 +-
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java |  27 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |  51 +-
 .../ignite/binary/BinaryBasicIdMapper.java      |   5 +-
 .../ignite/binary/BinaryBasicNameMapper.java    |   5 +-
 .../ignite/binary/BinaryTypeConfiguration.java  |  30 +-
 .../ignite/cache/CacheKeyConfiguration.java     |  10 +-
 .../org/apache/ignite/cache/QueryEntity.java    |  35 +-
 .../affinity/fair/FairAffinityFunction.java     |  21 +-
 .../rendezvous/RendezvousAffinityFunction.java  |  27 +-
 .../cache/eviction/AbstractEvictionPolicy.java  |  15 +-
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  74 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   2 +
 .../igfs/IgfsPerBlockLruEvictionPolicy.java     | 131 +++-
 .../cache/eviction/lru/LruEvictionPolicy.java   |  74 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  74 +-
 .../configuration/AtomicConfiguration.java      |  15 +-
 .../configuration/BinaryConfiguration.java      |  30 +-
 .../configuration/CacheConfiguration.java       |  71 +-
 .../configuration/CollectionConfiguration.java  |  35 +-
 .../configuration/ConnectorConfiguration.java   | 100 ++-
 .../configuration/FileSystemConfiguration.java  | 151 +++-
 .../configuration/HadoopConfiguration.java      |  31 +-
 .../configuration/NearCacheConfiguration.java   |  50 ++
 .../configuration/TransactionConfiguration.java |  45 +-
 .../igfs/IgfsGroupDataBlocksKeyMapper.java      |   5 +-
 .../igfs/IgfsIpcEndpointConfiguration.java      |  30 +-
 .../ignite/internal/MarshallerPlatformIds.java  |   3 +
 .../ignite/internal/binary/BinaryUtils.java     |   4 +
 .../client/GridClientConfiguration.java         | 100 ++-
 .../client/GridClientDataConfiguration.java     |  15 +-
 .../client/GridClientPartitionAffinity.java     |  15 +-
 .../balancer/GridClientBalancerAdapter.java     |   5 +-
 .../router/GridTcpRouterConfiguration.java      |  50 +-
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |  57 ++
 .../processors/cache/CacheObjectImpl.java       |  16 +-
 .../processors/cache/GridCacheProcessor.java    |  39 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  20 +-
 .../cache/transactions/IgniteTxAdapter.java     |   2 +
 .../platform/PlatformContextImpl.java           |  10 +-
 .../binary/PlatformBinaryProcessor.java         |  40 +-
 .../dotnet/PlatformDotNetConfigurationEx.java   |  21 +-
 .../utils/PlatformConfigurationUtils.java       |   3 +-
 .../service/GridServiceProcessor.java           |  14 +-
 .../optimized/OptimizedMarshaller.java          |  15 +-
 .../apache/ignite/mxbean/IgniteMBeanAware.java  |  28 +
 .../dotnet/PlatformDotNetAffinityFunction.java  |  10 +-
 .../PlatformDotNetBinaryConfiguration.java      |  31 +-
 .../PlatformDotNetBinaryTypeConfiguration.java  |  35 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  10 +-
 .../ignite/services/ServiceConfiguration.java   |  35 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  50 +-
 .../ignite/spi/IgniteSpiMBeanAdapter.java       |  78 ++
 .../checkpoint/cache/CacheCheckpointSpi.java    |  40 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  | 219 +++++-
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |   7 +
 .../sharedfs/SharedFsCheckpointSpi.java         |  54 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        | 134 +++-
 .../fifoqueue/FifoQueueCollisionSpiMBean.java   |   2 +-
 .../jobstealing/JobStealingCollisionSpi.java    | 281 ++++++-
 .../JobStealingCollisionSpiMBean.java           |   6 +-
 .../spi/collision/noop/NoopCollisionSpi.java    |   7 +
 .../PriorityQueueCollisionSpi.java              | 278 +++++--
 .../PriorityQueueCollisionSpiMBean.java         |   2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 536 +++++++++++--
 .../deployment/local/LocalDeploymentSpi.java    |  24 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   6 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 438 +++++++++--
 .../ipfinder/TcpDiscoveryIpFinderAdapter.java   |   5 +-
 .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java |  17 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  47 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |  12 +-
 .../tcp/ipfinder/vm/TcpDiscoveryVmIpFinder.java |  14 +-
 .../memory/MemoryEventStorageSpi.java           |  98 ++-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  53 +-
 .../jobstealing/JobStealingFailoverSpi.java     |  68 +-
 .../spi/failover/never/NeverFailoverSpi.java    |  30 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |   7 +
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  43 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  44 +-
 .../WeightedRandomLoadBalancingSpi.java         |  61 +-
 .../cache/MemoryPolicyConfigValidationTest.java |   2 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |  33 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  10 +-
 .../IgniteHadoopWeightedMapReducePlanner.java   |  26 +-
 .../query/h2/sql/H2CompareBigQueryTest.java     |   2 +-
 .../ignite/impl/binary/binary_type_impl.h       |   2 +-
 .../common/include/ignite/common/concurrent.h   |  30 +
 .../cpp/common/include/ignite/reference.h       |  14 +-
 .../cpp/core-test/config/cache-test.xml         |   2 +-
 .../project/vs/core-test.vcxproj.filters        |   3 +
 .../cpp/core-test/src/cache_invoke_test.cpp     |   6 +-
 .../platforms/cpp/core-test/src/cache_test.cpp  |  23 +-
 .../cpp/core-test/src/continuous_query_test.cpp | 204 ++++-
 .../cpp/core-test/src/reference_test.cpp        |  12 +-
 modules/platforms/cpp/core/Makefile.am          |   1 +
 modules/platforms/cpp/core/include/Makefile.am  |  70 +-
 .../cpp/core/include/ignite/cache/cache.h       |  22 +-
 .../ignite/cache/cache_entry_processor.h        |  42 +-
 .../cache/event/cache_entry_event_filter.h      | 109 +++
 .../cache/query/continuous/continuous_query.h   |  35 +-
 .../cpp/core/include/ignite/ignite_binding.h    |  39 +-
 .../include/ignite/ignite_binding_context.h     |   2 +-
 .../cpp/core/include/ignite/impl/bindings.h     |  95 +++
 .../impl/cache/cache_entry_processor_holder.h   |  15 -
 .../core/include/ignite/impl/cache/cache_impl.h |  81 +-
 .../cache/event/cache_entry_event_filter_base.h |  66 ++
 .../event/cache_entry_event_filter_holder.h     | 185 +++++
 .../continuous/continuous_query_handle_impl.h   |  10 -
 .../query/continuous/continuous_query_impl.h    |  60 +-
 .../include/ignite/impl/ignite_binding_impl.h   | 101 +--
 .../include/ignite/impl/ignite_environment.h    |  37 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  10 +-
 .../cpp/core/include/ignite/impl/operations.h   |   2 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   5 +
 .../cpp/core/project/vs/core.vcxproj.filters    |  18 +
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  90 ++-
 .../continuous/continuous_query_handle_impl.cpp |   5 -
 .../cpp/core/src/impl/ignite_binding_impl.cpp   |  88 +++
 .../cpp/core/src/impl/ignite_environment.cpp    | 124 ++-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   2 +-
 .../Apache.Ignite.Core.Tests.csproj             |  11 +-
 .../Binary/BinaryBuilderSelfTest.cs             | 112 +--
 .../BinaryBuilderSelfTestDynamicRegistration.cs |  40 +
 .../Binary/BinaryCompactFooterInteropTest.cs    |   2 +-
 .../Binary/BinaryDynamicRegistrationTest.cs     | 441 +++++++++++
 .../Binary/BinarySelfTest.cs                    | 197 +++--
 .../Binary/BinarySelfTestFullFooter.cs          |   5 +-
 .../Binary/JavaBinaryInteropTest.cs             | 182 +++++
 .../Serializable/AdvancedSerializationTest.cs   | 228 ++++++
 .../BasicSerializableObjectsTest.cs             | 124 +++
 .../Binary/Serializable/CallbacksTest.cs        | 369 +++++++++
 .../Binary/Serializable/DelegatesTest.cs        | 161 ++++
 .../Binary/Serializable/ObjectReferenceTests.cs | 131 ++++
 .../Binary/Serializable/PrimitivesTest.cs       | 754 +++++++++++++++++++
 .../Binary/Serializable/SqlDmlTest.cs           | 277 +++++++
 .../Cache/Affinity/AffinityFunctionTest.cs      |  21 -
 .../Cache/CacheAbstractTest.cs                  |  56 +-
 .../Cache/Query/CacheLinqTest.cs                |   2 +-
 .../Cache/Query/CacheQueriesTest.cs             |  15 +-
 .../Continuous/ContinuousQueryAbstractTest.cs   |  29 +-
 .../Cache/Store/CacheStoreTest.cs               |   5 +-
 .../Compute/ComputeApiTest.cs                   |  28 +-
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  40 +-
 .../Compute/ResourceTaskTest.cs                 |  13 +-
 .../Apache.Ignite.Core.Tests/DeploymentTest.cs  |  15 +-
 .../Examples/ExamplesTest.cs                    |   2 +-
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  28 +-
 .../SerializationTest.cs                        | 240 ------
 .../Services/ServiceProxyTest.cs                |  40 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  11 +
 .../Apache.Ignite.Core.csproj                   |   7 +-
 .../Cache/Configuration/CacheConfiguration.cs   |   8 +-
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  59 +-
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   4 +-
 .../Cache/Query/SqlFieldsQuery.cs               |   6 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  25 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |   4 +-
 .../Impl/Binary/BinarizableSerializer.cs        |   5 +-
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |  49 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |  27 +-
 .../Impl/Binary/BinaryObjectHeader.cs           |  13 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |   2 +
 .../Impl/Binary/BinaryProcessor.cs              |  38 +-
 .../Impl/Binary/BinaryReader.cs                 |  39 +-
 .../Impl/Binary/BinaryReflectiveActions.cs      |   2 +-
 .../BinaryReflectiveSerializerInternal.cs       |  84 ++-
 .../Binary/BinarySurrogateTypeDescriptor.cs     |  13 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |  96 +--
 .../Impl/Binary/BinarySystemTypeSerializer.cs   |   2 +-
 .../Impl/Binary/BinaryUtils.cs                  |  54 +-
 .../Impl/Binary/BinaryWriter.cs                 | 242 +++---
 .../Impl/Binary/DateTimeHolder.cs               | 101 ---
 .../Impl/Binary/DateTimeSerializer.cs           |  48 --
 .../Binary/DeserializationCallbackProcessor.cs  | 102 +++
 .../Impl/Binary/IBinarySerializerInternal.cs    |   4 +-
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |   8 +
 .../Impl/Binary/Io/BinaryStreamAdapter.cs       | 119 ---
 .../Impl/Binary/Marshaller.cs                   | 248 ++++--
 .../Impl/Binary/ReflectionUtils.cs              |  50 ++
 .../Impl/Binary/SerializableObjectHolder.cs     |  96 ---
 .../Impl/Binary/SerializableSerializer.cs       | 656 +++++++++++++++-
 .../Impl/Binary/TypeResolver.cs                 |   7 +
 .../Impl/Binary/UserSerializerProxy.cs          |   5 +-
 .../Affinity/AffinityFunctionSerializer.cs      |   3 -
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   3 +-
 .../Common/CopyOnWriteConcurrentDictionary.cs   |  35 +
 .../Impl/Common/DelegateConverter.cs            |  90 ++-
 .../Impl/Common/DelegateTypeDescriptor.cs       |   3 +-
 .../Impl/Common/SerializableTypeDescriptor.cs   | 222 ++++++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  13 +-
 .../Impl/Services/ServiceProxySerializer.cs     |   4 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   8 +-
 .../NuGet/LINQPad/ComputeExample.linq           |   1 -
 .../NuGet/LINQPad/PutGetExample.linq            |   5 +-
 .../NuGet/LINQPad/QueryExample.linq             |   5 +-
 .../NuGet/LINQPad/QueryExample.linq             |   5 +-
 .../dotnet/Apache.Ignite.Linq/QueryOptions.cs   |   2 +-
 .../examples/Apache.Ignite.Examples/App.config  |  16 +-
 .../Apache.Ignite.ExamplesDll/Binary/Address.cs |   2 -
 .../Compute/CharacterCountClosure.cs            |   1 -
 .../Datagrid/ContinuousQueryFilter.cs           |   2 -
 .../Datagrid/EmployeeStorePredicate.cs          |   2 -
 .../Messaging/RemoteOrderedListener.cs          |   1 -
 .../Messaging/RemoteUnorderedListener.cs        |   1 -
 .../Services/MapService.cs                      |   1 -
 .../spi/deployment/uri/UriDeploymentSpi.java    |  80 +-
 .../commands/cache/VisorCacheCommand.scala      |   2 +-
 .../helpers/jade/form/form-field-checkbox.pug   |  14 +-
 .../helpers/jade/form/form-field-datalist.pug   |   2 +-
 .../helpers/jade/form/form-field-dropdown.pug   |   3 +-
 .../app/helpers/jade/form/form-field-number.pug |   3 +-
 .../helpers/jade/form/form-field-password.pug   |   2 +-
 .../app/helpers/jade/form/form-field-text.pug   |   3 +-
 .../frontend/app/helpers/jade/mixins.pug        |   1 +
 .../frontend/app/primitives/tooltip/index.pug   |  25 +
 .../views/configuration/domains-import.tpl.pug  |   6 +
 .../zk/TcpDiscoveryZookeeperIpFinder.java       |  37 +-
 218 files changed, 10214 insertions(+), 2329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/64959753/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------


[2/8] ignite git commit: GC pressure

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 9597f87..5a8c49c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -28,7 +28,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
@@ -40,7 +39,6 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageAddRootRecord
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageCutRootRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.NewRootInitRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.RecycleRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.RemoveRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.ReplaceRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.SplitExistingPageRecord;
@@ -75,10 +73,6 @@ import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTre
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.NOT_FOUND;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.RETRY;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.RETRY_ROOT;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.initPage;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.isWalDeltaRecordNeeded;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.readPage;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.writePage;
 
 /**
  * Abstract B+Tree.
@@ -126,71 +120,86 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         private boolean keys = true;
 
-        @Override protected List<Long> getChildren(Long pageId) {
+        @Override protected List<Long> getChildren(final Long pageId) {
             if (pageId == null || pageId == 0L)
                 return null;
 
-            try (Page page = page(pageId)) {
-                long pageAddr = readLock(page); // No correctness guaranties.
+            try {
+                long page = acquirePage(pageId);
 
                 try {
-                    BPlusIO io = io(pageAddr);
+                    long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
-                    if (io.isLeaf())
-                        return null;
+                    try {
+                        BPlusIO io = io(pageAddr);
 
-                    int cnt = io.getCount(pageAddr);
+                        if (io.isLeaf())
+                            return null;
 
-                    assert cnt >= 0 : cnt;
+                        int cnt = io.getCount(pageAddr);
 
-                    List<Long> res;
+                        assert cnt >= 0 : cnt;
 
-                    if (cnt > 0) {
-                        res = new ArrayList<>(cnt + 1);
+                        List<Long> res;
 
-                        for (int i = 0; i < cnt; i++)
-                            res.add(inner(io).getLeft(pageAddr, i));
+                        if (cnt > 0) {
+                            res = new ArrayList<>(cnt + 1);
 
-                        res.add(inner(io).getRight(pageAddr, cnt - 1));
-                    }
-                    else {
-                        long left = inner(io).getLeft(pageAddr, 0);
+                            for (int i = 0; i < cnt; i++)
+                                res.add(inner(io).getLeft(pageAddr, i));
 
-                        res = left == 0 ? Collections.<Long>emptyList() : Collections.singletonList(left);
-                    }
+                            res.add(inner(io).getRight(pageAddr, cnt - 1));
+                        }
+                        else {
+                            long left = inner(io).getLeft(pageAddr, 0);
+
+                            res = left == 0 ? Collections.<Long>emptyList() : Collections.singletonList(left);
+                        }
 
-                    return res;
+                        return res;
+                    }
+                    finally {
+                        readUnlock(pageId, page, pageAddr);
+                    }
                 }
                 finally {
-                    readUnlock(page, pageAddr);
+                    releasePage(pageId, page);
                 }
             }
-            catch (IgniteCheckedException e) {
-                throw new IllegalStateException(e);
+            catch (IgniteCheckedException ignored) {
+                throw new AssertionError("Can not acquire page.");
             }
         }
 
-        @Override protected String formatTreeNode(Long pageId) {
+        @Override protected String formatTreeNode(final Long pageId) {
             if (pageId == null)
                 return ">NPE<";
 
             if (pageId == 0L)
                 return "<Zero>";
 
-            try (Page page = page(pageId)) {
-                long pageAddr = readLock(page); // No correctness guaranties.
-
+            try {
+                long page = acquirePage(pageId);
                 try {
-                    BPlusIO<L> io = io(pageAddr);
+                    long pageAddr = readLock(pageId, page); // No correctness guaranties.
+                    try {
+                        BPlusIO<L> io = io(pageAddr);
 
-                    return printPage(io, pageAddr, keys);
+                        return printPage(io, pageAddr, keys);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IllegalStateException(e);
+                    }
+                    finally {
+                        readUnlock(pageId, page, pageAddr);
+                    }
                 }
                 finally {
-                    readUnlock(page, pageAddr);
+                    releasePage(pageId, page);
                 }
             }
-            catch (IgniteCheckedException e) {
-                throw new IllegalStateException(e);
+            catch (IgniteCheckedException ignored) {
+                throw new AssertionError("Can not acquire page.");
             }
         }
     };
@@ -203,7 +212,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class AskNeighbor extends GetPageHandler<Get> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int isBack) {
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Get g, int isBack) {
             assert !io.isLeaf(); // Inner page.
 
             boolean back = isBack == TRUE.ordinal();
@@ -234,7 +243,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class Search extends GetPageHandler<Get> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int lvl)
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Get g, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
             if (io.getForward(pageAddr) != g.fwdId)
@@ -317,7 +326,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     private class Replace extends GetPageHandler<Put> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException  {
             // Check the triangle invariant.
             if (io.getForward(pageAddr) != p.fwdId)
@@ -359,12 +368,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 p.finish();
             }
 
-            boolean needWal = needWalDeltaRecord(page);
+            boolean needWal = needWalDeltaRecord(pageId, page, null);
 
             byte[] newRowBytes = io.store(pageAddr, idx, newRow, null, needWal);
 
             if (needWal)
-                wal.log(new ReplaceRecord<>(cacheId, page.id(), io, newRowBytes, idx));
+                wal.log(new ReplaceRecord<>(cacheId, pageId, io, newRowBytes, idx));
 
             return FOUND;
         }
@@ -378,7 +387,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class Insert extends GetPageHandler<Put> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             assert p.btmLvl == lvl : "we must always insert at the bottom level: " + p.btmLvl + " " + lvl;
 
@@ -395,7 +404,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             idx = fix(idx);
 
             // Do insert.
-            L moveUpRow = p.insert(page, io, pageAddr, idx, lvl);
+            L moveUpRow = p.insert(pageId, page, pageAddr, io, idx, lvl);
 
             // Check if split happened.
             if (moveUpRow != null) {
@@ -408,7 +417,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 // Here forward page can't be concurrently removed because we keep write lock on tail which is the only
                 // page who knows about the forward page, because it was just produced by split.
                 p.rightId = io.getForward(pageAddr);
-                p.tail(page, pageAddr);
+                p.tail(pageId, page, pageAddr);
 
                 assert p.rightId != 0;
             }
@@ -427,15 +436,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class RemoveFromLeaf extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(long leafId, long leafPage, long leafAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
-            final int cnt = io.getCount(pageAddr);
+            final int cnt = io.getCount(leafAddr);
 
             assert cnt <= Short.MAX_VALUE: cnt;
 
-            int idx = findInsertionPoint(io, pageAddr, 0, cnt, r.row, 0);
+            int idx = findInsertionPoint(io, leafAddr, 0, cnt, r.row, 0);
 
             if (idx < 0)
                 return RETRY; // We've found exact match on search but now it's gone.
@@ -444,14 +453,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             // Need to do inner replace when we remove the rightmost element and the leaf have no forward page,
             // i.e. it is not the rightmost leaf of the tree.
-            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(pageAddr) != 0;
+            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(leafAddr) != 0;
 
             // !!! Before modifying state we have to make sure that we will not go for retry.
 
             // We may need to replace inner key or want to merge this leaf with sibling after the remove -> keep lock.
             if (needReplaceInner ||
                 // We need to make sure that we have back or forward to be able to merge.
-                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(pageAddr, pageSize())))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(leafAddr, pageSize())))) {
                 // If we have backId then we've already locked back page, nothing to do here.
                 if (r.fwdId != 0 && r.backId == 0) {
                     Result res = r.lockForward(0);
@@ -475,7 +484,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 if (needReplaceInner)
                     r.needReplaceInner = TRUE;
 
-                Tail<L> t = r.addTail(leaf, pageAddr, io, 0, Tail.EXACT);
+                Tail<L> t = r.addTail(leafId, leafPage, leafAddr, io, 0, Tail.EXACT);
 
                 t.idx = (short)idx;
 
@@ -484,7 +493,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return FOUND;
             }
 
-            r.removeDataRowFromLeaf(leaf, io, pageAddr, cnt, idx);
+            r.removeDataRowFromLeaf(leafId, leafPage, leafAddr, null, io, cnt, idx);
 
             return FOUND;
         }
@@ -498,10 +507,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockBackAndRmvFromLeaf extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(long backId, long backPage, long backAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(pageAddr) != r.pageId)
+            if (io.getForward(backAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
@@ -509,7 +518,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             // Keep locks on back and leaf pages for subsequent merges.
             if (res == FOUND && r.tail != null)
-                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
+                r.addTail(backId, backPage, backAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -523,17 +532,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockBackAndTail extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(long backId, long backPage, long backAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(pageAddr) != r.pageId)
+            if (io.getForward(backAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
             Result res = r.doLockTail(lvl);
 
             if (res == FOUND)
-                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
+                r.addTail(backId, backPage, backAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -547,9 +556,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockTailForward extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
-            r.addTail(page, pageAddr, io, lvl, Tail.FORWARD);
+            r.addTail(pageId, page, pageAddr, io, lvl, Tail.FORWARD);
 
             return FOUND;
         }
@@ -563,7 +572,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockTail extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
@@ -579,7 +588,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return res; // Retry.
             }
 
-            r.addTail(page, pageAddr, io, lvl, Tail.EXACT);
+            r.addTail(pageId, page, pageAddr, io, lvl, Tail.EXACT);
 
             return FOUND;
         }
@@ -593,23 +602,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class CutRoot extends PageHandler<Void, Bool> {
         /** {@inheritDoc} */
-        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Void ignore, int lvl)
+        @Override public Bool run(int cacheId, long metaId, long metaPage, long metaAddr, PageIO iox, Boolean walPlc, Void ignore, int lvl)
             throws IgniteCheckedException {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            assert lvl == io.getRootLevel(pageAddr); // Can drop only root.
+            assert lvl == io.getRootLevel(metaAddr); // Can drop only root.
 
-            io.cutRoot(pageAddr, pageSize());
+            io.cutRoot(metaAddr, pageSize());
 
-            if (needWalDeltaRecord(meta))
-                wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
+            if (needWalDeltaRecord(metaId, metaPage, walPlc))
+                wal.log(new MetaPageCutRootRecord(cacheId, metaId));
 
             int newLvl = lvl - 1;
 
-            assert io.getRootLevel(pageAddr) == newLvl;
+            assert io.getRootLevel(metaAddr) == newLvl;
 
-            treeMeta = new TreeMetaData(newLvl, io.getFirstPageId(pageAddr, newLvl));
+            treeMeta = new TreeMetaData(newLvl, io.getFirstPageId(metaAddr, newLvl));
 
             return TRUE;
         }
@@ -623,7 +632,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class AddRoot extends PageHandler<Long, Bool> {
         /** {@inheritDoc} */
-        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootPageId, int lvl)
+        @Override public Bool run(int cacheId, long metaId, long metaPage, long pageAddr, PageIO iox, Boolean walPlc, Long rootPageId, int lvl)
             throws IgniteCheckedException {
             assert rootPageId != null;
 
@@ -634,8 +643,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             io.addRoot(pageAddr, rootPageId, pageSize());
 
-            if (needWalDeltaRecord(meta))
-                wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
+            if (needWalDeltaRecord(metaId, metaPage, walPlc))
+                wal.log(new MetaPageAddRootRecord(cacheId, metaId, rootPageId));
 
             assert io.getRootLevel(pageAddr) == lvl;
             assert io.getFirstPageId(pageAddr, lvl) == rootPageId;
@@ -654,7 +663,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class InitRoot extends PageHandler<Long, Bool> {
         /** {@inheritDoc} */
-        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootId, int inlineSize)
+        @Override public Bool run(int cacheId, long metaId, long metaPage, long pageAddr, PageIO iox, Boolean walPlc, Long rootId, int inlineSize)
             throws IgniteCheckedException {
             assert rootId != null;
 
@@ -664,8 +673,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             io.initRoot(pageAddr, rootId, pageSize());
             io.setInlineSize(pageAddr, inlineSize);
 
-            if (needWalDeltaRecord(meta))
-                wal.log(new MetaPageInitRootInlineRecord(cacheId, meta.id(), rootId, inlineSize));
+            if (needWalDeltaRecord(metaId, metaPage, walPlc))
+                wal.log(new MetaPageInitRootInlineRecord(cacheId, metaId, rootId, inlineSize));
 
             assert io.getRootLevel(pageAddr) == 0;
             assert io.getFirstPageId(pageAddr, 0) == rootId;
@@ -760,14 +769,6 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param page Updated page.
-     * @return {@code true} If we need to make a delta WAL record for the change in this page.
-     */
-    private boolean needWalDeltaRecord(Page page) {
-        return isWalDeltaRecordNeeded(wal, page);
-    }
-
-    /**
      * Initialize new tree.
      *
      * @param initNew {@code True} if new tree should be created.
@@ -789,16 +790,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             // Allocate the first leaf page, it will be our root.
             long rootId = allocatePage(null);
 
-            try (Page root = page(rootId)) {
-                initPage(pageMem, root, this, latestLeafIO(), wal);
-            }
+            init(rootId, latestLeafIO());
 
             // Initialize meta page with new root page.
-            try (Page meta = page(metaPageId)) {
-                Bool res = writePage(pageMem, meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, inlineSize, FALSE);
+            Bool res = write(metaPageId, initRoot, BPlusMetaIO.VERSIONS.latest(), rootId, inlineSize, FALSE);
 
-                assert res == TRUE: res;
-            }
+            assert res == TRUE: res;
 
             assert treeMeta != null;
         }
@@ -814,10 +811,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (meta0 != null)
             return meta0;
 
-        try (Page meta = page(metaPageId)) {
-            long pageAddr = readLock(meta); // Meta can't be removed.
+        final long metaPage = acquirePage(metaPageId);
+        try {
+            long pageAddr = readLock(metaPageId, metaPage); // Meta can't be removed.
 
-            assert pageAddr != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert pageAddr != 0 : "Failed to read lock meta page [metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
@@ -829,9 +827,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 treeMeta = meta0 = new TreeMetaData(rootLvl, rootId);
             }
             finally {
-                readUnlock(meta, pageAddr);
+                readUnlock(metaPageId, metaPage, pageAddr);
             }
         }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
 
         return meta0;
     }
@@ -849,12 +850,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param meta Meta page.
+     * @param metaId Meta page ID.
+     * @param metaPage Meta page pointer.
      * @param lvl Level, if {@code 0} then it is a bottom level, if negative then root.
      * @return Page ID.
      */
-    private long getFirstPageId(Page meta, int lvl) {
-        long pageAddr = readLock(meta); // Meta can't be removed.
+    private long getFirstPageId(long metaId, long metaPage, int lvl) {
+        long pageAddr = readLock(metaId, metaPage); // Meta can't be removed.
 
         try {
             BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
@@ -868,7 +870,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             return io.getFirstPageId(pageAddr, lvl);
         }
         finally {
-            readUnlock(meta, pageAddr);
+            readUnlock(metaId, metaPage, pageAddr);
         }
     }
 
@@ -882,20 +884,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         long firstPageId;
 
-        try (Page meta = page(metaPageId)) {
-            firstPageId = getFirstPageId(meta, 0); // Level 0 is always at the bottom.
+        long metaPage = acquirePage(metaPageId);
+        try  {
+            firstPageId = getFirstPageId(metaPageId, metaPage, 0); // Level 0 is always at the bottom.
+        }
+        finally {
+            releasePage(metaPageId, metaPage);
         }
 
-        try (Page first = page(firstPageId)) {
-            long pageAddr = readLock(first); // We always merge pages backwards, the first page is never removed.
+        long firstPage = acquirePage(firstPageId);
+
+        try {
+            long pageAddr = readLock(firstPageId, firstPage); // We always merge pages backwards, the first page is never removed.
 
             try {
                 cursor.init(pageAddr, io(pageAddr), 0);
             }
             finally {
-                readUnlock(first, pageAddr);
+                readUnlock(firstPageId, firstPage, pageAddr);
             }
         }
+        finally {
+            releasePage(firstPageId, firstPage);
+        }
 
         return cursor;
     }
@@ -1007,7 +1018,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private Result findDown(final Get g, final long pageId, final long fwdId, final int lvl)
         throws IgniteCheckedException {
-        Page page = page(pageId);
+        long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -1015,7 +1026,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 g.pageId = pageId;
                 g.fwdId = fwdId;
 
-                Result res = readPage(page, this, search, g, lvl, RETRY);
+                Result res = read(pageId, page, search, g, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN:
@@ -1049,8 +1060,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
         finally {
-            if (g.canRelease(page, lvl))
-                page.close();
+            if (g.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -1067,16 +1078,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * For debug.
      *
      * @return Tree as {@link String}.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unused")
-    public final String printTree() {
+    public final String printTree() throws IgniteCheckedException {
         long rootPageId;
 
-        try (Page meta = page(metaPageId)) {
-            rootPageId = getFirstPageId(meta, -1);
+        long metaPage = acquirePage(metaPageId);
+        try {
+            rootPageId = getFirstPageId(metaPageId, metaPage, -1);
         }
-        catch (IgniteCheckedException e) {
-            throw new IllegalStateException(e);
+        finally {
+            releasePage(metaPageId, metaPage);
         }
 
         return treePrinter.print(rootPageId);
@@ -1089,20 +1102,24 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long rootPageId;
         int rootLvl;
 
-        try (Page meta = page(metaPageId)) {
+        long metaPage = acquirePage(metaPageId);
+        try  {
             rootLvl = getRootLevel();
 
             if (rootLvl < 0)
                 fail("Root level: " + rootLvl);
 
-            validateFirstPages(meta, rootLvl);
+            validateFirstPages(metaPageId, metaPage, rootLvl);
 
-            rootPageId = getFirstPageId(meta, rootLvl);
+            rootPageId = getFirstPageId(metaPageId, metaPage, rootLvl);
 
             validateDownPages(rootPageId, 0L, rootLvl);
 
             validateDownKeys(rootPageId, null);
         }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
     }
 
     /**
@@ -1111,8 +1128,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @throws IgniteCheckedException If failed.
      */
     private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(pageAddr);
@@ -1161,9 +1179,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 validateDownKeys(rightId, minRow);
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
     }
 
     /**
@@ -1172,8 +1193,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @throws IgniteCheckedException If failed.
      */
     private L getGreatestRowInSubTree(long pageId) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(pageAddr);
@@ -1192,19 +1214,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return getGreatestRowInSubTree(rightId);
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
     }
 
     /**
-     * @param meta Meta page.
+     * @param metaId Meta page ID.
+     * @param metaPage Meta page pointer.
      * @param rootLvl Root level.
      * @throws IgniteCheckedException If failed.
      */
-    private void validateFirstPages(Page meta, int rootLvl) throws IgniteCheckedException {
+    private void validateFirstPages(long metaId, long metaPage, int rootLvl) throws IgniteCheckedException {
         for (int lvl = rootLvl; lvl > 0; lvl--)
-            validateFirstPage(meta, lvl);
+            validateFirstPage(metaId, metaPage, lvl);
     }
 
     /**
@@ -1215,20 +1241,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param meta Meta page.
+     * @param metaId Meta page ID.
+     * @param metaPage Meta page pointer.
      * @param lvl Level.
      * @throws IgniteCheckedException If failed.
      */
-    private void validateFirstPage(Page meta, int lvl) throws IgniteCheckedException {
+    private void validateFirstPage(long metaId, long metaPage, int lvl) throws IgniteCheckedException {
         if (lvl == 0)
             fail("Leaf level: " + lvl);
 
-        long pageId = getFirstPageId(meta, lvl);
+        long pageId = getFirstPageId(metaId, metaPage, lvl);
 
         long leftmostChildId;
 
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(pageAddr);
@@ -1239,11 +1267,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 leftmostChildId = inner(io).getLeft(pageAddr, 0);
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
 
-        long firstDownPageId = getFirstPageId(meta, lvl - 1);
+        long firstDownPageId = getFirstPageId(metaId, metaPage, lvl - 1);
 
         if (firstDownPageId != leftmostChildId)
             fail(new SB("First: meta ").appendHex(firstDownPageId).a(", child ").appendHex(leftmostChildId));
@@ -1255,9 +1286,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @param lvl Level.
      * @throws IgniteCheckedException If failed.
      */
-    private void validateDownPages(long pageId, long fwdId, final int lvl) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+    private void validateDownPages(long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 long realPageId = BPlusIO.getPageId(pageAddr);
@@ -1291,8 +1323,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                     if (fwdId != 0) {
                         // For the rightmost child ask neighbor.
-                        try (Page fwd = page(fwdId)) {
-                            long fwdPageAddr = readLock(fwd); // No correctness guaranties.
+                        long fwdId0 = fwdId;
+                        long fwdPage = acquirePage(fwdId0);
+                        try {
+                            long fwdPageAddr = readLock(fwdId0, fwdPage); // No correctness guaranties.
 
                             try {
                                 if (io(fwdPageAddr) != io)
@@ -1301,20 +1335,26 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                                 fwdId = inner(io).getLeft(fwdPageAddr, 0);
                             }
                             finally {
-                                readUnlock(fwd, fwdPageAddr);
+                                readUnlock(fwdId0, fwdPage, fwdPageAddr);
                             }
                         }
+                        finally {
+                            releasePage(fwdId0, fwdPage);
+                        }
                     }
 
-                    pageId = inner(io).getLeft(pageAddr, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
+                    long leftId = inner(io).getLeft(pageAddr, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
 
-                    validateDownPages(pageId, fwdId, lvl - 1);
+                    validateDownPages(leftId, fwdId, lvl - 1);
                 }
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
     }
 
     /**
@@ -1512,7 +1552,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (x.isTail(pageId, lvl))
             return FOUND; // We've already locked this page, so return that we are ok.
 
-        final Page page = page(pageId);
+        long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -1521,7 +1561,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 x.fwdId(fwdId);
                 x.backId(backId);
 
-                Result res = readPage(page, this, search, x, lvl, RETRY);
+                Result res = read(pageId, page, search, x, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN_X:
@@ -1540,7 +1580,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         // Intentional fallthrough.
                     case GO_DOWN:
-                        res = x.tryReplaceInner(page, pageId, fwdId, lvl);
+                        res = x.tryReplaceInner(pageId, page, fwdId, lvl);
 
                         if (res != RETRY)
                             res = invokeDown(x, x.pageId, x.backId, x.fwdId, lvl - 1);
@@ -1560,7 +1600,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         assert x.isRemove(); // Guarded by isFinished.
 
-                        res = x.finishOrLockTail(page, pageId, backId, fwdId, lvl);
+                        res = x.finishOrLockTail(pageId, page, backId, fwdId, lvl);
 
                         return res;
 
@@ -1568,13 +1608,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         if (lvl == 0)
                             x.invokeClosure();
 
-                        return x.onNotFound(page, pageId, fwdId, lvl);
+                        return x.onNotFound(pageId, page, fwdId, lvl);
 
                     case FOUND:
                         if (lvl == 0)
                             x.invokeClosure();
 
-                        return x.onFound(page, pageId, backId, fwdId, lvl);
+                        return x.onFound(pageId, page, backId, fwdId, lvl);
 
                     default:
                         return res;
@@ -1584,8 +1624,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         finally {
             x.levelExit();
 
-            if (x.canRelease(page, lvl))
-                page.close();
+            if (x.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -1666,7 +1706,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (r.isTail(pageId, lvl))
             return FOUND; // We've already locked this page, so return that we are ok.
 
-        final Page page = page(pageId);
+        long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -1675,7 +1715,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 r.fwdId = fwdId;
                 r.backId = backId;
 
-                Result res = readPage(page, this, search, r, lvl, RETRY);
+                Result res = read(pageId, page, search, r, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN_X:
@@ -1705,7 +1745,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         if (res == RETRY_ROOT || r.isFinished())
                             return res;
 
-                        res = r.finishOrLockTail(page, pageId, backId, fwdId, lvl);
+                        res = r.finishOrLockTail(pageId, page, backId, fwdId, lvl);
 
                         return res;
 
@@ -1718,7 +1758,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         return res;
 
                     case FOUND:
-                        return r.tryRemoveFromLeaf(page, pageId, backId, fwdId, lvl);
+                        return r.tryRemoveFromLeaf(pageId, page, backId, fwdId, lvl);
 
                     default:
                         return res;
@@ -1726,10 +1766,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
         finally {
-            r.page = null;
+            r.page = 0L;
 
-            if (r.canRelease(page, lvl))
-                page.close();
+            if (r.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -1782,8 +1822,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         long pageId;
 
-        try (Page meta = page(metaPageId)) {
-            pageId = getFirstPageId(meta, 0); // Level 0 is always at the bottom.
+        long metaPage = acquirePage(metaPageId);
+        try {
+            pageId = getFirstPageId(metaPageId, metaPage, 0); // Level 0 is always at the bottom.
+        }
+        finally {
+            releasePage(metaPageId, metaPage);
         }
 
         BPlusIO<L> io = null;
@@ -1791,24 +1835,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long cnt = 0;
 
         while (pageId != 0) {
-            try (Page page = page(pageId)) {
-                long pageAddr = readLock(page); // No correctness guaranties.
+            long pageId0 = pageId;
+            long page = acquirePage(pageId0);
+            try {
+                long curAddr = readLock(pageId0, page); // No correctness guaranties.
 
                 try {
                     if (io == null) {
-                        io = io(pageAddr);
+                        io = io(curAddr);
 
                         assert io.isLeaf();
                     }
 
-                    cnt += io.getCount(pageAddr);
+                    cnt += io.getCount(curAddr);
 
-                    pageId = io.getForward(pageAddr);
+                    pageId = io.getForward(curAddr);
                 }
                 finally {
-                    readUnlock(page, pageAddr);
+                    readUnlock(pageId0, page, curAddr);
                 }
             }
+            finally {
+                releasePage(pageId0, page);
+            }
         }
 
         return cnt;
@@ -1921,16 +1970,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         long pagesCnt = 0;
 
-        try (Page meta = page(metaPageId)) {
-            long metaPageAddr = writeLock(meta); // No checks, we must be out of use.
+        long metaPage = acquirePage(metaPageId);
+        try {
+            long metaPageAddr = writeLock(metaPageId, metaPage); // No checks, we must be out of use.
 
             try {
                 for (long pageId : getFirstPageIds(metaPageAddr)) {
                     assert pageId != 0;
 
                     do {
-                        try (Page page = page(pageId)) {
-                            long pageAddr = writeLock(page); // No checks, we must be out of use.
+                        long page = acquirePage(pageId);
+                        try {
+                            long pageAddr = writeLock(pageId, page); // No checks, we must be out of use.
 
                             try {
                                 BPlusIO<L> io = io(pageAddr);
@@ -1940,15 +1991,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                                 long fwdPageId = io.getForward(pageAddr);
 
-                                bag.addFreePage(recyclePage(pageId, page, pageAddr));
+                                bag.addFreePage(recyclePage(pageId, page, pageAddr, null));
                                 pagesCnt++;
 
                                 pageId = fwdPageId;
                             }
                             finally {
-                                writeUnlock(page, pageAddr, true);
+                                writeUnlock(pageId, page, pageAddr, true);
                             }
                         }
+                        finally {
+                            releasePage(pageId, page);
+                        }
 
                         if (bag.size() == 128) {
                             reuseList.addForRecycle(bag);
@@ -1959,13 +2013,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     while (pageId != 0);
                 }
 
-                bag.addFreePage(recyclePage(metaPageId, meta, metaPageAddr));
+                bag.addFreePage(recyclePage(metaPageId, metaPage, metaPageAddr, null));
                 pagesCnt++;
             }
             finally {
-                writeUnlock(meta, metaPageAddr, true);
+                writeUnlock(metaPageId, metaPage, metaPageAddr, true);
             }
         }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
 
         reuseList.addForRecycle(bag);
 
@@ -1998,45 +2055,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param pageId Page ID.
-     * @param page Page.
-     * @param pageAddr Page address.
-     * @return Recycled page ID.
-     * @throws IgniteCheckedException If failed.
-     */
-    private long recyclePage(long pageId, Page page, long pageAddr) throws IgniteCheckedException {
-        // Rotate page ID to avoid concurrency issues with reused pages.
-        pageId = PageIdUtils.rotatePageId(pageId);
-
-        // Update page ID inside of the buffer, Page.id() will always return the original page ID.
-        PageIO.setPageId(pageAddr, pageId);
-
-        if (needWalDeltaRecord(page))
-            wal.log(new RecycleRecord(cacheId, page.id(), pageId));
-
-        return pageId;
-    }
-
-    /**
+     * @param page Page pointer.
+     * @param pageAddr Page address
      * @param io IO.
-     * @param page Page to split.
-     * @param buf Splitting buffer.
      * @param fwdId Forward page ID.
-     * @param fwd Forward page.
      * @param fwdBuf Forward buffer.
      * @param idx Insertion index.
      * @return {@code true} The middle index was shifted to the right.
      * @throws IgniteCheckedException If failed.
      */
     private boolean splitPage(
-        BPlusIO io,
-        Page page,
-        long buf,
-        long fwdId,
-        Page fwd,
-        long fwdBuf,
-        int idx
+        long pageId, long page, long pageAddr, BPlusIO io, long fwdId, long fwdBuf, int idx
     ) throws IgniteCheckedException {
-        int cnt = io.getCount(buf);
+        int cnt = io.getCount(pageAddr);
         int mid = cnt >>> 1;
 
         boolean res = false;
@@ -2048,43 +2079,41 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         // Update forward page.
-        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt, pageSize());
-
-        // TODO GG-11640 log a correct forward page record.
-        fwd.fullPageWalRecordPolicy(Boolean.TRUE);
+        io.splitForwardPage(pageAddr, fwdId, fwdBuf, mid, cnt, pageSize());
 
         // Update existing page.
-        io.splitExistingPage(buf, mid, fwdId);
+        io.splitExistingPage(pageAddr, mid, fwdId);
 
-        if (needWalDeltaRecord(page))
-            wal.log(new SplitExistingPageRecord(cacheId, page.id(), mid, fwdId));
+        if (needWalDeltaRecord(pageId, page, null))
+            wal.log(new SplitExistingPageRecord(cacheId, pageId, mid, fwdId));
 
         return res;
     }
 
     /**
-     * @param page Page.
-     * @param pageAddr Page address.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address
+     * @param walPlc Full page WAL record policy.
      */
-    private void writeUnlockAndClose(Page page, long pageAddr) {
+    private void writeUnlockAndClose(long pageId, long page, long pageAddr, Boolean walPlc) {
         try {
-            writeUnlock(page, pageAddr, true);
+            writeUnlock(pageId, page, pageAddr, walPlc, true);
         }
         finally {
-            page.close();
+            releasePage(pageId, page);
         }
     }
 
     /**
      * @param pageId Inner page ID.
+     * @param g Get.
      * @param back Get back (if {@code true}) or forward page (if {@code false}).
      * @return Operation result.
+     * @throws IgniteCheckedException If failed.
      */
     private Result askNeighbor(long pageId, Get g, boolean back) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            return readPage(page, this, askNeighbor, g,
-                back ? TRUE.ordinal() : FALSE.ordinal(), RETRY);
-        }
+        return read(pageId, askNeighbor, g, back ? TRUE.ordinal() : FALSE.ordinal(), RETRY);
     }
 
     /**
@@ -2099,7 +2128,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         throws IgniteCheckedException {
         assert lvl >= 0 : lvl;
 
-        final Page page = page(pageId);
+        final long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -2107,7 +2136,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 p.pageId = pageId;
                 p.fwdId = fwdId;
 
-                Result res = readPage(page, this, search, p, lvl, RETRY);
+                Result res = read(pageId, page, search, p, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN:
@@ -2116,7 +2145,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         assert p.pageId != pageId;
                         assert p.fwdId != fwdId || fwdId == 0;
 
-                        res = p.tryReplaceInner(page, pageId, fwdId, lvl);
+                        res = p.tryReplaceInner(pageId, page, fwdId, lvl);
 
                         if (res != RETRY) // Go down recursively.
                             res = putDown(p, p.pageId, p.fwdId, lvl - 1);
@@ -2132,13 +2161,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     case FOUND: // Do replace.
                         assert lvl == 0 : "This replace can happen only at the bottom level.";
 
-                        return p.tryReplace(page, pageId, fwdId, lvl);
+                        return p.tryReplace(pageId, page, fwdId, lvl);
 
                     case NOT_FOUND: // Do insert.
                         assert lvl == p.btmLvl : "must insert at the bottom level";
                         assert p.needReplaceInner == FALSE : p.needReplaceInner + " " + lvl;
 
-                        return p.tryInsert(page, pageId, fwdId, lvl);
+                        return p.tryInsert(pageId, page, fwdId, lvl);
 
                     default:
                         return res;
@@ -2146,8 +2175,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
         finally {
-            if (p.canRelease(page, lvl))
-                page.close();
+            if (p.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -2289,12 +2318,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page.
          * @param lvl Level.
          * @return {@code true} If we can release the given page.
          */
-        boolean canRelease(Page page, int lvl) {
-            return page != null;
+        boolean canRelease(long pageId, int lvl) {
+            return pageId != 0L;
         }
 
         /**
@@ -2408,10 +2437,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * It is needed because split row will be "in flight" and if we'll release tail, remove on
          * split row may fail.
          */
-        Page tail;
+        long tailId;
+
+        /** */
+        long tailPage;
 
         /** */
-        long tailPageAddr;
+        long tailAddr;
 
         /**
          * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level.
@@ -2459,22 +2491,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param tail Tail page.
-         * @param tailPageAddr Tail page address.
+         * @param tailId Tail page ID.
+         * @param tailPage Tail page pointer.
+         * @param tailPageAddr Tail page address
          */
-        private void tail(Page tail, long tailPageAddr) {
-            assert (tail == null) == (tailPageAddr == 0L);
+        private void tail(long tailId, long tailPage, long tailPageAddr) {
+            assert (tailId == 0L) == (tailPage == 0L);
+            assert (tailPage == 0L) == (tailPageAddr == 0L);
 
-            if (this.tail != null)
-                writeUnlockAndClose(this.tail, this.tailPageAddr);
+            if (this.tailAddr != 0L)
+                writeUnlockAndClose(this.tailId, this.tailPage, this.tailAddr, null);
 
-            this.tail = tail;
-            this.tailPageAddr = tailPageAddr;
+            this.tailId = tailId;
+            this.tailPage = tailPage;
+            this.tailAddr = tailPageAddr;
         }
 
         /** {@inheritDoc} */
-        @Override boolean canRelease(Page page, int lvl) {
-            return page != null && tail != page;
+        @Override boolean canRelease(long pageId, int lvl) {
+            return pageId != 0L && tailId != pageId;
         }
 
         /**
@@ -2484,7 +2519,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             row = null;
             rightId = 0;
 
-            tail(null, 0L);
+            tail(0L, 0L, 0L);
         }
 
         /** {@inheritDoc} */
@@ -2493,87 +2528,92 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param io IO.
          * @param idx Index.
          * @param lvl Level.
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insert(Page page, BPlusIO<L> io, long pageAddr, int idx, int lvl)
+        private L insert(long pageId, long page, long pageAddr, BPlusIO<L> io, int idx, int lvl)
             throws IgniteCheckedException {
             int maxCnt = io.getMaxCount(pageAddr, pageSize());
             int cnt = io.getCount(pageAddr);
 
             if (cnt == maxCnt) // Need to split page.
-                return insertWithSplit(page, io, pageAddr, idx, lvl);
+                return insertWithSplit(pageId, page, pageAddr, io, idx, lvl);
 
-            insertSimple(page, io, pageAddr, idx);
+            insertSimple(pageId, page, pageAddr, io, idx, null);
 
             return null;
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param io IO.
          * @param idx Index.
+         * @param walPlc Full page WAL record policy.
          * @throws IgniteCheckedException If failed.
          */
-        private void insertSimple(Page page, BPlusIO<L> io, long pageAddr, int idx)
+        private void insertSimple(long pageId, long page, long pageAddr, BPlusIO<L> io, int idx, Boolean walPlc)
             throws IgniteCheckedException {
-            boolean needWal = needWalDeltaRecord(page);
+            boolean needWal = needWalDeltaRecord(pageId, page, walPlc);
 
             byte[] rowBytes = io.insert(pageAddr, idx, row, null, rightId, needWal);
 
             if (needWal)
-                wal.log(new InsertRecord<>(cacheId, page.id(), io, idx, rowBytes, rightId));
+                wal.log(new InsertRecord<>(cacheId, pageId, io, idx, rowBytes, rightId));
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param io IO.
          * @param idx Index.
          * @param lvl Level.
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insertWithSplit(Page page, BPlusIO<L> io, final long pageAddr, int idx, int lvl)
+        private L insertWithSplit(long pageId, long page, long pageAddr, BPlusIO<L> io, int idx, int lvl)
             throws IgniteCheckedException {
             long fwdId = allocatePage(null);
+            long fwdPage = acquirePage(fwdId);
 
-            try (Page fwd = page(fwdId)) {
+            try {
                 // Need to check this before the actual split, because after the split we will have new forward page here.
                 boolean hadFwd = io.getForward(pageAddr) != 0;
 
-                long fwdPageAddr = writeLock(fwd); // Initial write, no need to check for concurrent modification.
+                long fwdPageAddr = writeLock(fwdId, fwdPage); // Initial write, no need to check for concurrent modification.
 
                 assert fwdPageAddr != 0L;
 
-                try {
-                    // Never write full forward page, because it is known to be new.
-                    fwd.fullPageWalRecordPolicy(Boolean.FALSE);
+                // TODO GG-11640 log a correct forward page record.
+                Boolean fwdPageWalPlc = Boolean.TRUE;
 
-                    boolean midShift = splitPage(io, page, pageAddr, fwdId, fwd, fwdPageAddr, idx);
+                try {
+                    boolean midShift = splitPage(pageId, page, pageAddr, io, fwdId, fwdPageAddr, idx);
 
                     // Do insert.
                     int cnt = io.getCount(pageAddr);
 
                     if (idx < cnt || (idx == cnt && !midShift)) { // Insert into back page.
-                        insertSimple(page, io, pageAddr, idx);
+                        insertSimple(pageId, page, pageAddr, io, idx, null);
 
                         // Fix leftmost child of forward page, because newly inserted row will go up.
                         if (idx == cnt && !io.isLeaf()) {
                             inner(io).setLeft(fwdPageAddr, 0, rightId);
 
-                            if (needWalDeltaRecord(fwd)) // Rare case, we can afford separate WAL record to avoid complexity.
-                                wal.log(new FixLeftmostChildRecord(cacheId, fwd.id(), rightId));
+                            if (needWalDeltaRecord(fwdId, fwdPage, fwdPageWalPlc)) // Rare case, we can afford separate WAL record to avoid complexity.
+                                wal.log(new FixLeftmostChildRecord(cacheId, fwdId, rightId));
                         }
                     }
                     else // Insert into newly allocated forward page.
-                        insertSimple(fwd, io, fwdPageAddr, idx - cnt);
+                        insertSimple(fwdId, fwdPage, fwdPageAddr, io, idx - cnt, fwdPageWalPlc);
 
                     // Do move up.
                     cnt = io.getCount(pageAddr);
@@ -2584,30 +2624,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     if (!io.isLeaf()) { // Leaf pages must contain all the links, inner pages remove moveUpLink.
                         io.setCount(pageAddr, cnt - 1);
 
-                        if (needWalDeltaRecord(page)) // Rare case, we can afford separate WAL record to avoid complexity.
-                            wal.log(new FixCountRecord(cacheId, page.id(), cnt - 1));
+                        if (needWalDeltaRecord(pageId, page, null)) // Rare case, we can afford separate WAL record to avoid complexity.
+                            wal.log(new FixCountRecord(cacheId, pageId, cnt - 1));
                     }
 
                     if (!hadFwd && lvl == getRootLevel()) { // We are splitting root.
                         long newRootId = allocatePage(null);
+                        long newRootPage = acquirePage(newRootId);
 
-                        try (Page newRoot = page(newRootId)) {
+                        try {
                             if (io.isLeaf())
                                 io = latestInnerIO();
 
-                            long newRootPageAddr = writeLock(newRoot); // Initial write.
+                            long newRootAddr = writeLock(newRootId, newRootPage); // Initial write.
 
-                            assert newRootPageAddr != 0L;
+                            assert newRootAddr != 0L;
 
-                            try {
-                                // Never write full new root page, because it is known to be new.
-                                newRoot.fullPageWalRecordPolicy(Boolean.FALSE);
-
-                                long pageId = PageIO.getPageId(pageAddr);
+                            // Never write full new root page, because it is known to be new.
+                            Boolean newRootPageWalPlc = Boolean.FALSE;
 
-                                boolean needWal = needWalDeltaRecord(newRoot);
+                            try {
+                                boolean needWal = needWalDeltaRecord(newRootId, newRootPage, newRootPageWalPlc);
 
-                                byte[] moveUpRowBytes = inner(io).initNewRoot(newRootPageAddr,
+                                byte[] moveUpRowBytes = inner(io).initNewRoot(newRootAddr,
                                     newRootId,
                                     pageId,
                                     moveUpRow,
@@ -2617,19 +2656,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                                     needWal);
 
                                 if (needWal)
-                                    wal.log(new NewRootInitRecord<>(cacheId, newRoot.id(), newRootId,
+                                    wal.log(new NewRootInitRecord<>(cacheId, newRootId, newRootId,
                                         inner(io), pageId, moveUpRowBytes, fwdId));
                             }
                             finally {
-                                writeUnlock(newRoot, newRootPageAddr, true);
+                                writeUnlock(newRootId, newRootPage, newRootAddr, newRootPageWalPlc, true);
                             }
                         }
+                        finally {
+                            releasePage(newRootId, newRootPage);
+                        }
 
-                        try (Page meta = page(metaPageId)) {
-                            Bool res = writePage(pageMem, meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
+                        Bool res = write(metaPageId, addRoot, newRootId, lvl + 1, FALSE);
 
-                            assert res == TRUE : res;
-                        }
+                        assert res == TRUE : res;
 
                         return null; // We've just moved link up to root, nothing to return here.
                     }
@@ -2638,20 +2678,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return moveUpRow;
                 }
                 finally {
-                    writeUnlock(fwd, fwdPageAddr, true);
+                    writeUnlock(fwdId, fwdPage, fwdPageAddr, fwdPageWalPlc, true);
                 }
             }
+            finally {
+                releasePage(fwdId, fwdPage);
+            }
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result tryReplaceInner(Page page, long pageId, long fwdId, int lvl)
+        private Result tryReplaceInner(long pageId, long page, long fwdId, int lvl)
             throws IgniteCheckedException {
             // Need to replace key in inner page. There is no race because we keep tail lock after split.
             if (needReplaceInner == TRUE) {
@@ -2664,7 +2707,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 this.fwdId = fwdId;
                 this.pageId = pageId;
 
-                Result res = writePage(pageMem, page, BPlusTree.this, replace, this, lvl, RETRY);
+                Result res = write(pageId, page, replace, this, lvl, RETRY);
 
                 // Restore args.
                 this.pageId = oldPageId;
@@ -2682,35 +2725,35 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result tryInsert(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        private Result tryInsert(long pageId, long page, long fwdId, int lvl) throws IgniteCheckedException {
             // Init args.
             this.pageId = pageId;
             this.fwdId = fwdId;
 
-            return writePage(pageMem, page, BPlusTree.this, insert, this, lvl, RETRY);
+            return write(pageId, page, insert, this, lvl, RETRY);
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        public Result tryReplace(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        public Result tryReplace(long pageId, long page, long fwdId, int lvl) throws IgniteCheckedException {
             // Init args.
             this.pageId = pageId;
             this.fwdId = fwdId;
 
-            return writePage(pageMem, page, BPlusTree.this, replace, this, lvl, RETRY);
+            return write(pageId, page, replace, this, lvl, RETRY);
         }
     }
 
@@ -2855,14 +2898,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean canRelease(Page page, int lvl) {
-            if (page == null)
+        @Override boolean canRelease(long pageId, int lvl) {
+            if (pageId == 0L)
                 return false;
 
             if (op == null)
                 return true;
 
-            return op.canRelease(page, lvl);
+            return op.canRelease(pageId, lvl);
         }
 
         /**
@@ -2892,11 +2935,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          */
         private void levelExit() {
             if (isRemove())
-                ((Remove)op).page = null;
+                ((Remove)op).page = 0L;
         }
 
         /**
          * Release all the resources by the end of operation.
+         * @throws IgniteCheckedException if failed.
          */
         private void releaseAll() throws IgniteCheckedException {
             if (isRemove())
@@ -2904,14 +2948,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result onNotFound(Page page, long pageId, long fwdId, int lvl)
+        private Result onNotFound(long pageId, long page, long fwdId, int lvl)
             throws IgniteCheckedException {
             if (op == null)
                 return NOT_FOUND;
@@ -2924,27 +2968,27 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return NOT_FOUND;
             }
 
-            return ((Put)op).tryInsert(page, pageId, fwdId, lvl);
+            return ((Put)op).tryInsert(pageId, page, fwdId, lvl);
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result onFound(Page page, long pageId, long backId, long fwdId, int lvl)
+        private Result onFound(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             if (op == null)
                 return FOUND;
 
             if (isRemove())
-                return ((Remove)op).tryRemoveFromLeaf(page, pageId, backId, fwdId, lvl);
+                return ((Remove)op).tryRemoveFromLeaf(pageId, page, backId, fwdId, lvl);
 
-            return  ((Put)op).tryReplace(page, pageId, fwdId, lvl);
+            return  ((Put)op).tryReplace(pageId, page, fwdId, lvl);
         }
 
         /**
@@ -2979,32 +3023,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        Result tryReplaceInner(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        Result tryReplaceInner(long pageId, long page, long fwdId, int lvl) throws IgniteCheckedException {
             if (!isPut())
                 return NOT_FOUND;
 
-            return ((Put)op).tryReplaceInner(page, pageId, fwdId, lvl);
+            return ((Put)op).tryReplaceInner(pageId, page, fwdId, lvl);
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        public Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
+        public Result finishOrLockTail(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
-            return ((Remove)op).finishOrLockTail(page, pageId, backId, fwdId, lvl);
+            return ((Remove)op).finishOrLockTail(pageId, page, backId, fwdId, lvl);
         }
     }
 
@@ -3024,8 +3068,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** Removed row. */
         T rmvd;
 
-        /** Current page. */
-        Page page;
+        /** Current page absolute pointer. */
+        long page;
 
         /** */
         Object freePages;
@@ -3107,6 +3151,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @throws IgniteCheckedException If failed.
+         * @return Tail to release if an empty branch was not merged.
          */
         private Tail<L> mergeEmptyBranch() throws IgniteCheckedException {
             assert needMergeEmptyBranch == TRUE: needMergeEmptyBranch;
@@ -3301,7 +3346,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     if (tail.getCount() == 0 && tail.lvl != 0 && getRootLevel() == tail.lvl) {
                         // Free root if it became empty after merge.
                         cutRoot(tail.lvl);
-                        freePage(tail.page, tail.buf, false);
+                        freePage(tail.id, tail.page, tail.buf, tail.walPlc, false);
 
                         // Exit: we are done.
                     }
@@ -3336,21 +3381,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             Tail<L> leaf = getTail(t, 0);
 
-            removeDataRowFromLeaf(leaf.page, leaf.io, leaf.buf, leaf.getCount(), insertionPoint(leaf));
+            removeDataRowFromLeaf(leaf.id, leaf.page, leaf.buf, leaf.walPlc, leaf.io, leaf.getCount(), insertionPoint(leaf));
         }
 
         /**
          * @param leafId Leaf page ID.
-         * @param leaf Leaf page.
+         * @param leafPage Leaf page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        private Result removeFromLeaf(long leafId, Page leaf, long backId, long fwdId) throws IgniteCheckedException {
+        private Result removeFromLeaf(long leafId, long leafPage, long backId, long fwdId) throws IgniteCheckedException {
             // Init parameters.
             pageId = leafId;
-            page = leaf;
+            page = leafPage;
             this.backId = backId;
             this.fwdId = fwdId;
 
@@ -3359,14 +3404,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return doRemoveFromLeaf();
 
             // Lock back page before the remove, we'll need it for merges.
-            Page back = page(backId);
+            long backPage = acquirePage(backId);
 
             try {
-                return writePage(pageMem, back, BPlusTree.this, lockBackAndRmvFromLeaf, this, 0, RETRY);
+                return write(backId, backPage, lockBackAndRmvFromLeaf, this, 0, RETRY);
             }
             finally {
-                if (canRelease(back, 0))
-                    back.close();
+                if (canRelease(backId, 0))
+                    releasePage(backId, backPage);
             }
         }
 
@@ -3375,9 +3420,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private Result doRemoveFromLeaf() throws IgniteCheckedException {
-            assert page != null;
+            assert page != 0L;
 
-            return writePage(pageMem, page, BPlusTree.this, rmvFromLeaf, this, 0, RETRY);
+            return write(pageId, page, rmvFromLeaf, this, 0, RETRY);
         }
 
         /**
@@ -3386,21 +3431,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private Result doLockTail(int lvl) throws IgniteCheckedException {
-            assert page != null;
+            assert page != 0L;
 
-            return writePage(pageMem, page, BPlusTree.this, lockTail, this, lvl, RETRY);
+            return write(pageId, page, lockTail, this, lvl, RETRY);
         }
 
         /**
          * @param pageId Page ID.
-         * @param page Page.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Expected forward page ID.
          * @param lvl Level.
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        private Result lockTail(long pageId, Page page, long backId, long fwdId, int lvl)
+        private Result lockTail(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             assert tail != null;
 
@@ -3413,14 +3458,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (backId == 0) // Back page ID is provided only when the last move was to the right.
                 return doLockTail(lvl);
 
-            Page back = page(backId);
+            long backPage = acquirePage(backId);
 
             try {
-                return writePage(pageMem, back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
+                return write(backId, backPage, lockBackAndTail, this, lvl, RETRY);
             }
             finally {
-                if (canRelease(back, lvl))
-                    back.close();
+                if (canRelease(backId, lvl))
+                    releasePage(backId, backPage);
             }
         }
 
@@ -3433,28 +3478,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             assert fwdId != 0: fwdId;
             assert backId == 0: backId;
 
-            Page fwd = page(fwdId);
+            long fwdId = this.fwdId;
+            long fwdPage = acquirePage(fwdId);
 
             try {
-                return writePage(pageMem, fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
+                return write(fwdId, fwdPage, lockTailForward, this, lvl, RETRY);
             }
             finally {
                 // If we were not able to lock forward page as tail, release the page.
-                if (canRelease(fwd, lvl))
-                    fwd.close();
+                if (canRelease(fwdId, lvl))
+                    releasePage(fwdId, fwdPage);
             }
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param walPlc Full page WAL record policy.
+         * @param io IO.
          * @param cnt Count.
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
         @SuppressWarnings("unchecked")
-        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, long pageAddr, int cnt, int idx)
+        private void removeDataRowFromLeaf(long pageId, long page, long pageAddr, Boolean walPlc, BPlusIO<L> io, int cnt,
+            int idx)
             throws IgniteCheckedException {
             assert idx >= 0 && idx < cnt: idx;
             assert io.isLeaf(): "inner";
@@ -3463,28 +3512,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             // Detach the row.
             rmvd = needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
 
-            doRemove(page, io, pageAddr, cnt, idx);
+            doRemove(pageId, page, pageAddr, walPlc, io, cnt, idx);
 
             assert isRemoved();
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         *
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param walPlc Full page WAL record policy.
+         * @param io IO.
          * @param cnt Count.
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void doRemove(Page page, BPlusIO<L> io, long pageAddr, int cnt, int idx)
+        private void doRemove(long pageId, long page, long pageAddr, Boolean walPlc, BPlusIO<L> io, int cnt,
+            int idx)
             throws IgniteCheckedException {
             assert cnt > 0 : cnt;
             assert idx >= 0 && idx < cnt : idx + " " + cnt;
 
             io.remove(pageAddr, idx, cnt);
 
-            if (needWalDeltaRecord(page))
-                wal.log(new RemoveRecord(cacheId, page.id(), idx, cnt));
+            if (needWalDeltaRecord(pageId, page, walPlc))
+                wal.log(new RemoveRecord(cacheId, pageId, idx, cnt));
         }
 
         /**
@@ -3524,7 +3577,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Tail<L> left = t.getLeftChild();
             Tail<L> right = t.getRightChild();
 
-            assert left.page.id() != right.page.id();
+            assert left.id != right.id;
 
             int cnt = t.getCount();
 
@@ -3575,7 +3628,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (right && cnt != 0)
                 idx++;
 
-            return inner(prnt.io).getLeft(prnt.buf, idx) == child.page.id();
+            return inner(prnt.io).getLeft(prnt.buf, idx) == child.id;
         }
 
         /**
@@ -3588,8 +3641,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private boolean checkChildren(Tail<L> prnt, Tail<L> left, Tail<L> right, int idx) {
             assert idx >= 0 && idx < prnt.getCount(): idx;
 
-            return inner(prnt.io).getLeft(prnt.buf, idx) == left.page.id() &&
-                inner(prnt.io).getRight(prnt.buf, idx) == right.page.id();
+            return inner(prnt.io).getLeft(prnt.buf, idx) == left.id &&
+                inner(prnt.io).getRight(prnt.buf, idx) == right.id;
         }
 
         /**
@@ -3602,7 +3655,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private boolean doMerge(Tail<L> prnt, Tail<L> left, Tail<L> right)
             throws IgniteCheckedException {
             assert right.io == left.io; // Otherwise incompatible.
-            assert left.io.getForward(left.buf) == right.page.id();
+            assert left.io.getForward(left.buf) == right.id;
 
             int prntCnt = prnt.getCount();
             int prntIdx = fix(insertionPoint(prnt));
@@ -3630,39 +3683,40 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             left.idx = Short.MIN_VALUE;
 
             // TODO GG-11640 log a correct merge record.
-            left.page.fullPageWalRecordPolicy(Boolean.TRUE);
+            left.walPlc = Boolean.TRUE;
 
             // Remove split key from parent. If we are merging empty branch then remove only on the top iteration.
             if (needMergeEmptyBranch != READY)
-                doRemove(prnt.page, prnt.io, prnt.buf, prntCnt, prntIdx);
+                doRemove(prnt.id, prnt.page, prnt.buf, prnt.walPlc, prnt.io, prntCnt, prntIdx);
 
             // Forward page is now empty and has no links, can free and release it right away.
-            freePage(right.page, right.buf, true);
+            freePage(right.id, right.page, right.buf, right.walPlc, true);
 
             return true;
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param walPlc Full page WAL record policy.
          * @param release Release write lock and release page.
          * @throws IgniteCheckedException If failed.
          */
-        private void freePage(Page page, long pageAddr, boolean release)
+        private void freePage(long pageId, long page, long pageAddr, Boolean walPlc, boolean release)
             throws IgniteCheckedException {
-            long pageId = page.id();
 
             long effectivePageId = PageIdUtils.effectivePageId(pageId);
 
-            pageId = recyclePage(pageId, page, pageAddr);
+            long recycled = recyclePage(pageId, page, pageAddr, walPlc);
 
             if (effectivePageId != PageIdUtils.effectivePageId(pageId))
                 throw new IllegalStateException("Effective page ID must stay the same.");
 
             if (release)
-                writeUnlockAndClose(page, pageAddr);
+                writeUnlockAndClose(pageId, page, pageAddr, walPlc);
 
-            addFreePage(pageId);
+            addFreePage(recycled);
         }
 
         /**
@@ -3670,11 +3724,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private void cutRoot(int lvl) throws IgniteCheckedException {
-            try (Page meta = page(metaPageId)) {
-                Bool res = writePage(pageMem, meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
+                Bool res = write(metaPageId, cutRoot, lvl, FALSE);
 
                 assert res == TRUE : res;
-            }
         }
 
         /**
@@ -3732,13 +3784,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             inner.io.setRemoveId(inner.buf, rmvId);
 
             // TODO GG-11640 log a correct inner replace record.
-            inner.page.fullPageWalRecordPolicy(Boolean.TRUE);
+            inner.walPlc = Boolean.TRUE;
 
             // Update remove ID for the leaf page.
             leaf.io.setRemoveId(leaf.buf, rmvId);
 
-            if (needWalDeltaRecord(leaf.page))
-                wal.log(new FixRemoveId(cacheId, leaf.page.id(), rmvId));
+            if (needWalDeltaRecord(leaf.id, leaf.page, leaf.walPlc))
+                wal.log(new FixRemoveId(cacheId, leaf.id, rmvId));
         }
 
         /**
@@ -3796,20 +3848,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          */
         private void doReleaseTail(Tail<L> t) {
             while (t != null) {
-                writeUnlockAndClose(t.page, t.buf);
+                writeUnlockAndClose(t.id, t.page, t.buf, t.walPlc);
 
                 Tail<L> s = t.sibling;
 
                 if (s != null)
-                    writeUnlockAndClose(s.page, s.buf);
+                    writeUnlockAndClose(s.id, s.page, s.buf, s.walPlc);
 
                 t = t.down;
             }
         }
 
         /** {@inheritDoc} */
-        @Override boolean canRelease(Page page, int lvl) {
-            return page != null && !isTail(page.id(), lvl);
+        @Override boolean canRelease(long pageId, int lvl) {
+            return pageId != 0L && !isTail(pageId, lvl);
         }
 
         /**
@@ -3825,12 +3877,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return false;
 
                 if (t.lvl == lvl) {
-                    if (t.page.id() == pageId)
+                    if (t.id == pageId)
                         return true;
 
                     t = t.sibling;
 
-                    return t != null && t.page.id() == pageId;
+                    return t != null && t.id == pageId;
                 }
 
                 t = t.down;
@@ -3840,15 +3892,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
          * @param io IO.
          * @param lvl Level.
          * @param type Type.
          * @return Added tail.
          */
-        private Tail<L> addTail(Page page, long pageAddr, BPlusIO<L> io, int lvl, byte type) {
-            final Tail<L> t = new Tail<>(page, pageAddr, io, type, lvl);
+        private Tail<L> addTail(long pageId, long page, long pageAddr, BPlusIO<L> io, int lvl, byte type) {
+            final Tail<L> t = new Tail<>(pageId, page, pageAddr, io, type, lvl);
 
             if (tail == null)
                 tail = t;
@@ -3946,15 +3999,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
+        private Result finishOrLockTail(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             Result res = finishTail();
 
@@ -3965,15 +4018,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result tryRemoveFromLeaf(Page page, long pageId, long backId, long fwdId, int lvl)
+        private Result tryRemoveFromLeaf(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             // We must be at the bottom here, just need to remove row from the current page.
             assert lvl == 0 : lvl;
@@ -4001,12 +4054,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         static final byte FORWARD = 2;
 
         /** */
-        private final Page page;
+        private final long id;
+
+        /** */
+        private final long page;
 
         /** */
         private final long buf;
 
         /** */
+        private Boolean walPlc;
+
+        /** */
         private final BPlusIO<L> io;
 
         /** */
@@ -4025,17 +4084,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Tail<L> down;
 
         /**
-         * @param page Write locked page.
+         * @param pageId Page ID.
+         * @param page Page absolute pointer.
          * @param buf Buffer.
          * @param io IO.
          * @param type Type.
          *

<TRUNCATED>

[5/8] ignite git commit: GC pressure

Posted by sb...@apache.org.
GC pressure


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

Branch: refs/heads/ignite-4811-no-start-ver
Commit: 682f3bed2e8c7dee4f13d11c8fc5d3cf856934c5
Parents: 37eed34
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Tue Mar 28 19:28:20 2017 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Tue Mar 28 19:28:20 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/PageSupport.java    |  6 ++--
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 18 +++++-----
 .../cache/database/freelist/FreeListImpl.java   |  4 +--
 .../cache/database/freelist/PagesList.java      | 38 ++++++++++----------
 .../cache/database/tree/BPlusTree.java          | 10 +++---
 .../cache/database/tree/util/PageHandler.java   | 24 ++++---------
 .../processors/query/h2/database/H2Tree.java    |  1 +
 7 files changed, 47 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/682f3bed/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
index 7d1f711..8076f28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
@@ -66,7 +66,8 @@ public interface PageSupport {
      * @param cacheId Cache ID.
      * @param pageId Page ID.
      * @param page Page pointer.
-     * @return ByteBuffer for modifying the page.
+     * @return Address of a buffer with contents of the given page or
+     *            {@code 0L} if attempt to take the write lock failed.
      */
     public long writeLock(int cacheId, long pageId, long page);
 
@@ -75,7 +76,8 @@ public interface PageSupport {
      * @param cacheId Cache ID.
      * @param pageId Page ID.
      * @param page Page pointer.
-     * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
+     * @return Address of a buffer with contents of the given page or
+     *            {@code 0L} if attempt to take the write lock failed.
      */
     public long tryWriteLock(int cacheId, long pageId, long page);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/682f3bed/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 7afd5bd..0b65ce6 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
@@ -66,7 +66,7 @@ import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
 @SuppressWarnings({"LockAcquiredButNotSafelyReleased", "FieldAccessedSynchronizedAndUnsynchronized"})
 public class PageMemoryNoStoreImpl implements PageMemory {
     /** */
-    private static final long PAGE_MARKER = 0xBEEAAFDEADBEEF01L;
+    public static final long PAGE_MARKER = 0xBEEAAFDEADBEEF01L;
 
     /** Full relative pointer mask. */
     private static final long RELATIVE_PTR_MASK = 0xFFFFFFFFFFFFFFL;
@@ -84,16 +84,16 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     private static final long COUNTER_INC = ADDRESS_MASK + 1;
 
     /** Page ID offset. */
-    private static final int PAGE_ID_OFFSET = 8;
+    public static final int PAGE_ID_OFFSET = 8;
 
     /** Page pin counter offset. */
-    private static final int LOCK_OFFSET = 16;
+    public static final int LOCK_OFFSET = 16;
 
     /**
      * Need a 8-byte pointer for linked list, 8 bytes for internal needs (flags),
      * 4 bytes cache ID, 8 bytes timestamp.
      */
-    static final int PAGE_OVERHEAD = LOCK_OFFSET + OffheapReadWriteLock.LOCK_SIZE;
+    public static final int PAGE_OVERHEAD = LOCK_OFFSET + OffheapReadWriteLock.LOCK_SIZE;
 
     /** Page size. */
     private int sysPageSize;
@@ -386,7 +386,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
         Segment seg = segment(pageIdx);
 
-        return seg.acquire(pageIdx);
+        return seg.acquirePage(pageIdx);
     }
 
     /** {@inheritDoc} */
@@ -505,7 +505,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
          * @param pageIdx Page index.
          * @return Page absolute pointer.
          */
-        private long acquire(int pageIdx) {
+        private long acquirePage(int pageIdx) {
             long absPtr = absolute(pageIdx);
 
             assert absPtr % 8 == 0 : absPtr;
@@ -589,12 +589,12 @@ public class PageMemoryNoStoreImpl implements PageMemory {
                 long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
 
                 if (freePageRelPtr != INVALID_REL_PTR) {
-                    long freePage = absolute(PageIdUtils.pageIndex(freePageRelPtr));
+                    long freePageAbsPtr = absolute(PageIdUtils.pageIndex(freePageRelPtr));
 
-                    long nextFreePageRelPtr = GridUnsafe.getLong(freePage) & ADDRESS_MASK;
+                    long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
 
                     if (GridUnsafe.compareAndSwapLong(null, freePageListPtr, freePageRelPtrMasked, nextFreePageRelPtr | cnt)) {
-                        GridUnsafe.putLong(freePage, PAGE_MARKER);
+                        GridUnsafe.putLong(freePageAbsPtr, PAGE_MARKER);
 
                         allocatedPages.incrementAndGet();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/682f3bed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
index 4d3270c..8ca95f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
@@ -400,7 +400,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
      * @return Page ID.
      * @throws IgniteCheckedException If failed.
      */
-    private long allocate(int part) throws IgniteCheckedException {
+    private long allocateDataPage(int part) throws IgniteCheckedException {
         assert part <= PageIdAllocator.MAX_PARTITION_ID;
         assert part != PageIdAllocator.INDEX_PARTITION;
 
@@ -438,7 +438,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
             boolean allocated = pageId == 0L;
 
             if(allocated)
-                pageId = allocate(row.partition());
+                pageId = allocateDataPage(row.partition());
 
             DataPageIO init = reuseBucket || allocated ? DataPageIO.VERSIONS.latest() : null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/682f3bed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
index 6c51096..4cf38ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
@@ -257,12 +257,12 @@ public abstract class PagesList extends DataStructure {
         assert metaPageId != 0;
 
         long curId = 0L;
-        long cur = 0L;
+        long curPage = 0L;
         long curAddr = 0L;
 
         PagesListMetaIO curIo = null;
 
-        long nextId = metaPageId;
+        long nextPageId = metaPageId;
 
         try {
             for (int bucket = 0; bucket < buckets; bucket++) {
@@ -275,36 +275,36 @@ public abstract class PagesList extends DataStructure {
                         int written = curAddr != 0L ? curIo.addTails(pageMem.pageSize(), curAddr, bucket, tails, tailIdx) : 0;
 
                         if (written == 0) {
-                            if (nextId == 0L) {
-                                nextId = allocatePageNoReuse();
+                            if (nextPageId == 0L) {
+                                nextPageId = allocatePageNoReuse();
 
                                 if (curAddr != 0L) {
-                                    curIo.setNextMetaPageId(curAddr, nextId);
+                                    curIo.setNextMetaPageId(curAddr, nextPageId);
 
-                                    releaseAndClose(curId, cur, curAddr);
+                                    releaseAndClose(curId, curPage, curAddr);
                                 }
 
-                                curId = nextId;
-                                cur = acquirePage(curId);
-                                curAddr = writeLock(curId, cur);
+                                curId = nextPageId;
+                                curPage = acquirePage(curId);
+                                curAddr = writeLock(curId, curPage);
 
                                 curIo = PagesListMetaIO.VERSIONS.latest();
 
                                 curIo.initNewPage(curAddr, curId, pageSize());
                             }
                             else {
-                                releaseAndClose(curId, cur, curAddr);
+                                releaseAndClose(curId, curPage, curAddr);
 
-                                curId = nextId;
-                                cur = acquirePage(curId);
-                                curAddr = writeLock(curId, cur);
+                                curId = nextPageId;
+                                curPage = acquirePage(curId);
+                                curAddr = writeLock(curId, curPage);
 
                                 curIo = PagesListMetaIO.VERSIONS.forPage(curAddr);
 
                                 curIo.resetCount(curAddr);
                             }
 
-                            nextId = curIo.getNextMetaPageId(curAddr);
+                            nextPageId = curIo.getNextMetaPageId(curAddr);
                         }
                         else
                             tailIdx += written;
@@ -313,11 +313,11 @@ public abstract class PagesList extends DataStructure {
             }
         }
         finally {
-            releaseAndClose(curId, cur, curAddr);
+            releaseAndClose(curId, curPage, curAddr);
         }
 
-        while (nextId != 0L) {
-            long pageId = nextId;
+        while (nextPageId != 0L) {
+            long pageId = nextPageId;
 
             long page = acquirePage(pageId);
             try {
@@ -331,7 +331,7 @@ public abstract class PagesList extends DataStructure {
                     if (needWalDeltaRecord(pageId, page, null))
                         wal.log(new PageListMetaResetCountRecord(cacheId, pageId));
 
-                    nextId = io.getNextMetaPageId(pageAddr);
+                    nextPageId = io.getNextMetaPageId(pageAddr);
                 }
                 finally {
                     writeUnlock(pageId, page, pageAddr, true);
@@ -350,7 +350,7 @@ public abstract class PagesList extends DataStructure {
      * @throws IgniteCheckedException If failed.
      */
     private void releaseAndClose(long pageId, long page, long pageAddr) throws IgniteCheckedException {
-        if (pageAddr != 0L) {
+        if (pageAddr != 0L && page != 0L) {
             try {
                 // No special WAL record because we most likely changed the whole page.
                 writeUnlock(pageId, page, pageAddr, TRUE, true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/682f3bed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 5a8c49c..7be8297 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -1835,10 +1835,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long cnt = 0;
 
         while (pageId != 0) {
-            long pageId0 = pageId;
-            long page = acquirePage(pageId0);
+            long curId = pageId;
+            long curPage = acquirePage(curId);
             try {
-                long curAddr = readLock(pageId0, page); // No correctness guaranties.
+                long curAddr = readLock(curId, curPage); // No correctness guaranties.
 
                 try {
                     if (io == null) {
@@ -1852,11 +1852,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     pageId = io.getForward(curAddr);
                 }
                 finally {
-                    readUnlock(pageId0, page, curAddr);
+                    readUnlock(curId, curPage, curAddr);
                 }
             }
             finally {
-                releasePage(pageId0, page);
+                releasePage(curId, curPage);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/682f3bed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
index f660686..0ca2abc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
@@ -176,15 +176,11 @@ public abstract class PageHandler<X, R> {
         long pageId,
         long page,
         PageLockListener lsnr) {
-        boolean notifyLsnr = lsnr != null;
-
-        if(notifyLsnr)
-            lsnr.onBeforeReadLock(cacheId, pageId, page);
+        lsnr.onBeforeReadLock(cacheId, pageId, page);
 
         long pageAddr = pageMem.readLock(cacheId, pageId, page);
 
-        if(notifyLsnr)
-            lsnr.onReadLock(cacheId, pageId, page, pageAddr);
+        lsnr.onReadLock(cacheId, pageId, page, pageAddr);
 
         return pageAddr;
     }
@@ -204,8 +200,7 @@ public abstract class PageHandler<X, R> {
         long page,
         long pageAddr,
         PageLockListener lsnr) {
-        if(lsnr != null)
-            lsnr.onReadUnlock(cacheId, pageId, page, pageAddr);
+        lsnr.onReadUnlock(cacheId, pageId, page, pageAddr);
 
         pageMem.readUnlock(cacheId, pageId, page);
     }
@@ -294,7 +289,7 @@ public abstract class PageHandler<X, R> {
             }
         }
         finally {
-            if(releaseAfterWrite)
+            if (releaseAfterWrite)
                 pageMem.releasePage(cacheId, pageId, page);
         }
     }
@@ -379,8 +374,7 @@ public abstract class PageHandler<X, R> {
         PageLockListener lsnr,
         Boolean walPlc,
         boolean dirty) {
-        if(lsnr != null)
-            lsnr.onWriteUnlock(cacheId, pageId, page, pageAddr);
+        lsnr.onWriteUnlock(cacheId, pageId, page, pageAddr);
 
         pageMem.writeUnlock(cacheId, pageId, page, walPlc, dirty);
     }
@@ -401,15 +395,11 @@ public abstract class PageHandler<X, R> {
         long page,
         PageLockListener lsnr,
         boolean tryLock) {
-        boolean notifyLsnr = lsnr != null;
-
-        if(notifyLsnr)
-            lsnr.onBeforeWriteLock(cacheId, pageId, page);
+        lsnr.onBeforeWriteLock(cacheId, pageId, page);
 
         long pageAddr = tryLock ? pageMem.tryWriteLock(cacheId, pageId, page) : pageMem.writeLock(cacheId, pageId, page);
 
-        if(notifyLsnr)
-            lsnr.onWriteLock(cacheId, pageId, page, pageAddr);
+        lsnr.onWriteLock(cacheId, pageId, page, pageAddr);
 
         return pageAddr;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/682f3bed/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
index d7f5b29..f673717 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
@@ -136,6 +136,7 @@ public abstract class H2Tree extends BPlusTree<SearchRow, GridH2Row> {
      */
     private int getMetaInlineSize() throws IgniteCheckedException {
         final long metaPage = acquirePage(metaPageId);
+
         try {
             long pageAddr = readLock(metaPageId, metaPage); // Meta can't be removed.
 


[7/8] ignite git commit: Merge branch 'ignite-3477-master' into ignite-4811

Posted by sb...@apache.org.
Merge branch 'ignite-3477-master' into ignite-4811


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

Branch: refs/heads/ignite-4811-no-start-ver
Commit: 7215c25bb9e5bca141fac374645f38b8836bc49e
Parents: e34222b d758a82
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Fri Mar 31 15:38:53 2017 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Fri Mar 31 15:38:53 2017 +0300

----------------------------------------------------------------------
 .../computegrid/ComputeAsyncExample.java        |   8 +-
 .../ComputeFibonacciContinuationExample.java    |  13 +-
 .../examples/datagrid/CacheAsyncApiExample.java |  17 +-
 .../datastructures/IgniteLockExample.java       |   4 +-
 .../datastructures/IgniteSemaphoreExample.java  |   4 +-
 .../examples/ScalarContinuationExample.scala    |  10 +-
 .../singlesplit/SingleSplitsLoadTest.java       |   8 +-
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java |   6 +-
 ...3IpFinderAwsCredentialsProviderSelfTest.java |   2 +-
 .../ignite/tests/utils/TestTransaction.java     |  11 +
 modules/clients/pom.xml                         |   7 +
 .../src/test/config/jdbc-config-cache-store.xml | 124 +++
 .../internal/client/ClientStartNodeTask.java    |  10 +-
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |   3 +
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |  28 +-
 .../internal/jdbc2/JdbcSpringSelfTest.java      | 125 +++
 .../jdbc/AbstractJdbcPojoQuerySelfTest.java     | 169 ++++
 .../jdbc/JdbcPojoLegacyQuerySelfTest.java       |  44 +
 .../ignite/jdbc/JdbcPojoQuerySelfTest.java      |  56 ++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   5 +
 .../java/org/apache/ignite/IgniteCache.java     | 611 +++++++++++-
 .../java/org/apache/ignite/IgniteCluster.java   | 126 +++
 .../java/org/apache/ignite/IgniteCompute.java   | 262 ++++-
 .../java/org/apache/ignite/IgniteEvents.java    | 128 ++-
 .../org/apache/ignite/IgniteFileSystem.java     |  76 ++
 .../java/org/apache/ignite/IgniteMessaging.java |  27 +
 .../java/org/apache/ignite/IgniteServices.java  | 221 ++++-
 .../ignite/binary/BinaryObjectBuilder.java      |  10 -
 .../store/jdbc/CacheAbstractJdbcStore.java      |  24 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  18 +-
 .../apache/ignite/internal/GridComponent.java   |   5 +-
 .../ignite/internal/GridJobCancelRequest.java   |   2 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   2 +-
 .../ignite/internal/GridJobExecuteResponse.java |   2 +-
 .../ignite/internal/GridJobSiblingsRequest.java |   2 +-
 .../internal/GridJobSiblingsResponse.java       |   2 +-
 .../ignite/internal/GridTaskCancelRequest.java  |   2 +-
 .../ignite/internal/GridTaskSessionRequest.java |   2 +-
 .../org/apache/ignite/internal/GridTopic.java   |   5 +-
 .../ignite/internal/IgniteComputeImpl.java      | 546 +++++++++--
 .../ignite/internal/IgniteEventsImpl.java       |  79 ++
 .../ignite/internal/IgniteMessagingImpl.java    |  65 +-
 .../ignite/internal/IgniteServicesImpl.java     | 108 +++
 .../ignite/internal/MarshallerContextImpl.java  |   4 +-
 .../binary/BinaryCachingMetadataHandler.java    |   6 +
 .../internal/binary/BinaryClassDescriptor.java  |  33 +-
 .../ignite/internal/binary/BinaryContext.java   |  29 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |   2 +-
 .../internal/binary/BinaryFieldMetadata.java    | 127 +++
 .../ignite/internal/binary/BinaryMetadata.java  |  49 +-
 .../binary/BinaryMetadataCollector.java         |  17 +-
 .../internal/binary/BinaryMetadataHandler.java  |  18 +-
 .../binary/BinaryNoopMetadataHandler.java       |   5 +
 .../internal/binary/BinaryObjectExImpl.java     |   4 -
 .../internal/binary/BinaryObjectImpl.java       |   2 +-
 .../binary/BinaryObjectOffheapImpl.java         |   2 +-
 .../internal/binary/BinaryReaderExImpl.java     |   2 +-
 .../ignite/internal/binary/BinaryUtils.java     |  21 +-
 .../internal/binary/BinaryWriterExImpl.java     |  45 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  34 +-
 .../internal/cluster/ClusterGroupAdapter.java   |   2 +-
 .../cluster/IgniteClusterAsyncImpl.java         |  28 +-
 .../internal/cluster/IgniteClusterImpl.java     |  24 +-
 .../internal/direct/DirectMessageWriter.java    |   4 +-
 .../stream/v1/DirectByteBufferStreamImplV1.java |   8 +-
 .../stream/v2/DirectByteBufferStreamImplV2.java |   9 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |  19 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |   2 +-
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |   2 +-
 .../checkpoint/GridCheckpointRequest.java       |   2 +-
 .../managers/communication/GridIoMessage.java   |   2 +-
 .../communication/GridIoMessageFactory.java     |  23 +-
 .../communication/GridIoUserMessage.java        |   2 +-
 .../communication/IgniteIoTestMessage.java      |   2 +-
 .../deployment/GridDeploymentInfoBean.java      |   2 +-
 .../deployment/GridDeploymentRequest.java       |   2 +-
 .../deployment/GridDeploymentResponse.java      |   2 +-
 .../discovery/DiscoveryCustomMessage.java       |  48 +
 .../discovery/GridDiscoveryManager.java         |  38 +-
 .../eventstorage/GridEventStorageManager.java   |   6 +-
 .../eventstorage/GridEventStorageMessage.java   |   2 +-
 .../snapshot/SnapshotFinishedMessage.java       |   2 +-
 .../snapshot/SnapshotProgressMessage.java       |   2 +-
 .../affinity/AffinityTopologyVersion.java       |   2 +-
 .../cache/CacheAffinitySharedManager.java       |   4 -
 .../cache/CacheEntryInfoCollection.java         |   2 +-
 .../cache/CacheEntryPredicateAdapter.java       |   2 +-
 .../cache/CacheEntryPredicateContainsValue.java |   2 +-
 .../cache/CacheEntrySerializablePredicate.java  |   2 +-
 .../processors/cache/CacheEvictionEntry.java    |   2 +-
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../cache/CacheObjectByteArrayImpl.java         |   2 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheAdapter.java      | 187 +---
 .../processors/cache/GridCacheEntryInfo.java    |   2 +-
 .../cache/GridCacheEvictionRequest.java         |   2 +-
 .../cache/GridCacheEvictionResponse.java        |   2 +-
 .../GridCachePartitionExchangeManager.java      |  12 +-
 .../processors/cache/GridCacheReturn.java       |   2 +-
 .../processors/cache/GridCacheUtils.java        |   4 -
 .../GridChangeGlobalStateMessageResponse.java   |   2 +-
 .../processors/cache/IgniteCacheProxy.java      | 580 ++++++++++-
 .../processors/cache/KeyCacheObjectImpl.java    |   2 +-
 .../cache/binary/BinaryMetadataHolder.java      |  73 ++
 .../cache/binary/BinaryMetadataTransport.java   | 641 ++++++++++++
 .../binary/BinaryMetadataUpdatedListener.java   |  29 +
 .../binary/CacheObjectBinaryProcessor.java      |  12 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  | 504 +++-------
 .../binary/ClientMetadataRequestFuture.java     | 161 ++++
 .../cache/binary/MetadataRequestMessage.java    | 122 +++
 .../cache/binary/MetadataResponseMessage.java   | 195 ++++
 .../binary/MetadataUpdateAcceptedMessage.java   |  96 ++
 .../binary/MetadataUpdateProposedMessage.java   | 224 +++++
 .../cache/binary/MetadataUpdateResult.java      |  96 ++
 .../distributed/GridCacheTtlUpdateRequest.java  |   2 +-
 .../distributed/GridCacheTxRecoveryRequest.java |   2 +-
 .../GridCacheTxRecoveryResponse.java            |   2 +-
 .../distributed/GridDistributedLockRequest.java |   2 +-
 .../GridDistributedLockResponse.java            |   2 +-
 .../GridDistributedTxFinishRequest.java         |   2 +-
 .../GridDistributedTxFinishResponse.java        |   2 +-
 .../GridDistributedTxPrepareRequest.java        |   2 +-
 .../GridDistributedTxPrepareResponse.java       |   2 +-
 .../GridDistributedUnlockRequest.java           |   2 +-
 .../dht/GridClientPartitionTopology.java        |  46 +-
 .../dht/GridDhtAffinityAssignmentRequest.java   |   2 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  | 103 +-
 .../distributed/dht/GridDhtLockRequest.java     |   2 +-
 .../distributed/dht/GridDhtLockResponse.java    |   2 +-
 .../dht/GridDhtPartitionTopology.java           |  12 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  54 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |   2 +-
 .../dht/GridDhtTxFinishResponse.java            |   2 +-
 .../dht/GridDhtTxOnePhaseCommitAckRequest.java  |   2 +-
 .../dht/GridDhtTxPrepareRequest.java            |   2 +-
 .../dht/GridDhtTxPrepareResponse.java           |   2 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   2 +-
 .../dht/GridPartitionedGetFuture.java           |   8 +-
 .../dht/GridPartitionedSingleGetFuture.java     |  56 +-
 .../GridDhtAtomicDeferredUpdateResponse.java    |   2 +-
 .../dht/atomic/GridDhtAtomicNearResponse.java   |   2 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |   2 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   2 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   2 +-
 .../GridNearAtomicCheckUpdateRequest.java       |   2 +-
 .../atomic/GridNearAtomicFullUpdateRequest.java |   2 +-
 ...GridNearAtomicSingleUpdateFilterRequest.java |   2 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |   2 +-
 .../GridNearAtomicSingleUpdateRequest.java      |   2 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |   2 +-
 .../dht/atomic/NearCacheUpdates.java            |   2 +-
 .../distributed/dht/atomic/UpdateErrors.java    |   2 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |   2 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |   2 +-
 .../GridDhtPartitionDemandMessage.java          |   2 +-
 .../dht/preloader/GridDhtPartitionDemander.java | 501 +---------
 .../preloader/GridDhtPartitionExchangeId.java   |   2 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |  22 +-
 .../dht/preloader/GridDhtPartitionMap.java      | 325 +++++++
 .../dht/preloader/GridDhtPartitionMap2.java     | 327 -------
 .../GridDhtPartitionSupplyMessage.java          |   2 +-
 .../GridDhtPartitionSupplyMessageV2.java        |   2 +-
 .../GridDhtPartitionsAbstractMessage.java       |   4 -
 .../GridDhtPartitionsExchangeFuture.java        |   2 +-
 .../preloader/GridDhtPartitionsFullMessage.java |   8 +-
 .../GridDhtPartitionsSingleMessage.java         |  12 +-
 .../GridDhtPartitionsSingleRequest.java         |   2 +-
 .../dht/preloader/GridDhtPreloader.java         |  18 +-
 .../distributed/near/CacheVersionedValue.java   |   2 +-
 .../distributed/near/GridNearGetRequest.java    |   2 +-
 .../distributed/near/GridNearGetResponse.java   |   2 +-
 .../distributed/near/GridNearLockRequest.java   |   2 +-
 .../distributed/near/GridNearLockResponse.java  |   2 +-
 .../near/GridNearSingleGetRequest.java          |   2 +-
 .../near/GridNearSingleGetResponse.java         |   2 +-
 .../near/GridNearTxFinishRequest.java           |   2 +-
 .../near/GridNearTxFinishResponse.java          |   2 +-
 .../near/GridNearTxPrepareRequest.java          |   2 +-
 .../near/GridNearTxPrepareResponse.java         |   2 +-
 .../distributed/near/GridNearUnlockRequest.java |   2 +-
 .../query/GridCacheDistributedQueryManager.java |   7 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |   3 +-
 .../cache/query/GridCacheQueryRequest.java      |   2 +-
 .../cache/query/GridCacheQueryResponse.java     |   2 +-
 .../cache/query/GridCacheSqlQuery.java          |   2 +-
 .../CacheContinuousQueryBatchAck.java           |   6 +-
 .../continuous/CacheContinuousQueryEntry.java   |   2 +-
 .../continuous/CacheContinuousQueryHandler.java |   6 +-
 .../continuous/CacheContinuousQueryManager.java |  87 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |  35 +-
 .../cache/transactions/IgniteTxEntry.java       |   2 +-
 .../cache/transactions/IgniteTxKey.java         |   2 +-
 .../transactions/TransactionProxyImpl.java      |  39 +-
 .../cache/transactions/TxEntryValueHolder.java  |   2 +-
 .../processors/cache/transactions/TxLock.java   |   2 +-
 .../cache/transactions/TxLockList.java          |   2 +-
 .../cache/transactions/TxLocksRequest.java      |   2 +-
 .../cache/transactions/TxLocksResponse.java     |   2 +-
 .../version/GridCacheRawVersionedEntry.java     |   2 +-
 .../cache/version/GridCacheVersion.java         |   2 +-
 .../cache/version/GridCacheVersionEx.java       |   2 +-
 .../clock/GridClockDeltaSnapshotMessage.java    |   2 +-
 .../processors/clock/GridClockDeltaVersion.java |   2 +-
 .../closure/GridClosureProcessor.java           | 398 +-------
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../continuous/GridContinuousProcessor.java     |   3 -
 .../datastreamer/DataStreamerEntry.java         |   2 +-
 .../datastreamer/DataStreamerRequest.java       |   2 +-
 .../datastreamer/DataStreamerResponse.java      |   2 +-
 .../internal/processors/hadoop/HadoopJobId.java |   2 +-
 .../shuffle/HadoopDirectShuffleMessage.java     |   2 +-
 .../hadoop/shuffle/HadoopShuffleAck.java        |   2 +-
 .../shuffle/HadoopShuffleFinishRequest.java     |   2 +-
 .../shuffle/HadoopShuffleFinishResponse.java    |   2 +-
 .../hadoop/shuffle/HadoopShuffleMessage.java    |   2 +-
 .../processors/igfs/IgfsAckMessage.java         |   2 +-
 .../internal/processors/igfs/IgfsAsyncImpl.java |  43 +-
 .../internal/processors/igfs/IgfsBlockKey.java  |   3 +-
 .../processors/igfs/IgfsBlocksMessage.java      |   2 +-
 .../processors/igfs/IgfsDeleteMessage.java      |   2 +-
 .../processors/igfs/IgfsFileAffinityRange.java  |   2 +-
 .../igfs/IgfsFragmentizerRequest.java           |   2 +-
 .../igfs/IgfsFragmentizerResponse.java          |   2 +-
 .../internal/processors/igfs/IgfsImpl.java      |  62 +-
 .../processors/igfs/IgfsSyncMessage.java        |   2 +-
 .../GridMarshallerMappingProcessor.java         |   6 +-
 .../MissingMappingRequestMessage.java           |   2 +-
 .../MissingMappingResponseMessage.java          |   2 +-
 .../platform/PlatformAbstractTarget.java        |  24 +-
 .../platform/PlatformAsyncTarget.java           |  44 -
 .../platform/PlatformContextImpl.java           |  82 +-
 .../platform/PlatformTargetProxy.java           |  22 -
 .../platform/PlatformTargetProxyImpl.java       |  36 +-
 .../platform/cache/PlatformCache.java           | 218 +++--
 .../platform/compute/PlatformCompute.java       |  14 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  12 +-
 ...formDotNetEntityFrameworkCacheExtension.java |   8 +-
 .../platform/events/PlatformEvents.java         |  70 +-
 .../memory/PlatformOutputStreamImpl.java        |   8 +-
 .../platform/messaging/PlatformMessaging.java   |  35 +-
 .../platform/services/PlatformServices.java     |  95 +-
 .../transactions/PlatformTransactions.java      |   9 +-
 .../utils/PlatformConfigurationUtils.java       |   2 +
 .../processors/query/GridQueryProcessor.java    |  10 +-
 .../query/QueryTypeDescriptorImpl.java          |  17 -
 .../messages/GridQueryCancelRequest.java        |   2 +-
 .../twostep/messages/GridQueryFailResponse.java |   2 +-
 .../messages/GridQueryNextPageRequest.java      |   2 +-
 .../messages/GridQueryNextPageResponse.java     |   2 +-
 .../h2/twostep/messages/GridQueryRequest.java   | 368 -------
 .../handlers/task/GridTaskResultRequest.java    |   2 +-
 .../handlers/task/GridTaskResultResponse.java   |   2 +-
 .../service/GridServiceProcessor.java           | 137 +--
 .../processors/task/GridTaskProcessor.java      |   9 +-
 .../ignite/internal/util/GridByteArrayList.java |   2 +-
 .../ignite/internal/util/GridLongList.java      |   2 +-
 .../internal/util/GridMessageCollection.java    |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |  10 -
 .../internal/util/UUIDCollectionMessage.java    |   2 +-
 .../internal/util/nio/GridDirectParser.java     |  12 +-
 .../ignite/internal/visor/cache/VisorCache.java |  13 +-
 .../visor/cache/VisorCacheClearTask.java        |  14 +-
 .../visor/compute/VisorGatewayTask.java         |   6 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |  15 +-
 .../internal/visor/query/VisorQueryArg.java     |  27 +-
 .../internal/visor/query/VisorQueryJob.java     |   6 +-
 .../apache/ignite/lang/IgniteAsyncSupport.java  |  52 +-
 .../ignite/lang/IgniteAsyncSupported.java       |   4 +-
 .../extensions/communication/Message.java       |   5 +-
 .../communication/MessageFactory.java           |   2 +-
 .../extensions/communication/MessageWriter.java |   5 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   2 +-
 .../jobstealing/JobStealingRequest.java         |   2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 249 +++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 104 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   9 -
 .../messages/TcpDiscoveryClientAckResponse.java |   4 -
 .../apache/ignite/transactions/Transaction.java |  22 +
 .../resources/META-INF/classnames.properties    | 141 +--
 .../IgniteCacheExpiryStoreLoadSelfTest.java     |  18 +-
 .../internal/ClusterGroupAbstractTest.java      |  73 +-
 .../ComputeJobCancelWithServiceSelfTest.java    |   7 +-
 .../internal/GridAffinityNoCacheSelfTest.java   |   2 +-
 .../internal/GridCancelOnGridStopSelfTest.java  |   2 +-
 .../GridCancelledJobsMetricsSelfTest.java       |  11 +-
 .../internal/GridContinuousTaskSelfTest.java    |  21 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |  12 +-
 .../GridFailoverCustomTopologySelfTest.java     |   7 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |  93 +-
 .../internal/GridMultipleJobsSelfTest.java      |   8 +-
 .../ignite/internal/GridReduceSelfTest.java     |   9 +-
 .../GridTaskCancelSingleNodeSelfTest.java       |   7 +-
 .../internal/GridTaskExecutionSelfTest.java     |  58 +-
 ...xecutionWithoutPeerClassLoadingSelfTest.java |  31 +
 .../GridTaskFailoverAffinityRunTest.java        |   7 +-
 .../GridTaskInstanceExecutionSelfTest.java      |   7 +-
 .../internal/GridTaskJobRejectSelfTest.java     |   7 +-
 .../IgniteClientReconnectApiExceptionTest.java  |  10 +-
 .../IgniteComputeEmptyClusterGroupTest.java     |  26 +-
 .../binary/BinaryMarshallerSelfTest.java        |   4 -
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   9 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java | 127 +--
 .../binary/TestCachingMetadataHandler.java      |   5 +
 ...ByteBufferStreamImplV2ByteOrderSelfTest.java |   2 +-
 ...GridManagerLocalMessageListenerSelfTest.java |   2 +-
 .../GridCommunicationSendMessageSelfTest.java   |  79 +-
 .../communication/GridIoManagerSelfTest.java    |   2 +-
 .../cache/CacheConcurrentReadThroughTest.java   |   8 +-
 .../cache/CacheConfigurationLeakTest.java       |   1 +
 ...CacheExchangeMessageDuplicatedStateTest.java |  12 +-
 .../cache/CacheFutureExceptionSelfTest.java     |   6 +-
 .../CachePutEventListenerErrorSelfTest.java     |   7 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   6 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java | 105 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java | 876 ++++++++++++++---
 .../cache/GridCacheAbstractMetricsSelfTest.java |  52 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |   9 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |  10 +-
 .../GridCacheConditionalDeploymentSelfTest.java |   4 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |  31 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   6 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |  16 +-
 .../GridCacheReferenceCleanupSelfTest.java      |  15 +-
 ...ridCacheStoreManagerDeserializationTest.java |   1 -
 .../cache/GridCacheUtilsSelfTest.java           |   4 -
 ...calCacheStoreManagerDeserializationTest.java |   2 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |   7 +-
 .../IgniteCacheBinaryObjectsScanSelfTest.java   |   2 +-
 .../IgniteCacheConfigVariationsFullApiTest.java | 965 +++++++++++++++++--
 .../cache/IgniteCacheInvokeAbstractTest.java    |  20 +-
 .../IgniteCacheManyAsyncOperationsTest.java     |   6 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |  34 +-
 .../IgniteIncompleteCacheObjectSelfTest.java    |   2 +-
 .../cache/MemoryPolicyConfigValidationTest.java |   2 +-
 .../cache/WithKeepBinaryCacheFullApiTest.java   | 228 ++---
 .../binary/BinaryMetadataUpdatesFlowTest.java   | 592 ++++++++++++
 .../CacheKeepBinaryWithInterceptorTest.java     |  16 +-
 ...naryObjectMetadataExchangeMultinodeTest.java | 463 +++++++++
 .../GridCacheBinaryObjectsAbstractSelfTest.java | 111 +--
 ...eAbstractDataStructuresFailoverSelfTest.java |   7 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |  12 +-
 .../IgniteCountDownLatchAbstractSelfTest.java   |   7 +-
 .../IgniteLockAbstractSelfTest.java             |   7 +-
 .../IgniteSemaphoreAbstractSelfTest.java        |   7 +-
 ...acheAsyncOperationsFailoverAbstractTest.java |  12 +-
 .../distributed/CacheAsyncOperationsTest.java   |  32 +-
 .../CachePutAllFailoverAbstractTest.java        |   8 +-
 .../GridCacheAbstractJobExecutionTest.java      |  15 +-
 .../GridCacheBasicOpAbstractTest.java           |  38 +-
 .../distributed/GridCacheEventAbstractTest.java |  80 +-
 .../GridCacheMultiNodeAbstractTest.java         |  25 +-
 ...yMetadataUpdateChangingTopologySelfTest.java |  13 +-
 .../IgniteCacheConnectionRecoveryTest.java      |  10 +-
 ...eCacheMessageRecoveryIdleConnectionTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |  12 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../dht/GridCacheGlobalLoadTest.java            |  21 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |  12 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |  26 +-
 .../atomic/IgniteCacheAtomicProtocolTest.java   |  34 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |  11 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |   9 +-
 .../GridCacheRebalancingSyncSelfTest.java       |   7 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |  21 +-
 .../CacheContinuousQueryVariationsTest.java     |   5 +
 .../GridCacheContinuousQueryConcurrentTest.java |   5 +-
 .../closure/GridClosureProcessorSelfTest.java   |  74 +-
 ...ComputeJobExecutionErrorToLogManualTest.java |  10 +-
 ...gniteComputeConfigVariationsFullApiTest.java | 533 +++++++++-
 .../continuous/GridEventConsumeSelfTest.java    | 196 +++-
 .../database/FreeListImplSelfTest.java          |   2 +-
 .../internal/processors/igfs/IgfsMock.java      |  41 +
 .../processors/igfs/IgfsTaskSelfTest.java       |  19 +
 ...niteMessagingConfigVariationFullApiTest.java |  93 +-
 .../GridServiceProcessorAbstractSelfTest.java   | 291 +++++-
 .../GridServiceProcessorMultiNodeSelfTest.java  |  18 +-
 .../GridServiceProcessorStopSelfTest.java       |   7 +-
 .../nio/IgniteExceptionInNioWorkerSelfTest.java |   2 +-
 .../loadtest/GridSingleExecutionTest.java       |  10 +-
 .../loadtests/colocation/GridTestMain.java      |   7 +-
 .../communication/GridTestMessage.java          |   2 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |   7 +-
 ...ridSingleSplitsNewNodesAbstractLoadTest.java |   8 +-
 .../ignite/loadtests/dsi/GridDsiClient.java     |  12 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |   9 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |  10 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |   7 +-
 .../mergesort/GridMergeSortLoadTask.java        |   7 +-
 .../marshaller/MarshallerContextSelfTest.java   |  10 +-
 .../ignite/messaging/GridMessagingSelfTest.java |  73 +-
 .../messaging/IgniteMessagingSendAsyncTest.java |  83 +-
 ...idSessionFutureWaitJobAttributeSelfTest.java |   7 +-
 ...GridSessionSetJobAttributeOrderSelfTest.java |   8 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |   7 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |   7 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |   7 +-
 .../GridSessionWaitAttributeSelfTest.java       |   9 +-
 .../communication/GridCacheMessageSelfTest.java |  20 +-
 .../spi/communication/GridTestMessage.java      |   4 +-
 .../TcpDiscoverySpiFailureTimeoutSelfTest.java  |  60 --
 .../ignite/testframework/GridTestUtils.java     |  60 ++
 .../testframework/junits/GridAbstractTest.java  |  68 +-
 .../cache/GridAbstractCacheStoreSelfTest.java   |  11 +
 .../junits/common/GridCommonAbstractTest.java   |  30 +-
 .../multijvm/IgniteCacheProcessProxy.java       | 172 ++++
 .../multijvm/IgniteClusterProcessProxy.java     |  13 +
 .../multijvm/IgniteEventsProcessProxy.java      |  31 +
 .../IgniteBinaryObjectsTestSuite.java           |   4 +
 .../testsuites/IgniteComputeGridTestSuite.java  |   2 +
 .../ignite/util/GridMessageCollectionTest.java  |   5 +-
 .../query/h2/DmlStatementsProcessor.java        |  35 -
 .../processors/query/h2/IgniteH2Indexing.java   |  18 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  39 -
 .../h2/twostep/GridReduceQueryExecutor.java     |  60 +-
 .../query/h2/twostep/msg/GridH2Array.java       |   2 +-
 .../query/h2/twostep/msg/GridH2Boolean.java     |   2 +-
 .../query/h2/twostep/msg/GridH2Byte.java        |   2 +-
 .../query/h2/twostep/msg/GridH2Bytes.java       |   2 +-
 .../query/h2/twostep/msg/GridH2CacheObject.java |   2 +-
 .../query/h2/twostep/msg/GridH2Date.java        |   2 +-
 .../query/h2/twostep/msg/GridH2Decimal.java     |   2 +-
 .../query/h2/twostep/msg/GridH2Double.java      |   2 +-
 .../query/h2/twostep/msg/GridH2Float.java       |   2 +-
 .../query/h2/twostep/msg/GridH2Geometry.java    |   2 +-
 .../h2/twostep/msg/GridH2IndexRangeRequest.java |   2 +-
 .../twostep/msg/GridH2IndexRangeResponse.java   |   2 +-
 .../query/h2/twostep/msg/GridH2Integer.java     |   2 +-
 .../query/h2/twostep/msg/GridH2JavaObject.java  |   2 +-
 .../query/h2/twostep/msg/GridH2Long.java        |   2 +-
 .../query/h2/twostep/msg/GridH2Null.java        |   2 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |   2 +-
 .../query/h2/twostep/msg/GridH2RowMessage.java  |   2 +-
 .../query/h2/twostep/msg/GridH2RowRange.java    |   2 +-
 .../h2/twostep/msg/GridH2RowRangeBounds.java    |   2 +-
 .../query/h2/twostep/msg/GridH2Short.java       |   2 +-
 .../query/h2/twostep/msg/GridH2String.java      |   2 +-
 .../query/h2/twostep/msg/GridH2Time.java        |   2 +-
 .../query/h2/twostep/msg/GridH2Timestamp.java   |   2 +-
 .../query/h2/twostep/msg/GridH2Uuid.java        |   2 +-
 .../twostep/msg/GridH2ValueMessageFactory.java  |   2 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |   4 +
 ...niteCacheLockPartitionOnAffinityRunTest.java |   6 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |   4 +
 .../cache/IgniteCacheQueryLoadSelfTest.java     |  20 +-
 .../cache/IncorrectCacheTypeMetadataTest.java   |   8 +-
 .../cache/IncorrectQueryEntityTest.java         |   8 +-
 .../query/IgniteQueryDedicatedPoolTest.java     | 223 +++++
 .../query/IgniteSqlQueryDedicatedPoolTest.java  | 110 ---
 .../query/IgniteSqlSegmentedIndexSelfTest.java  |  26 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +-
 modules/platforms/cpp/binary/Makefile.am        |   2 +-
 .../platforms/cpp/binary/include/Makefile.am    |   1 +
 .../include/ignite/binary/binary_object.h       |  75 +-
 .../ignite/impl/binary/binary_field_meta.h      | 110 +++
 .../ignite/impl/binary/binary_id_resolver.h     |  96 +-
 .../ignite/impl/binary/binary_object_header.h   |  13 +
 .../ignite/impl/binary/binary_object_impl.h     | 124 ++-
 .../include/ignite/impl/binary/binary_schema.h  |  10 +-
 .../ignite/impl/binary/binary_type_handler.h    |  47 +-
 .../ignite/impl/binary/binary_type_manager.h    |  48 +-
 .../ignite/impl/binary/binary_type_snapshot.h   |  82 +-
 .../ignite/impl/binary/binary_type_updater.h    |  19 +-
 .../ignite/impl/binary/binary_writer_impl.h     |  17 +-
 .../cpp/binary/project/vs/binary.vcxproj        |   3 +-
 .../binary/project/vs/binary.vcxproj.filters    |   9 +-
 .../src/impl/binary/binary_field_meta.cpp       |  42 +
 .../src/impl/binary/binary_object_impl.cpp      | 139 ++-
 .../src/impl/binary/binary_type_handler.cpp     |  45 +-
 .../src/impl/binary/binary_type_manager.cpp     | 187 ++--
 .../src/impl/binary/binary_type_snapshot.cpp    |  50 +-
 .../src/impl/binary/binary_type_updater.cpp     |  32 -
 .../cpp/core-test/config/cache-identity.xml     |  33 +
 .../core-test/include/ignite/binary_test_defs.h |   5 +
 .../src/binary_identity_resolver_test.cpp       |  91 +-
 .../cpp/core-test/src/binary_object_test.cpp    | 220 ++++-
 .../cpp/core-test/src/cache_invoke_test.cpp     |   2 +
 .../cpp/core-test/src/cluster_test.cpp          |   5 +-
 .../impl/binary/binary_type_updater_impl.h      |   6 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  11 +
 .../ignite/impl/interop/interop_target.h        |   2 +-
 .../impl/binary/binary_type_updater_impl.cpp    | 101 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |   4 +-
 .../core/src/impl/interop/interop_target.cpp    |   2 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   2 -
 .../platforms/cpp/jni/include/ignite/jni/java.h |   4 -
 modules/platforms/cpp/jni/project/vs/module.def |   2 -
 modules/platforms/cpp/jni/src/exports.cpp       |   8 -
 modules/platforms/cpp/jni/src/java.cpp          |  20 -
 .../cpp/odbc/install/install_amd64.cmd          |   4 +-
 .../platforms/cpp/odbc/install/install_x86.cmd  |   2 +-
 .../Apache.Ignite.Core.Tests.csproj             |   1 -
 .../Binary/BinaryBuilderSelfTest.cs             | 151 +--
 .../BinaryBuilderSelfTestArrayIdentity.cs       |  34 -
 .../Binary/BinarySelfTest.cs                    |   8 -
 .../Cache/CacheConfigurationTest.cs             |   4 +-
 .../Cache/Query/CacheDmlQueriesTest.cs          |   5 +-
 .../Cache/Query/CacheLinqTest.cs                | 129 +--
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   7 +
 .../Cache/Store/CacheStoreAdapterTest.cs        |  14 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |   2 +-
 .../Cache/Store/CacheTestParallelLoadStore.cs   |  16 +-
 .../Cache/Store/CacheTestStore.cs               |  13 +-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   1 -
 .../IgniteConfigurationSerializerTest.cs        |   6 +-
 .../Apache.Ignite.Core.csproj                   |   8 +-
 .../Binary/BinaryArrayEqualityComparer.cs       |   8 +
 .../Binary/IBinaryObjectBuilder.cs              |  10 -
 .../Cache/Affinity/AffinityFunctionBase.cs      | 139 +++
 .../Cache/Affinity/Fair/FairAffinityFunction.cs |   1 -
 .../Rendezvous/RendezvousAffinityFunction.cs    |   1 -
 .../Cache/Configuration/CacheConfiguration.cs   |   2 +-
 .../Cache/Configuration/QueryEntity.cs          |   8 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  12 +-
 .../Store/CacheParallelLoadStoreAdapter.cs      |  38 +-
 .../Cache/Store/CacheStoreAdapter.cs            |  30 +-
 .../Cache/Store/ICacheStore.cs                  |  39 +-
 .../Cache/Store/ICacheStoreSession.cs           |   2 +-
 .../Datastream/IDataStreamer.cs                 |   2 +-
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |   8 -
 .../IgniteConfigurationSection.xsd              |   7 +
 .../Impl/Binary/BinaryObject.cs                 |  72 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |  67 +-
 .../Impl/Binary/BinaryProcessor.cs              |   3 +-
 .../Impl/Binary/BinaryUtils.cs                  |  10 +
 .../Impl/Binary/BinaryWriter.cs                 |  10 +-
 .../Impl/Binary/IBinaryEqualityComparer.cs      |  10 +
 .../Impl/Binary/Io/IBinaryStream.cs             |   3 +-
 .../Impl/Binary/Io/IBinaryStreamProcessor.cs    |   5 +-
 .../Impl/Binary/Marshaller.cs                   |   6 +-
 .../Impl/Binary/Metadata/BinaryField.cs         |  72 ++
 .../Impl/Binary/Metadata/BinaryType.cs          |  38 +-
 .../Binary/Metadata/BinaryTypeHashsetHandler.cs |  10 +-
 .../Impl/Binary/Metadata/BinaryTypeHolder.cs    |   9 +-
 .../Impl/Binary/Metadata/IBinaryTypeHandler.cs  |   4 +-
 .../Impl/Cache/Affinity/AffinityFunctionBase.cs | 140 ---
 .../Impl/Cache/Store/CacheStore.cs              | 233 +----
 .../Impl/Cache/Store/CacheStoreInternal.cs      | 285 ++++++
 .../Impl/Cache/Store/ICacheStoreInternal.cs     |  43 +
 .../Impl/Collections/MultiValueDictionary.cs    |   2 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |   3 +-
 .../Impl/Common/IFutureConverter.cs             |   4 +-
 .../Impl/Common/IFutureInternal.cs              |   3 +-
 .../Impl/Common/LoadedAssembliesResolver.cs     |   2 +-
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |   2 +-
 .../Impl/Handle/HandleRegistry.cs               |   2 +-
 .../Apache.Ignite.Core/Impl/Handle/IHandle.cs   |   2 +-
 .../Impl/Memory/IPlatformMemory.cs              |   5 +-
 .../Impl/Memory/PlatformMemory.cs               |   3 +-
 .../Impl/Memory/PlatformMemoryManager.cs        |   4 +-
 .../Impl/Memory/PlatformMemoryPool.cs           |   3 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |   3 +-
 .../Impl/Memory/PlatformRawMemory.cs            |  96 --
 .../Apache.Ignite.Linq.csproj                   |   1 -
 .../dotnet/Apache.Ignite.Linq/CompiledQuery.cs  | 112 ++-
 .../dotnet/Apache.Ignite.Linq/CompiledQuery2.cs | 257 -----
 .../Impl/CacheFieldsQueryProvider.cs            |  10 +-
 .../Datagrid/LinqExample.cs                     |   2 +-
 .../Datagrid/StoreExample.cs                    |   1 +
 .../Datagrid/EmployeeStore.cs                   |  27 +-
 .../scalar/pimps/ScalarProjectionPimp.scala     |  24 +-
 ...gniteProjectionStartStopRestartSelfTest.java |   6 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |  22 +-
 .../clusters/general/discovery/s3.pug           |   2 +-
 .../cache/IgniteBinaryIdentityBenchmark.java    |  21 -
 .../IgniteLegacyBinaryIdentityGetBenchmark.java |  30 -
 .../IgniteLegacyBinaryIdentityPutBenchmark.java |  30 -
 .../cache/WaitMapExchangeFinishCallable.java    |   4 +-
 .../IgniteAtomicInvokeRetryBenchmark.java       |  12 +-
 .../failover/IgniteAtomicRetriesBenchmark.java  |  12 +-
 .../IgniteFailoverAbstractBenchmark.java        |  22 +-
 ...IgniteTransactionalInvokeRetryBenchmark.java |  10 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java |  16 +-
 .../IgniteTransactionalWriteReadBenchmark.java  |  12 +-
 .../apache/ignite/yarn/ClusterProperties.java   |   1 +
 .../yarn/IgniteApplicationMasterSelfTest.java   |  13 +
 scripts/git-remoteless-branches.sh              |  16 +
 579 files changed, 14691 insertions(+), 7797 deletions(-)
----------------------------------------------------------------------



[4/8] ignite git commit: GC pressure

Posted by sb...@apache.org.
GC pressure


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

Branch: refs/heads/ignite-4811-no-start-ver
Commit: 37eed3420d559922568261b37f1e9f7aa5f25905
Parents: 226d698
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Mon Mar 27 13:36:51 2017 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Tue Mar 28 16:39:21 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/pagemem/Page.java    |  84 --
 .../ignite/internal/pagemem/PageMemory.java     |  20 +-
 .../ignite/internal/pagemem/PageSupport.java    | 102 +++
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 177 ++--
 .../internal/pagemem/impl/PageNoStoreImpl.java  | 141 ---
 .../cache/CacheOffheapEvictionManager.java      |   3 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  16 +-
 .../cache/database/CacheDataRowAdapter.java     |  26 +-
 .../cache/database/DataStructure.java           | 243 +++++-
 .../processors/cache/database/RowStore.java     |  10 -
 .../cache/database/freelist/FreeListImpl.java   | 148 ++--
 .../cache/database/freelist/PagesList.java      | 687 ++++++++-------
 .../cache/database/tree/BPlusTree.java          | 856 ++++++++++---------
 .../cache/database/tree/io/PageIO.java          |   3 +-
 .../database/tree/reuse/ReuseListImpl.java      |   2 +-
 .../cache/database/tree/util/PageHandler.java   | 391 ++++++---
 .../database/tree/util/PageLockListener.java    |  42 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  18 +-
 .../cache/version/GridCacheVersionManager.java  |   3 +
 .../clock/GridClockDeltaSnapshot.java           |   2 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  | 116 +--
 .../database/BPlusTreeReuseSelfTest.java        |  29 +-
 .../processors/database/BPlusTreeSelfTest.java  |  92 +-
 .../processors/query/h2/database/H2Tree.java    |  13 +-
 .../h2/database/InlineIndexHelperTest.java      |  44 +-
 25 files changed, 1817 insertions(+), 1451 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
deleted file mode 100644
index a93d186..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
+++ /dev/null
@@ -1,84 +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.pagemem;
-
-/**
- *
- */
-public interface Page extends AutoCloseable {
-    /**
-     * Gets the page ID. Page ID is a unique page identifier that does not change when partitions migrate
-     * from one node to another. Links (which is a page ID and 8-byte offset within a page) must be used
-     * when referencing data across pages.
-     *
-     * @return Page ID.
-     */
-    public long id();
-
-    /**
-     * @return Full page ID.
-     */
-    public FullPageId fullId();
-
-    /**
-     * @return Pointer for reading the page.
-     */
-    public long getForReadPointer();
-
-    /**
-     * Releases reserved page. Released page can be evicted from RAM after flushing modifications to disk.
-     */
-    public void releaseRead();
-
-    /**
-     * @return ByteBuffer for modifying the page.
-     */
-    public long getForWritePointer();
-
-    /**
-     * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
-     */
-    public long tryGetForWritePointer();
-
-    /**
-     * Releases reserved page. Released page can be evicted from RAM after flushing modifications to disk.
-     */
-    public void releaseWrite(boolean markDirty);
-
-    /**
-     * @return {@code True} if the page was modified since the last checkpoint.
-     */
-    public boolean isDirty();
-
-    /**
-     * @param plc {@code true} If page should be always recorded to WAL on {@link #releaseWrite(boolean)},
-     *            {@code false} if the page must never be recorded and {@code null} for the default behavior.
-     */
-    public void fullPageWalRecordPolicy(Boolean plc);
-
-    /**
-     * @return Policy for the page.
-     * @see #fullPageWalRecordPolicy(Boolean)
-     */
-    public Boolean fullPageWalRecordPolicy();
-
-    /**
-     * Release page.
-     */
-    @Override public void close();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
index 6333ff9..c20e1a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
@@ -18,29 +18,11 @@
 package org.apache.ignite.internal.pagemem;
 
 import java.nio.ByteBuffer;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.lifecycle.LifecycleAware;
 
 /**
  */
-public interface PageMemory extends LifecycleAware, PageIdAllocator {
-    /**
-     * Gets the page associated with the given page ID. Each page obtained with this method must be released by
-     * calling {@link #releasePage(Page)}. This method will allocate page with given ID if it doesn't exist.
-     *
-     * @param cacheId Cache ID.
-     * @param pageId Page ID.
-     * @return Page.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Page page(int cacheId, long pageId) throws IgniteCheckedException;
-
-    /**
-     * @param page Page to release.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void releasePage(Page page) throws IgniteCheckedException;
-
+public interface PageMemory extends LifecycleAware, PageIdAllocator, PageSupport {
     /**
      * @return Page size in bytes.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
new file mode 100644
index 0000000..7d1f711
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
@@ -0,0 +1,102 @@
+/*
+ * 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.pagemem;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Supports operations on pages.
+ */
+public interface PageSupport {
+    /**
+     * Gets the page absolute pointer associated with the given page ID. Each page obtained with this method must be
+     * released by calling {@link #releasePage(int, long, long)}. This method will allocate page with given ID if it doesn't
+     * exist.
+     *
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @return Page pointer.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long acquirePage(int cacheId, long pageId) throws IgniteCheckedException;
+
+    /**
+     *
+     * @param cacheId Cache ID.
+     * @param pageId Page ID to release.
+     * @param page Page pointer.
+     */
+    public void releasePage(int cacheId, long pageId, long page);
+
+    /**
+     *
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @return Pointer for reading the page.
+     */
+    public long readLock(int cacheId, long pageId, long page);
+
+    /**
+     * Releases locked page.
+     *
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     */
+    public void readUnlock(int cacheId, long pageId, long page);
+
+    /**
+     *
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @return ByteBuffer for modifying the page.
+     */
+    public long writeLock(int cacheId, long pageId, long page);
+
+    /**
+     *
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
+     */
+    public long tryWriteLock(int cacheId, long pageId, long page);
+
+    /**
+     * Releases locked page.
+     *
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param walPlc {@code True} if page should be recorded to WAL, {@code false} if the page must not
+*                                be recorded and {@code null} for the default behavior.
+     * @param dirtyFlag Determines whether the page was modified since the last checkpoint.
+     */
+    public void writeUnlock(int cacheId, long pageId, long page, Boolean walPlc,
+        boolean dirtyFlag);
+
+    /**
+     * @param cacheId Cache ID.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @return {@code True} if the page is dirty.
+     */
+    public boolean isDirty(int cacheId, long pageId, long page);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/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 05fce3d..7afd5bd 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
@@ -22,17 +22,16 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.ignite.IgniteCheckedException;
 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.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.OutOfMemoryException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
@@ -67,7 +66,7 @@ import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
 @SuppressWarnings({"LockAcquiredButNotSafelyReleased", "FieldAccessedSynchronizedAndUnsynchronized"})
 public class PageMemoryNoStoreImpl implements PageMemory {
     /** */
-    public static final long PAGE_MARKER = 0xBEEAAFDEADBEEF01L;
+    private static final long PAGE_MARKER = 0xBEEAAFDEADBEEF01L;
 
     /** Full relative pointer mask. */
     private static final long RELATIVE_PTR_MASK = 0xFFFFFFFFFFFFFFL;
@@ -85,16 +84,16 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     private static final long COUNTER_INC = ADDRESS_MASK + 1;
 
     /** Page ID offset. */
-    public static final int PAGE_ID_OFFSET = 8;
+    private static final int PAGE_ID_OFFSET = 8;
 
     /** Page pin counter offset. */
-    public static final int LOCK_OFFSET = 16;
+    private static final int LOCK_OFFSET = 16;
 
     /**
      * Need a 8-byte pointer for linked list, 8 bytes for internal needs (flags),
      * 4 bytes cache ID, 8 bytes timestamp.
      */
-    public static final int PAGE_OVERHEAD = LOCK_OFFSET + OffheapReadWriteLock.LOCK_SIZE;
+    static final int PAGE_OVERHEAD = LOCK_OFFSET + OffheapReadWriteLock.LOCK_SIZE;
 
     /** Page size. */
     private int sysPageSize;
@@ -269,24 +268,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /** {@inheritDoc} */
-    @Override public Page page(int cacheId, long pageId) throws IgniteCheckedException {
-        int pageIdx = PageIdUtils.pageIndex(pageId);
-
-        Segment seg = segment(pageIdx);
-
-        return seg.acquirePage(pageIdx, pageId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void releasePage(Page p) {
-        if (trackAcquiredPages) {
-            Segment seg = segment(PageIdUtils.pageIndex(p.id()));
-
-            seg.onPageRelease();
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public int pageSize() {
         return sysPageSize - PAGE_OVERHEAD;
     }
@@ -296,7 +277,9 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         return sysPageSize;
     }
 
-    /** */
+    /**
+     * @return Next index.
+     */
     private int nextRoundRobinIndex() {
         while (true) {
             int idx = selector.get();
@@ -339,7 +322,11 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             seg.readLock().lock();
 
             try {
-                total += seg.acquiredPages();
+                int acquired = seg.acquiredPages();
+
+                assert acquired >= 0;
+
+                total += acquired;
             }
             finally {
                 seg.readLock().unlock();
@@ -350,68 +337,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /**
-     * @param absPtr Page absolute address.
-     */
-    boolean readLockPage(long absPtr, int tag) {
-        return rwLock.readLock(absPtr + LOCK_OFFSET, tag);
-    }
-
-    /**
-     * @param absPtr Page absolute address.
-     */
-    void readUnlockPage(long absPtr) {
-        rwLock.readUnlock(absPtr + LOCK_OFFSET);
-    }
-
-    /**
-     * @param absPtr Page absolute address.
-     */
-    boolean writeLockPage(long absPtr, int tag) {
-        return rwLock.writeLock(absPtr + LOCK_OFFSET, tag);
-    }
-
-    /**
-     * @param absPtr Page absolute address.
-     * @return {@code True} if locked page.
-     */
-    boolean tryWriteLockPage(long absPtr, int tag) {
-        return rwLock.tryWriteLock(absPtr + LOCK_OFFSET, tag);
-    }
-
-    /**
-     * @param absPtr Page absolute address.
-     */
-    void writeUnlockPage(long absPtr, int newTag) {
-        rwLock.writeUnlock(absPtr + LOCK_OFFSET, newTag);
-    }
-
-    /**
-     * @param absPtr Absolute pointer to the page.
-     * @return {@code True} if write lock acquired for the page.
-     */
-    boolean isPageWriteLocked(long absPtr) {
-        return rwLock.isWriteLocked(absPtr + LOCK_OFFSET);
-    }
-
-    /**
-     * @param absPtr Absolute pointer to the page.
-     * @return {@code True} if read lock acquired for the page.
-     */
-    boolean isPageReadLocked(long absPtr) {
-        return rwLock.isReadLocked(absPtr + LOCK_OFFSET);
-    }
-
-    /**
-     * Reads page ID from the page at the given absolute position.
-     *
-     * @param absPtr Absolute memory pointer to the page header.
-     * @return Page ID written to the page.
-     */
-    long readPageId(long absPtr) {
-        return GridUnsafe.getLong(absPtr + PAGE_ID_OFFSET);
-    }
-
-    /**
      * Writes page ID to the page at the given absolute position.
      *
      * @param absPtr Absolute memory pointer to the page header.
@@ -453,6 +378,68 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         return res;
     }
 
+    // *** PageSupport methods ***
+
+    /** {@inheritDoc} */
+    @Override public long acquirePage(int cacheId, long pageId) {
+        int pageIdx = PageIdUtils.pageIndex(pageId);
+
+        Segment seg = segment(pageIdx);
+
+        return seg.acquire(pageIdx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void releasePage(int cacheId, long pageId, long page) {
+        if (trackAcquiredPages) {
+            Segment seg = segment(PageIdUtils.pageIndex(pageId));
+
+            seg.onPageRelease();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLock(int cacheId, long pageId, long page) {
+        if (rwLock.readLock(page + LOCK_OFFSET, PageIdUtils.tag(pageId)))
+            return page + PAGE_OVERHEAD;
+
+        return 0L;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readUnlock(int cacheId, long pageId, long page) {
+        rwLock.readUnlock(page + LOCK_OFFSET);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long writeLock(int cacheId, long pageId, long page) {
+        if (rwLock.writeLock(page + LOCK_OFFSET, PageIdUtils.tag(pageId)))
+            return page + PAGE_OVERHEAD;
+
+        return 0L;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long tryWriteLock(int cacheId, long pageId, long page) {
+        if (rwLock.tryWriteLock(page + LOCK_OFFSET, PageIdUtils.tag(pageId)))
+            return page + PAGE_OVERHEAD;
+
+        return 0L;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUnlock(int cacheId, long pageId, long page,
+        Boolean walPlc,
+        boolean dirtyFlag) {
+        long actualId = PageIO.getPageId(page + PAGE_OVERHEAD);
+        rwLock.writeUnlock(page + LOCK_OFFSET, PageIdUtils.tag(actualId));
+    }
+
+    @Override public boolean isDirty(int cacheId, long pageId, long page) {
+        // always false for page no store.
+        return false;
+    }
+
     /**
      *
      */
@@ -516,11 +503,9 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
         /**
          * @param pageIdx Page index.
-         * @param pageId Page ID to pin.
-         * @return Pinned page impl.
+         * @return Page absolute pointer.
          */
-        @SuppressWarnings("TypeMayBeWeakened")
-        private PageNoStoreImpl acquirePage(int pageIdx, long pageId) {
+        private long acquire(int pageIdx) {
             long absPtr = absolute(pageIdx);
 
             assert absPtr % 8 == 0 : absPtr;
@@ -528,7 +513,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             if (trackAcquiredPages)
                 acquiredPages.incrementAndGet();
 
-            return new PageNoStoreImpl(PageMemoryNoStoreImpl.this, absPtr, pageId);
+            return absPtr;
         }
 
         /**
@@ -604,12 +589,12 @@ public class PageMemoryNoStoreImpl implements PageMemory {
                 long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
 
                 if (freePageRelPtr != INVALID_REL_PTR) {
-                    long freePageAbsPtr = absolute(PageIdUtils.pageIndex(freePageRelPtr));
+                    long freePage = absolute(PageIdUtils.pageIndex(freePageRelPtr));
 
-                    long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
+                    long nextFreePageRelPtr = GridUnsafe.getLong(freePage) & ADDRESS_MASK;
 
                     if (GridUnsafe.compareAndSwapLong(null, freePageListPtr, freePageRelPtrMasked, nextFreePageRelPtr | cnt)) {
-                        GridUnsafe.putLong(freePageAbsPtr, PAGE_MARKER);
+                        GridUnsafe.putLong(freePage, PAGE_MARKER);
 
                         allocatedPages.incrementAndGet();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
deleted file mode 100644
index e82b5d2..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
+++ /dev/null
@@ -1,141 +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.pagemem.impl;
-
-import org.apache.ignite.internal.pagemem.FullPageId;
-import org.apache.ignite.internal.pagemem.Page;
-import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-
-/**
- *
- */
-public class PageNoStoreImpl implements Page {
-    /** */
-    private long absPtr;
-
-    /** */
-    private long pageId;
-
-    /** */
-    private PageMemoryNoStoreImpl pageMem;
-
-    /**
-     * @param pageMem Page memory.
-     * @param absPtr Absolute pointer.
-     * @param pageId Page ID.
-     */
-    PageNoStoreImpl(PageMemoryNoStoreImpl pageMem, long absPtr, long pageId) {
-        assert absPtr % 8 == 0 : absPtr;
-
-        this.pageMem = pageMem;
-        this.absPtr = absPtr;
-
-        this.pageId = pageId;
-    }
-
-    /**
-     * @return Data pointer.
-     */
-    private long pointer() {
-        return absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long id() {
-        return pageId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public FullPageId fullId() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getForReadPointer() {
-        if (pageMem.readLockPage(absPtr, PageIdUtils.tag(pageId)))
-            return pointer();
-
-        return 0L;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void releaseRead() {
-        pageMem.readUnlockPage(absPtr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getForWritePointer() {
-        int tag = PageIdUtils.tag(pageId);
-        boolean locked = pageMem.writeLockPage(absPtr, tag);
-
-        if (!locked)
-            return 0L;
-
-        return pointer();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long tryGetForWritePointer() {
-        int tag = PageIdUtils.tag(pageId);
-
-        if (pageMem.tryWriteLockPage(absPtr, tag))
-            return pointer();
-
-        return 0L;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void releaseWrite(boolean markDirty) {
-        long updatedPageId = PageIO.getPageId(pointer());
-
-        pageMem.writeUnlockPage(absPtr, PageIdUtils.tag(updatedPageId));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isDirty() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void fullPageWalRecordPolicy(Boolean plc) {
-        // No-op
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean fullPageWalRecordPolicy() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        pageMem.releasePage(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        SB sb = new SB("PageNoStoreImpl [absPtr=0x");
-
-        sb.appendHex(absPtr);
-        sb.a(", pageId=0x").appendHex(pageId);
-        sb.a("]");
-
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
index 6c925ad..e6a9ee7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
@@ -22,6 +22,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
@@ -50,7 +51,7 @@ public class CacheOffheapEvictionManager extends GridCacheManagerAdapter impleme
                 return;
             }
 
-            boolean evicted = e.evictInternal(cctx.versions().next(), null);
+            boolean evicted = e.evictInternal(GridCacheVersionManager.EVICT_VER, null);
 
             if (evicted)
                 cctx.cache().removeEntry(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 16d3715..6aac083 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -30,7 +30,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.pagemem.FullPageId;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
@@ -1472,10 +1471,10 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         private int compareKeys(KeyCacheObject key, final long link) throws IgniteCheckedException {
             byte[] bytes = key.valueBytes(cctx.cacheObjectContext());
 
-            PageMemory pageMem = cctx.shared().database().pageMemory();
-
-            try (Page page = page(pageId(link))) {
-                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
+            final long pageId = pageId(link);
+            final long page = acquirePage(pageId);
+            try {
+                long pageAddr = readLock(pageId, page); // Non-empty data page must not be recycled.
 
                 assert pageAddr != 0L : link;
 
@@ -1484,7 +1483,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
                     DataPagePayload data = io.readPayload(pageAddr,
                         itemId(link),
-                        pageMem.pageSize());
+                        pageSize());
 
                     if (data.nextLink() == 0) {
                         long addr = pageAddr + data.offset();
@@ -1524,9 +1523,12 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
                     }
                 }
                 finally {
-                    page.releaseRead();
+                    readUnlock(pageId, page, pageAddr);
                 }
             }
+            finally {
+                releasePage(pageId, page);
+            }
 
             // TODO GG-11768.
             CacheDataRowAdapter other = new CacheDataRowAdapter(link);

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
index 5a62e75..b751274 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.database;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
@@ -98,16 +97,20 @@ public class CacheDataRowAdapter implements CacheDataRow {
         assert key == null : "key";
 
         final CacheObjectContext coctx = cctx.cacheObjectContext();
+        final PageMemory pageMem = cctx.shared().database().pageMemory();
+
+        final int cacheId = cctx.cacheId();
 
         long nextLink = link;
         IncompleteObject<?> incomplete = null;
         boolean first = true;
 
         do {
-            PageMemory pageMem = cctx.shared().database().pageMemory();
+            final long pageId = pageId(nextLink);
+            final long page = pageMem.acquirePage(cacheId, pageId);
 
-            try (Page page = page(pageId(nextLink), cctx)) {
-                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
+            try {
+                long pageAddr = pageMem.readLock(cacheId, pageId, page); // Non-empty data page must not be recycled.
 
                 assert pageAddr != 0L : nextLink;
 
@@ -144,9 +147,12 @@ public class CacheDataRowAdapter implements CacheDataRow {
                         return;
                 }
                 finally {
-                    page.releaseRead();
+                    pageMem.readUnlock(cacheId, pageId, page);
                 }
             }
+            finally {
+                pageMem.releasePage(cacheId, pageId, page);
+            }
         }
         while(nextLink != 0);
 
@@ -454,16 +460,6 @@ public class CacheDataRowAdapter implements CacheDataRow {
     }
 
     /**
-     * @param pageId Page ID.
-     * @param cctx Cache context.
-     * @return Page.
-     * @throws IgniteCheckedException If failed.
-     */
-    private Page page(final long pageId, final GridCacheContext cctx) throws IgniteCheckedException {
-        return cctx.shared().database().pageMemory().page(cctx.cacheId(), pageId);
-    }
-
-    /**
      *
      */
     public enum RowData {

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
index f47a697..0e35bf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
@@ -20,11 +20,12 @@ package org.apache.ignite.internal.processors.cache.database;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.pagemem.wal.record.delta.RecycleRecord;
+import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseBag;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
@@ -118,56 +119,236 @@ public abstract class DataStructure implements PageLockListener {
 
     /**
      * @param pageId Page ID.
-     * @return Page.
+     * @return Page absolute pointer.
      * @throws IgniteCheckedException If failed.
      */
-    protected final Page page(long pageId) throws IgniteCheckedException {
+    protected final long acquirePage(long pageId) throws IgniteCheckedException {
         assert PageIdUtils.flag(pageId) == FLAG_IDX && PageIdUtils.partId(pageId) == INDEX_PARTITION ||
             PageIdUtils.flag(pageId) == FLAG_DATA && PageIdUtils.partId(pageId) <= MAX_PARTITION_ID : U.hexLong(pageId);
 
-        return pageMem.page(cacheId, pageId);
+        return pageMem.acquirePage(cacheId, pageId);
     }
 
     /**
-     * @param page Page.
-     * @return Page address.
+     * @param pageId Page ID.
+     * @param page  Page pointer.
      */
-    protected final long tryWriteLock(Page page) {
-        return PageHandler.writeLock(page, this, true);
+    protected final void releasePage(long pageId, long page) {
+        pageMem.releasePage(cacheId, pageId, page);
     }
 
+    /**
+     * @param pageId Page ID
+     * @param page Page pointer.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
+     */
+    protected final long tryWriteLock(long pageId, long page) {
+        return PageHandler.writeLock(pageMem, cacheId, pageId, page, this, true);
+    }
 
     /**
-     * @param page Page.
+     * @param pageId Page ID
+     * @param page Page pointer.
      * @return Page address.
      */
-    protected final long writeLock(Page page) {
-        return PageHandler.writeLock(page, this, false);
+    protected final long writeLock(long pageId, long page) {
+        return PageHandler.writeLock(pageMem, cacheId, pageId, page, this, false);
     }
 
     /**
-     * @param page Page.
+     * <p>
+     * Note: Default WAL record policy will be used.
+     * </p>
+     * @param pageId Page ID
+     * @param page Page pointer.
      * @param pageAddr Page address.
-     * @param dirty Dirty page.
+     * @param dirty Dirty flag.
      */
-    protected final void writeUnlock(Page page, long pageAddr, boolean dirty) {
-        PageHandler.writeUnlock(page, pageAddr, this, dirty);
+    protected final void writeUnlock(long pageId, long page, long pageAddr, boolean dirty) {
+        writeUnlock(pageId, page, pageAddr, null, dirty);
     }
 
     /**
-     * @param page Page.
+     * @param pageId Page ID
+     * @param page Page pointer.
      * @return Page address.
      */
-    protected final long readLock(Page page) {
-        return PageHandler.readLock(page, this);
+    protected final long readLock(long pageId, long page) {
+        return PageHandler.readLock(pageMem, cacheId, pageId, page, this);
+    }
+
+    /**
+     * @param pageId Page ID
+     * @param page Page pointer.
+     * @param pageAddr  Page address.
+     */
+    protected final void readUnlock(long pageId, long page, long pageAddr) {
+        PageHandler.readUnlock(pageMem, cacheId, pageId, page, pageAddr, this);
     }
 
     /**
-     * @param page Page.
-     * @param buf Buffer.
+     * @param pageId Page ID
+     * @param page Page pointer.
+     * @param pageAddr  Page address.
+     * @param walPlc Full page WAL record policy.
+     * @param dirty Dirty flag.
      */
-    protected final void readUnlock(Page page, long buf) {
-        PageHandler.readUnlock(page, buf, this);
+    protected final void writeUnlock(long pageId, long page, long pageAddr, Boolean walPlc, boolean dirty) {
+        PageHandler.writeUnlock(pageMem, cacheId, pageId, page, pageAddr, this, walPlc, dirty);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param walPlc Full page WAL record policy.
+     * @return {@code true} If we need to make a delta WAL record for the change in this page.
+     */
+    protected final boolean needWalDeltaRecord(long pageId, long page, Boolean walPlc) {
+        return PageHandler.isWalDeltaRecordNeeded(pageMem, cacheId, pageId, page, wal, walPlc);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param h Handler.
+     * @param intArg Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @return Handler result.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected final <R> R write(
+        long pageId,
+        PageHandler<?, R> h,
+        int intArg,
+        R lockFailed) throws IgniteCheckedException {
+        return PageHandler.writePage(pageMem, cacheId, pageId, this, h, null, null, null, null, intArg, lockFailed);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param h Handler.
+     * @param arg Argument.
+     * @param intArg Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @return Handler result.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected final <X, R> R write(
+        long pageId,
+        PageHandler<X, R> h,
+        X arg,
+        int intArg,
+        R lockFailed) throws IgniteCheckedException {
+        return PageHandler.writePage(pageMem, cacheId, pageId, this, h, null, null, null, arg, intArg, lockFailed);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param h Handler.
+     * @param arg Argument.
+     * @param intArg Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @return Handler result.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected final <X, R> R write(
+        long pageId,
+        long page,
+        PageHandler<X, R> h,
+        X arg,
+        int intArg,
+        R lockFailed) throws IgniteCheckedException {
+        return PageHandler.writePage(pageMem, cacheId, pageId, page, this, h, null, null, null, arg, intArg, lockFailed);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param h Handler.
+     * @param init IO for new page initialization or {@code null} if it is an existing page.
+     * @param arg Argument.
+     * @param intArg Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @return Handler result.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected final <X, R> R write(
+        long pageId,
+        PageHandler<X, R> h,
+        PageIO init,
+        X arg,
+        int intArg,
+        R lockFailed) throws IgniteCheckedException {
+        return PageHandler.writePage(pageMem, cacheId, pageId, this, h, init, wal, null, arg, intArg, lockFailed);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param h Handler.
+     * @param arg Argument.
+     * @param intArg Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @return Handler result.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected final <X, R> R read(
+        long pageId,
+        PageHandler<X, R> h,
+        X arg,
+        int intArg,
+        R lockFailed) throws IgniteCheckedException {
+        return PageHandler.readPage(pageMem, cacheId, pageId, this, h, arg, intArg, lockFailed);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param h Handler.
+     * @param arg Argument.
+     * @param intArg Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @return Handler result.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected final <X, R> R read(
+        long pageId,
+        long page,
+        PageHandler<X, R> h,
+        X arg,
+        int intArg,
+        R lockFailed) throws IgniteCheckedException {
+        return PageHandler.readPage(pageMem, cacheId, pageId, page, this, h, arg, intArg, lockFailed);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param init IO for new page initialization.
+     * @throws IgniteCheckedException if failed.
+     */
+    protected final void init(long pageId, PageIO init) throws IgniteCheckedException {
+        PageHandler.initPage(pageMem, cacheId, pageId, init, wal, this);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address.
+     * @param walPlc Full page WAL record policy.
+     * @return Rotated page ID.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected final long recyclePage(
+        long pageId,
+        long page,
+        long pageAddr,
+        Boolean walPlc) throws IgniteCheckedException {
+        long rotated = PageIdUtils.rotatePageId(pageId);
+
+        PageIO.setPageId(pageAddr, rotated);
+
+        if (needWalDeltaRecord(pageId, page, walPlc))
+            wal.log(new RecycleRecord(cacheId, pageId, rotated));
+
+        return rotated;
     }
 
     /**
@@ -177,33 +358,27 @@ public abstract class DataStructure implements PageLockListener {
         return pageMem.pageSize();
     }
 
-    /** {@inheritDoc} */
-    @Override public void onBeforeWriteLock(Page page) {
+    @Override public void onBeforeWriteLock(int cacheId, long pageId, long page) {
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public void onWriteLock(Page page, long pageAddr) {
+    @Override public void onWriteLock(int cacheId, long pageId, long page, long pageAddr) {
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public void onWriteUnlock(Page page, long pageAddr) {
+    @Override public void onWriteUnlock(int cacheId, long pageId, long page, long pageAddr) {
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public void onBeforeReadLock(Page page) {
+    @Override public void onBeforeReadLock(int cacheId, long pageId, long page) {
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public void onReadLock(Page page, long pageAddr) {
+    @Override public void onReadLock(int cacheId, long pageId, long page, long pageAddr) {
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public void onReadUnlock(Page page, long pageAddr) {
+    @Override public void onReadUnlock(int cacheId, long pageId, long page, long pageAddr) {
         // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
index 8d54542..ce0a4a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/RowStore.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.database;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -56,15 +55,6 @@ public class RowStore {
     }
 
     /**
-     * @param pageId Page ID.
-     * @return Page.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected final Page page(long pageId) throws IgniteCheckedException {
-        return pageMem.page(cctx.cacheId(), pageId);
-    }
-
-    /**
      * @param link Row link.
      * @throws IgniteCheckedException If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
index d6debd8..4d3270c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.database.freelist;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -40,8 +39,6 @@ import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.writePage;
-
 /**
  */
 public class FreeListImpl extends PagesList implements FreeList, ReuseList {
@@ -61,6 +58,9 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     private static final Long FAIL_L = Long.MAX_VALUE;
 
     /** */
+    private static final Boolean FAIL_B = null;
+
+    /** */
     private static final int MIN_PAGE_FREE_SPACE = 8;
 
     /** */
@@ -78,9 +78,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     /**
      *
      */
-    private class UpdateRowHandler extends PageHandler<CacheDataRow, Boolean> {
-        /** {@inheritDoc} */
-        @Override public Boolean run(Page page, PageIO iox, long pageAddr, CacheDataRow row, int itemId)
+    private final class UpdateRowHandler extends PageHandler<CacheDataRow, Boolean> {
+        @Override
+        public Boolean run(
+            int cacheId,
+            long pageId,
+            long page,
+            long pageAddr,
+            PageIO iox,
+            Boolean walPlc,
+            CacheDataRow row,
+            int itemId)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 
@@ -88,7 +96,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
             boolean updated = io.updateRow(pageAddr, itemId, pageSize(), null, row, rowSize);
 
-            if (updated && isWalDeltaRecordNeeded(wal, page)) {
+            if (updated && needWalDeltaRecord(pageId, page, walPlc)) {
                 // TODO This record must contain only a reference to a logical WAL record with the actual data.
                 byte[] payload = new byte[rowSize];
 
@@ -100,14 +108,14 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                 wal.log(new DataPageUpdateRecord(
                     cacheId,
-                    page.id(),
+                    pageId,
                     itemId,
                     payload));
             }
 
             return updated;
         }
-    };
+    }
 
     /** */
     private final PageHandler<CacheDataRow, Integer> writeRow = new WriteRowHandler();
@@ -115,9 +123,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     /**
      *
      */
-    private class WriteRowHandler extends PageHandler<CacheDataRow, Integer> {
-        /** {@inheritDoc} */
-        @Override public Integer run(Page page, PageIO iox, long pageAddr, CacheDataRow row, int written)
+    private final class WriteRowHandler extends PageHandler<CacheDataRow, Integer> {
+        @Override
+        public Integer run(
+            int cacheId,
+            long pageId,
+            long page,
+            long pageAddr,
+            PageIO iox,
+            Boolean walPlc,
+            CacheDataRow row,
+            int written)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 
@@ -127,8 +143,8 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
             assert oldFreeSpace > 0 : oldFreeSpace;
 
             // If the full row does not fit into this page write only a fragment.
-            written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(page, pageAddr, io, row, rowSize):
-                addRowFragment(page, pageAddr, io, row, written, rowSize);
+            written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(pageId, page, pageAddr, io, row, rowSize):
+                addRowFragment(pageId, page, pageAddr, io, row, written, rowSize);
 
             // Reread free space after update.
             int newFreeSpace = io.getFreeSpace(pageAddr);
@@ -136,7 +152,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
             if (newFreeSpace > MIN_PAGE_FREE_SPACE) {
                 int bucket = bucket(newFreeSpace, false);
 
-                put(null, page, pageAddr, bucket);
+                put(null, pageId, page, pageAddr, bucket);
             }
 
             // Avoid boxing with garbage generation for usual case.
@@ -144,7 +160,8 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
          * @param io IO.
          * @param row Row.
@@ -153,7 +170,8 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
          * @throws IgniteCheckedException If failed.
          */
         private int addRow(
-            Page page,
+            long pageId,
+            long page,
             long pageAddr,
             DataPageIO io,
             CacheDataRow row,
@@ -161,7 +179,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         ) throws IgniteCheckedException {
             io.addRow(pageAddr, row, rowSize, pageSize());
 
-            if (isWalDeltaRecordNeeded(wal, page)) {
+            if (needWalDeltaRecord(pageId, page, null)) {
                 // TODO This record must contain only a reference to a logical WAL record with the actual data.
                 byte[] payload = new byte[rowSize];
 
@@ -173,7 +191,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                 wal.log(new DataPageInsertRecord(
                     cacheId,
-                    page.id(),
+                    pageId,
                     payload));
             }
 
@@ -181,7 +199,8 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
          * @param io IO.
          * @param row Row.
@@ -191,7 +210,8 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
          * @throws IgniteCheckedException If failed.
          */
         private int addRowFragment(
-            Page page,
+            long pageId,
+            long page,
             long pageAddr,
             DataPageIO io,
             CacheDataRow row,
@@ -205,7 +225,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
             assert payloadSize > 0 : payloadSize;
 
-            if (isWalDeltaRecordNeeded(wal, page)) {
+            if (needWalDeltaRecord(pageId, page, null)) {
                 // TODO This record must contain only a reference to a logical WAL record with the actual data.
                 byte[] payload = new byte[payloadSize];
 
@@ -213,12 +233,12 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                 PageUtils.getBytes(pageAddr, data.offset(), payload, 0, payloadSize);
 
-                wal.log(new DataPageInsertFragmentRecord(cacheId, page.id(), payload, lastLink));
+                wal.log(new DataPageInsertFragmentRecord(cacheId, pageId, payload, lastLink));
             }
 
             return written + payloadSize;
         }
-    };
+    }
 
 
     /** */
@@ -227,9 +247,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     /**
      *
      */
-    private class RemoveRowHandler extends PageHandler<Void, Long> {
-        /** {@inheritDoc} */
-        @Override public Long run(Page page, PageIO iox, long pageAddr, Void arg, int itemId)
+    private final class RemoveRowHandler extends PageHandler<Void, Long> {
+        @Override
+        public Long run(
+            int cacheId,
+            long pageId,
+            long page,
+            long pageAddr,
+            PageIO iox,
+            Boolean walPlc,
+            Void ignored,
+            int itemId)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 
@@ -239,16 +267,8 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
             long nextLink = io.removeRow(pageAddr, itemId, pageSize());
 
-            if (isWalDeltaRecordNeeded(wal, page))
-                wal.log(new DataPageRemoveRecord(cacheId, page.id(), itemId));
-
-            // TODO: properly handle reuse bucket.
-//            if (io.isEmpty(buf)) {
-//                int oldBucket = oldFreeSpace > MIN_PAGE_FREE_SPACE ? bucket(oldFreeSpace, false) : -1;
-//
-//                if (oldBucket == -1 || removeDataPage(page, buf, io, oldBucket))
-//                    put(null, page, buf, REUSE_BUCKET);
-//            }
+            if (needWalDeltaRecord(pageId, page, walPlc))
+                wal.log(new DataPageRemoveRecord(cacheId, pageId, itemId));
 
             int newFreeSpace = io.getFreeSpace(pageAddr);
 
@@ -260,18 +280,18 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                     if (oldBucket != newBucket) {
                         // It is possible that page was concurrently taken for put, in this case put will handle bucket change.
-                        if (removeDataPage(page, pageAddr, io, oldBucket))
-                            put(null, page, pageAddr, newBucket);
+                        if (removeDataPage(pageId, page, pageAddr, io, oldBucket))
+                            put(null, pageId, page, pageAddr, newBucket);
                     }
                 }
                 else
-                    put(null, page, pageAddr, newBucket);
+                    put(null, pageId, page, pageAddr, newBucket);
             }
 
             // For common case boxed 0L will be cached inside of Long, so no garbage will be produced.
             return nextLink;
         }
-    };
+    }
 
     /**
      * @param cacheId Cache ID.
@@ -377,16 +397,14 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
     /**
      * @param part Partition.
-     * @return Page.
+     * @return Page ID.
      * @throws IgniteCheckedException If failed.
      */
-    private Page allocateDataPage(int part) throws IgniteCheckedException {
+    private long allocate(int part) throws IgniteCheckedException {
         assert part <= PageIdAllocator.MAX_PARTITION_ID;
         assert part != PageIdAllocator.INDEX_PARTITION;
 
-        long pageId = pageMem.allocatePage(cacheId, part, PageIdAllocator.FLAG_DATA);
-
-        return pageMem.page(cacheId, pageId);
+        return pageMem.allocatePage(cacheId, part, PageIdAllocator.FLAG_DATA);
     }
 
     /** {@inheritDoc} */
@@ -417,14 +435,16 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
             if (pageId == 0L)
                 pageId = takeEmptyPage(bucket, DataPageIO.VERSIONS);
 
-            try (Page page = pageId == 0 ? allocateDataPage(row.partition()) : pageMem.page(cacheId, pageId)) {
-                // If it is an existing page, we do not need to initialize it.
-                DataPageIO init = reuseBucket || pageId == 0L ? DataPageIO.VERSIONS.latest() : null;
+            boolean allocated = pageId == 0L;
 
-                written = writePage(pageMem, page, this, writeRow, init, wal, row, written, FAIL_I);
+            if(allocated)
+                pageId = allocate(row.partition());
 
-                assert written != FAIL_I; // We can't fail here.
-            }
+            DataPageIO init = reuseBucket || allocated ? DataPageIO.VERSIONS.latest() : null;
+
+            written = write(pageId, writeRow, init, row, written, FAIL_I);
+
+            assert written != FAIL_I; // We can't fail here.
         }
         while (written != COMPLETE);
     }
@@ -436,13 +456,11 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         long pageId = PageIdUtils.pageId(link);
         int itemId = PageIdUtils.itemId(link);
 
-        try (Page page = pageMem.page(cacheId, pageId)) {
-            Boolean updated = writePage(pageMem, page, this, updateRow, row, itemId, null);
+        Boolean updated = write(pageId, updateRow, row, itemId, FAIL_B);
 
-            assert updated != null; // Can't fail here.
+        assert updated != FAIL_B; // Can't fail here.
 
-            return updated != null ? updated : false;
-        }
+        return updated;
     }
 
     /** {@inheritDoc} */
@@ -452,23 +470,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         long pageId = PageIdUtils.pageId(link);
         int itemId = PageIdUtils.itemId(link);
 
-        long nextLink;
+        long nextLink = write(pageId, rmvRow, itemId, FAIL_L);
 
-        try (Page page = pageMem.page(cacheId, pageId)) {
-            nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
-
-            assert nextLink != FAIL_L; // Can't fail here.
-        }
+        assert nextLink != FAIL_L; // Can't fail here.
 
         while (nextLink != 0L) {
             itemId = PageIdUtils.itemId(nextLink);
             pageId = PageIdUtils.pageId(nextLink);
 
-            try (Page page = pageMem.page(cacheId, pageId)) {
-                nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
+            nextLink = write(pageId, rmvRow, itemId, FAIL_L);
 
-                assert nextLink != FAIL_L; // Can't fail here.
-            }
+            assert nextLink != FAIL_L; // Can't fail here.
         }
     }
 
@@ -491,7 +503,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     @Override public void addForRecycle(ReuseBag bag) throws IgniteCheckedException {
         assert reuseList == this: "not allowed to be a reuse list";
 
-        put(bag, null, 0L, REUSE_BUCKET);
+        put(bag, 0, 0, 0L, REUSE_BUCKET);
     }
 
     /** {@inheritDoc} */


[3/8] ignite git commit: GC pressure

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
index 5c66b10..6c51096 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
@@ -37,7 +36,6 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListInitNewPageR
 import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListRemovePageRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListSetNextRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListSetPreviousRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.RecycleRecord;
 import org.apache.ignite.internal.processors.cache.database.DataStructure;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListMetaIO;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
@@ -58,9 +56,6 @@ import static java.lang.Boolean.TRUE;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
 import static org.apache.ignite.internal.processors.cache.database.tree.io.PageIO.getPageId;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.initPage;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.isWalDeltaRecordNeeded;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.writePage;
 
 /**
  * Striped doubly-linked list of page IDs optionally organized in buckets.
@@ -93,13 +88,20 @@ public abstract class PagesList extends DataStructure {
     /**
      *
      */
-    private class CutTail extends PageHandler<Void, Boolean> {
-        /** {@inheritDoc} */
-        @Override public Boolean run(Page page, PageIO pageIo, long pageAddr, Void ignore, int bucket)
-            throws IgniteCheckedException {
-            assert getPageId(pageAddr) == page.id();
-
-            PagesListNodeIO io = (PagesListNodeIO)pageIo;
+    private final class CutTail extends PageHandler<Void, Boolean> {
+        @Override
+        public Boolean run(
+            int cacheId,
+            long pageId,
+            long page,
+            long pageAddr,
+            PageIO iox,
+            Boolean walPlc,
+            Void ignore,
+            int bucket) throws IgniteCheckedException {
+            assert getPageId(pageAddr) == pageId;
+
+            PagesListNodeIO io = (PagesListNodeIO)iox;
 
             long tailId = io.getNextId(pageAddr);
 
@@ -107,10 +109,10 @@ public abstract class PagesList extends DataStructure {
 
             io.setNextId(pageAddr, 0L);
 
-            if (isWalDeltaRecordNeeded(wal, page))
-                wal.log(new PagesListSetNextRecord(cacheId, page.id(), 0L));
+            if (needWalDeltaRecord(pageId, page, walPlc))
+                wal.log(new PagesListSetNextRecord(cacheId, pageId, 0L));
 
-            updateTail(bucket, tailId, page.id());
+            updateTail(bucket, tailId, pageId);
 
             return TRUE;
         }
@@ -153,18 +155,19 @@ public abstract class PagesList extends DataStructure {
     protected final void init(long metaPageId, boolean initNew) throws IgniteCheckedException {
         if (metaPageId != 0L) {
             if (initNew) {
-                try (Page page = page(metaPageId)) {
-                    initPage(pageMem, page, this, PagesListMetaIO.VERSIONS.latest(), wal);
-                }
+                init(metaPageId, PagesListMetaIO.VERSIONS.latest());
             }
             else {
                 Map<Integer, GridLongList> bucketsData = new HashMap<>();
 
-                long nextPageId = metaPageId;
+                long nextId = metaPageId;
 
-                while (nextPageId != 0) {
-                    try (Page page = page(nextPageId)) {
-                        long pageAddr = readLock(page); // No concurrent recycling on init.
+                while (nextId != 0) {
+                    final long pageId = nextId;
+                    final long page = acquirePage(pageId);
+
+                    try {
+                        long pageAddr = readLock(pageId, page); // No concurrent recycling on init.
 
                         assert pageAddr != 0L;
 
@@ -173,17 +176,20 @@ public abstract class PagesList extends DataStructure {
 
                             io.getBucketsData(pageAddr, bucketsData);
 
-                            long next0 = io.getNextMetaPageId(pageAddr);
+                            nextId = io.getNextMetaPageId(pageAddr);
+
+                            assert nextId != pageId :
+                                "Loop detected [next=" + U.hexLong(nextId) + ", cur=" + U.hexLong(pageId) + ']';
 
-                            assert next0 != nextPageId :
-                                "Loop detected [next=" + U.hexLong(next0) + ", cur=" + U.hexLong(nextPageId) + ']';
 
-                            nextPageId = next0;
                         }
                         finally {
-                            readUnlock(page, pageAddr);
+                            readUnlock(pageId, page, pageAddr);
                         }
                     }
+                    finally {
+                        releasePage(pageId, page);
+                    }
                 }
 
                 for (Map.Entry<Integer, GridLongList> e : bucketsData.entrySet()) {
@@ -200,12 +206,14 @@ public abstract class PagesList extends DataStructure {
                     for (int i = 0; i < upd.length; i++) {
                         long tailId = upd[i];
 
-                        long pageId = tailId;
+                        long prevId = tailId;
                         int cnt = 0;
 
-                        while (pageId != 0L) {
-                            try (Page page = page(pageId)) {
-                                long pageAddr = readLock(page);
+                        while (prevId != 0L) {
+                            final long pageId = prevId;
+                            final long page = acquirePage(pageId);
+                            try  {
+                                long pageAddr = readLock(pageId, page);
 
                                 assert pageAddr != 0L;
 
@@ -213,16 +221,19 @@ public abstract class PagesList extends DataStructure {
                                     PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
                                     cnt += io.getCount(pageAddr);
-                                    pageId = io.getPreviousId(pageAddr);
+                                    prevId = io.getPreviousId(pageAddr);
 
                                     // In reuse bucket the page itself can be used as a free page.
-                                    if (isReuseBucket(bucket) && pageId != 0L)
+                                    if (isReuseBucket(bucket) && prevId != 0L)
                                         cnt++;
                                 }
                                 finally {
-                                    readUnlock(page, pageAddr);
+                                    readUnlock(pageId, page, pageAddr);
                                 }
                             }
+                            finally {
+                                releasePage(pageId, page);
+                            }
                         }
 
                         Stripe stripe = new Stripe(tailId, cnt == 0);
@@ -245,11 +256,13 @@ public abstract class PagesList extends DataStructure {
     public void saveMetadata() throws IgniteCheckedException {
         assert metaPageId != 0;
 
-        Page curPage = null;
-        long curPageAddr = 0L;
+        long curId = 0L;
+        long cur = 0L;
+        long curAddr = 0L;
+
         PagesListMetaIO curIo = null;
 
-        long nextPageId = metaPageId;
+        long nextId = metaPageId;
 
         try {
             for (int bucket = 0; bucket < buckets; bucket++) {
@@ -259,39 +272,39 @@ public abstract class PagesList extends DataStructure {
                     int tailIdx = 0;
 
                     while (tailIdx < tails.length) {
-                        int written = curPage != null ? curIo.addTails(pageMem.pageSize(), curPageAddr, bucket, tails, tailIdx) : 0;
+                        int written = curAddr != 0L ? curIo.addTails(pageMem.pageSize(), curAddr, bucket, tails, tailIdx) : 0;
 
                         if (written == 0) {
-                            if (nextPageId == 0L) {
-                                nextPageId = allocatePageNoReuse();
+                            if (nextId == 0L) {
+                                nextId = allocatePageNoReuse();
 
-                                if (curPage != null) {
-                                    curIo.setNextMetaPageId(curPageAddr, nextPageId);
+                                if (curAddr != 0L) {
+                                    curIo.setNextMetaPageId(curAddr, nextId);
 
-                                    releaseAndClose(curPage, curPageAddr);
-                                    curPage = null;
+                                    releaseAndClose(curId, cur, curAddr);
                                 }
 
-                                curPage = page(nextPageId);
-                                curPageAddr = writeLock(curPage);
+                                curId = nextId;
+                                cur = acquirePage(curId);
+                                curAddr = writeLock(curId, cur);
 
                                 curIo = PagesListMetaIO.VERSIONS.latest();
 
-                                curIo.initNewPage(curPageAddr, nextPageId, pageSize());
+                                curIo.initNewPage(curAddr, curId, pageSize());
                             }
                             else {
-                                releaseAndClose(curPage, curPageAddr);
-                                curPage = null;
+                                releaseAndClose(curId, cur, curAddr);
 
-                                curPage = page(nextPageId);
-                                curPageAddr = writeLock(curPage);
+                                curId = nextId;
+                                cur = acquirePage(curId);
+                                curAddr = writeLock(curId, cur);
 
-                                curIo = PagesListMetaIO.VERSIONS.forPage(curPageAddr);
+                                curIo = PagesListMetaIO.VERSIONS.forPage(curAddr);
 
-                                curIo.resetCount(curPageAddr);
+                                curIo.resetCount(curAddr);
                             }
 
-                            nextPageId = curIo.getNextMetaPageId(curPageAddr);
+                            nextId = curIo.getNextMetaPageId(curAddr);
                         }
                         else
                             tailIdx += written;
@@ -300,44 +313,50 @@ public abstract class PagesList extends DataStructure {
             }
         }
         finally {
-            releaseAndClose(curPage, curPageAddr);
+            releaseAndClose(curId, cur, curAddr);
         }
 
-        while (nextPageId != 0L) {
-            try (Page page = page(nextPageId)) {
-                long pageAddr = writeLock(page);
+        while (nextId != 0L) {
+            long pageId = nextId;
+
+            long page = acquirePage(pageId);
+            try {
+                long pageAddr = writeLock(pageId, page);
 
                 try {
                     PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
                     io.resetCount(pageAddr);
 
-                    if (PageHandler.isWalDeltaRecordNeeded(wal, page))
-                        wal.log(new PageListMetaResetCountRecord(cacheId, nextPageId));
+                    if (needWalDeltaRecord(pageId, page, null))
+                        wal.log(new PageListMetaResetCountRecord(cacheId, pageId));
 
-                    nextPageId = io.getNextMetaPageId(pageAddr);
+                    nextId = io.getNextMetaPageId(pageAddr);
                 }
                 finally {
-                    writeUnlock(page, pageAddr, true);
+                    writeUnlock(pageId, page, pageAddr, true);
                 }
             }
+            finally {
+                releasePage(pageId, page);
+            }
         }
     }
 
     /**
-     * @param page Page.
-     * @param buf Buffer.
+     * @param pageId Page ID.
+     * @param page Page absolute pointer.
+     * @param pageAddr Page address.
+     * @throws IgniteCheckedException If failed.
      */
-    private void releaseAndClose(Page page, long buf) {
-        if (page != null) {
+    private void releaseAndClose(long pageId, long page, long pageAddr) throws IgniteCheckedException {
+        if (pageAddr != 0L) {
             try {
                 // No special WAL record because we most likely changed the whole page.
-                page.fullPageWalRecordPolicy(true);
-
-                writeUnlock(page, buf, true);
+                writeUnlock(pageId, page, pageAddr, TRUE, true);
             }
             finally {
-                page.close();
+                releasePage(pageId, page);
             }
         }
     }
@@ -389,9 +408,7 @@ public abstract class PagesList extends DataStructure {
     private Stripe addStripe(int bucket, boolean reuse) throws IgniteCheckedException {
         long pageId = reuse ? allocatePage(null) : allocatePageNoReuse();
 
-        try (Page page = page(pageId)) {
-            initPage(pageMem, page, this, PagesListNodeIO.VERSIONS.latest(), wal);
-        }
+        init(pageId, PagesListNodeIO.VERSIONS.latest());
 
         Stripe stripe = new Stripe(pageId, true);
 
@@ -518,28 +535,37 @@ public abstract class PagesList extends DataStructure {
 
         if (tails != null) {
             for (Stripe tail : tails) {
-                long pageId = tail.tailId;
+                long tailId = tail.tailId;
 
-                while (pageId != 0L) {
-                    try (Page page = page(pageId)) {
-                        long pageAddr = readLock(page);
+                while (tailId != 0L) {
+                    final long pageId = tailId;
+                    final long page = acquirePage(pageId);
+                    try {
+                        long pageAddr = readLock(pageId, page);
 
                         assert pageAddr != 0L;
 
                         try {
                             PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                            res += io.getCount(pageAddr);
-                            pageId = io.getPreviousId(pageAddr);
+                            int cnt = io.getCount(pageAddr);
+
+                            assert cnt >= 0;
+
+                            res += cnt;
+                            tailId = io.getPreviousId(pageAddr);
 
                             // In reuse bucket the page itself can be used as a free page.
-                            if (isReuseBucket(bucket) && pageId != 0L)
+                            if (isReuseBucket(bucket) && tailId != 0L)
                                 res++;
                         }
                         finally {
-                            readUnlock(page, pageAddr);
+                            readUnlock(pageId, page, pageAddr);
                         }
                     }
+                    finally {
+                        releasePage(pageId, page);
+                    }
                 }
             }
         }
@@ -551,48 +577,55 @@ public abstract class PagesList extends DataStructure {
 
     /**
      * @param bag Reuse bag.
-     * @param dataPage Data page.
-     * @param dataPageAddr Data page address.
+     * @param dataId Data page ID.
+     * @param dataPage Data page pointer.
+     * @param dataAddr Data page address.
      * @param bucket Bucket.
      * @throws IgniteCheckedException If failed.
      */
-    protected final void put(ReuseBag bag, Page dataPage, long dataPageAddr, int bucket)
+    protected final void put(
+        ReuseBag bag,
+        final long dataId,
+        final long dataPage,
+        final long dataAddr,
+        int bucket)
         throws IgniteCheckedException {
-        assert bag == null ^ dataPageAddr == 0L;
+        assert bag == null ^ dataAddr == 0L;
 
         for (int lockAttempt = 0; ;) {
             Stripe stripe = getPageForPut(bucket);
 
-            long tailId = stripe.tailId;
+            final long tailId = stripe.tailId;
+            final long tailPage = acquirePage(tailId);
 
-            try (Page tail = page(tailId)) {
-                long pageAddr = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+            try {
+                long tailAddr = writeLockPage(tailId, tailPage, bucket, lockAttempt++); // Explicit check.
 
-                if (pageAddr == 0L) {
+                if (tailAddr == 0L) {
                     if (isReuseBucket(bucket) && lockAttempt == TRY_LOCK_ATTEMPTS)
                         addStripeForReuseBucket(bucket);
 
                     continue;
                 }
 
-                assert PageIO.getPageId(pageAddr) == tailId : "pageId = " + PageIO.getPageId(pageAddr) + ", tailId = " + tailId;
-                assert PageIO.getType(pageAddr) == PageIO.T_PAGE_LIST_NODE;
+                assert PageIO.getPageId(tailAddr) == tailId : "pageId = " + PageIO.getPageId(tailAddr) + ", tailId = " + tailId;
+                assert PageIO.getType(tailAddr) == PageIO.T_PAGE_LIST_NODE;
 
                 boolean ok = false;
 
                 try {
-                    PagesListNodeIO io = PageIO.getPageIO(pageAddr);
+                    PagesListNodeIO io = PageIO.getPageIO(tailAddr);
 
                     ok = bag != null ?
                         // Here we can always take pages from the bag to build our list.
-                        putReuseBag(tailId, tail, pageAddr, io, bag, bucket) :
+                        putReuseBag(tailId, tailPage, tailAddr, io, bag, bucket) :
                         // Here we can use the data page to build list only if it is empty and
                         // it is being put into reuse bucket. Usually this will be true, but there is
                         // a case when there is no reuse bucket in the free list, but then deadlock
                         // on node page allocation from separate reuse list is impossible.
                         // If the data page is not empty it can not be put into reuse bucket and thus
                         // the deadlock is impossible as well.
-                        putDataPage(tailId, tail, pageAddr, io, dataPage, dataPageAddr, bucket);
+                        putDataPage(tailId, tailPage, tailAddr, io, dataId, dataPage, dataAddr, bucket);
 
                     if (ok) {
                         stripe.empty = false;
@@ -601,52 +634,55 @@ public abstract class PagesList extends DataStructure {
                     }
                 }
                 finally {
-                    writeUnlock(tail, pageAddr, ok);
+                    writeUnlock(tailId, tailPage, tailAddr, ok);
                 }
             }
+            finally {
+                releasePage(tailId, tailPage);
+            }
         }
     }
 
     /**
      * @param pageId Page ID.
-     * @param page Page.
+     * @param page Page pointer.
      * @param pageAddr Page address.
      * @param io IO.
-     * @param dataPage Data page.
-     * @param dataPageAddr Data page address.
+     * @param dataId Data page ID.
+     * @param dataPage Data page pointer.
+     * @param dataAddr Data page address.
      * @param bucket Bucket.
      * @return {@code true} If succeeded.
      * @throws IgniteCheckedException If failed.
      */
     private boolean putDataPage(
-        long pageId,
-        Page page,
-        long pageAddr,
+        final long pageId,
+        final long page,
+        final long pageAddr,
         PagesListNodeIO io,
-        Page dataPage,
-        long dataPageAddr,
+        final long dataId,
+        final long dataPage,
+        final long dataAddr,
         int bucket
     ) throws IgniteCheckedException {
         if (io.getNextId(pageAddr) != 0L)
             return false; // Splitted.
 
-        long dataPageId = dataPage.id();
-
-        int idx = io.addPage(pageAddr, dataPageId, pageSize());
+        int idx = io.addPage(pageAddr, dataId, pageSize());
 
         if (idx == -1)
-            handlePageFull(pageId, page, pageAddr, io, dataPage, dataPageAddr, bucket);
+            handlePageFull(pageId, page, pageAddr, io, dataId, dataPage, dataAddr, bucket);
         else {
             bucketsSize[bucket].incrementAndGet();
 
-            if (isWalDeltaRecordNeeded(wal, page))
-                wal.log(new PagesListAddPageRecord(cacheId, pageId, dataPageId));
+            if (needWalDeltaRecord(pageId, page, null))
+                wal.log(new PagesListAddPageRecord(cacheId, pageId, dataId));
 
-            DataPageIO dataIO = DataPageIO.VERSIONS.forPage(dataPageAddr);
-            dataIO.setFreeListPageId(dataPageAddr, pageId);
+            DataPageIO dataIO = DataPageIO.VERSIONS.forPage(dataAddr);
+            dataIO.setFreeListPageId(dataAddr, pageId);
 
-            if (isWalDeltaRecordNeeded(wal, dataPage))
-                wal.log(new DataPageSetFreeListPageRecord(cacheId, dataPage.id(), pageId));
+            if (needWalDeltaRecord(dataId, dataPage, null))
+                wal.log(new DataPageSetFreeListPageRecord(cacheId, dataId, pageId));
         }
 
         return true;
@@ -654,74 +690,76 @@ public abstract class PagesList extends DataStructure {
 
     /**
      * @param pageId Page ID.
-     * @param page Page.
+     * @param page Page pointer.
      * @param pageAddr Page address.
      * @param io IO.
-     * @param dataPage Data page.
-     * @param dataPageAddr Data page address.
+     * @param dataId Data page ID.
+     * @param data Data page pointer.
+     * @param dataAddr Data page address.
      * @param bucket Bucket index.
      * @throws IgniteCheckedException If failed.
-     */
+     * */
     private void handlePageFull(
-        long pageId,
-        Page page,
-        long pageAddr,
+        final long pageId,
+        final long page,
+        final long pageAddr,
         PagesListNodeIO io,
-        Page dataPage,
-        long dataPageAddr,
+        final long dataId,
+        final long data,
+        final long dataAddr,
         int bucket
     ) throws IgniteCheckedException {
-        long dataPageId = dataPage.id();
-        DataPageIO dataIO = DataPageIO.VERSIONS.forPage(dataPageAddr);
+        DataPageIO dataIO = DataPageIO.VERSIONS.forPage(dataAddr);
 
         // Attempt to add page failed: the node page is full.
         if (isReuseBucket(bucket)) {
             // If we are on the reuse bucket, we can not allocate new page, because it may cause deadlock.
-            assert dataIO.isEmpty(dataPageAddr); // We can put only empty data pages to reuse bucket.
+            assert dataIO.isEmpty(dataAddr); // We can put only empty data pages to reuse bucket.
 
             // Change page type to index and add it as next node page to this list.
-            dataPageId = PageIdUtils.changeType(dataPageId, FLAG_IDX);
+            long newDataId = PageIdUtils.changeType(dataId, FLAG_IDX);
 
-            setupNextPage(io, pageId, pageAddr, dataPageId, dataPageAddr);
+            setupNextPage(io, pageId, pageAddr, newDataId, dataAddr);
 
-            if (isWalDeltaRecordNeeded(wal, page))
-                wal.log(new PagesListSetNextRecord(cacheId, pageId, dataPageId));
+            if (needWalDeltaRecord(pageId, page, null))
+                wal.log(new PagesListSetNextRecord(cacheId, pageId, newDataId));
 
-            if (isWalDeltaRecordNeeded(wal, dataPage))
+            if (needWalDeltaRecord(dataId, data, null))
                 wal.log(new PagesListInitNewPageRecord(
                     cacheId,
-                    dataPageId,
+                    dataId,
                     io.getType(),
                     io.getVersion(),
-                    dataPageId,
+                    newDataId,
                     pageId, 0L));
 
             // In reuse bucket the page itself can be used as a free page.
             bucketsSize[bucket].incrementAndGet();
 
-            updateTail(bucket, pageId, dataPageId);
+            updateTail(bucket, pageId, newDataId);
         }
         else {
             // Just allocate a new node page and add our data page there.
-            long nextId = allocatePage(null);
+            final long nextId = allocatePage(null);
+            final long nextPage = acquirePage(nextId);
 
-            try (Page next = page(nextId)) {
-                long nextPageAddr = writeLock(next); // Newly allocated page.
+            try {
+                long nextPageAddr = writeLock(nextId, nextPage); // Newly allocated page.
 
                 assert nextPageAddr != 0L;
 
+                // Here we should never write full page, because it is known to be new.
+                Boolean nextWalPlc = FALSE;
+
                 try {
                     setupNextPage(io, pageId, pageAddr, nextId, nextPageAddr);
 
-                    if (isWalDeltaRecordNeeded(wal, page))
+                    if (needWalDeltaRecord(pageId, page, null))
                         wal.log(new PagesListSetNextRecord(cacheId, pageId, nextId));
 
-                    int idx = io.addPage(nextPageAddr, dataPageId, pageSize());
+                    int idx = io.addPage(nextPageAddr, dataId, pageSize());
 
-                    // Here we should never write full page, because it is known to be new.
-                    next.fullPageWalRecordPolicy(FALSE);
-
-                    if (isWalDeltaRecordNeeded(wal, next))
+                    if (needWalDeltaRecord(nextId, nextPage, nextWalPlc))
                         wal.log(new PagesListInitNewPageRecord(
                             cacheId,
                             nextId,
@@ -729,30 +767,33 @@ public abstract class PagesList extends DataStructure {
                             io.getVersion(),
                             nextId,
                             pageId,
-                            dataPageId
+                            dataId
                         ));
 
                     assert idx != -1;
 
-                    dataIO.setFreeListPageId(dataPageAddr, nextId);
+                    dataIO.setFreeListPageId(dataAddr, nextId);
 
-                    if (isWalDeltaRecordNeeded(wal, dataPage))
-                        wal.log(new DataPageSetFreeListPageRecord(cacheId, dataPageId, nextId));
+                    if (needWalDeltaRecord(dataId, data, null))
+                        wal.log(new DataPageSetFreeListPageRecord(cacheId, dataId, nextId));
 
                     bucketsSize[bucket].incrementAndGet();
 
                     updateTail(bucket, pageId, nextId);
                 }
                 finally {
-                    writeUnlock(next, nextPageAddr, true);
+                    writeUnlock(nextId, nextPage, nextPageAddr, nextWalPlc, true);
                 }
             }
+            finally {
+                releasePage(nextId, nextPage);
+            }
         }
     }
 
     /**
      * @param pageId Page ID.
-     * @param page Page.
+     * @param page Page pointer.
      * @param pageAddr Page address.
      * @param io IO.
      * @param bag Reuse bag.
@@ -763,7 +804,7 @@ public abstract class PagesList extends DataStructure {
     @SuppressWarnings("ForLoopReplaceableByForEach")
     private boolean putReuseBag(
         final long pageId,
-        Page page,
+        final long page,
         final long pageAddr,
         PagesListNodeIO io,
         ReuseBag bag,
@@ -773,39 +814,41 @@ public abstract class PagesList extends DataStructure {
             return false; // Splitted.
 
         long nextId;
-        long prevPageAddr = pageAddr;
+
         long prevId = pageId;
+        long prevPage = page;
+        long prevAddr = pageAddr;
 
-        List<Page> locked = null; // TODO may be unlock right away and do not keep all these pages locked?
-        List<Long> lockedAddrs = null;
+        Boolean walPlc = null;
+
+        List<long[]> locked = null; // TODO may be unlock right away and do not keep all these pages locked?
 
         try {
             while ((nextId = bag.pollFreePage()) != 0L) {
-                int idx = io.addPage(prevPageAddr, nextId, pageSize());
+                int idx = io.addPage(prevAddr, nextId, pageSize());
 
                 if (idx == -1) { // Attempt to add page failed: the node page is full.
-                    try (Page next = page(nextId)) {
-                        long nextPageAddr = writeLock(next); // Page from reuse bag can't be concurrently recycled.
+
+                    final long nextPage = acquirePage(nextId);
+
+                    try {
+                        long nextPageAddr = writeLock(nextId, nextPage); // Page from reuse bag can't be concurrently recycled.
 
                         assert nextPageAddr != 0L;
 
                         if (locked == null) {
-                            lockedAddrs = new ArrayList<>(2);
                             locked = new ArrayList<>(2);
                         }
 
-                        locked.add(next);
-                        lockedAddrs.add(nextPageAddr);
+                        locked.add(new long[]{nextId, nextPage, nextPageAddr});
 
-                        setupNextPage(io, prevId, prevPageAddr, nextId, nextPageAddr);
+                        setupNextPage(io, prevId, prevAddr, nextId, nextPageAddr);
 
-                        if (isWalDeltaRecordNeeded(wal, page))
+                        if (needWalDeltaRecord(prevId, prevPage, walPlc))
                             wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
 
                         // Here we should never write full page, because it is known to be new.
-                        next.fullPageWalRecordPolicy(FALSE);
-
-                        if (isWalDeltaRecordNeeded(wal, next))
+                        if (needWalDeltaRecord(nextId, nextPage, walPlc = FALSE))
                             wal.log(new PagesListInitNewPageRecord(
                                 cacheId,
                                 nextId,
@@ -822,14 +865,17 @@ public abstract class PagesList extends DataStructure {
 
                         // Switch to this new page, which is now a part of our list
                         // to add the rest of the bag to the new page.
-                        prevPageAddr = nextPageAddr;
+                        prevAddr = nextPageAddr;
                         prevId = nextId;
-                        page = next;
+                        prevPage = nextPage;
+                    }
+                    finally {
+                        releasePage(nextId, nextPage);
                     }
                 }
                 else {
                     // TODO: use single WAL record for bag?
-                    if (isWalDeltaRecordNeeded(wal, page))
+                    if (needWalDeltaRecord(prevId, prevPage, walPlc))
                         wal.log(new PagesListAddPageRecord(cacheId, prevId, nextId));
 
                     bucketsSize[bucket].incrementAndGet();
@@ -842,8 +888,10 @@ public abstract class PagesList extends DataStructure {
                 updateTail(bucket, pageId, prevId);
 
                 // Release write.
-                for (int i = 0; i < locked.size(); i++)
-                    writeUnlock(locked.get(i), lockedAddrs.get(i), true);
+                for (int i = 0; i < locked.size(); i++) {
+                    long[] vals = locked.get(i);
+                    writeUnlock(vals[0], vals[1], vals[2], FALSE, true);
+                }
             }
         }
 
@@ -876,15 +924,16 @@ public abstract class PagesList extends DataStructure {
     }
 
     /**
-     * @param page Page.
+     * @param pageId Page ID.
+     * @param page Page pointer.
      * @param bucket Bucket.
      * @param lockAttempt Lock attempts counter.
      * @return Page address if page is locked of {@code null} if can retry lock.
      * @throws IgniteCheckedException If failed.
      */
-    private long writeLockPage(Page page, int bucket, int lockAttempt)
+    private long writeLockPage(long pageId, long page, int bucket, int lockAttempt)
         throws IgniteCheckedException {
-        long pageAddr = tryWriteLock(page);
+        long pageAddr = tryWriteLock(pageId, page);
 
         if (pageAddr != 0L)
             return pageAddr;
@@ -900,7 +949,7 @@ public abstract class PagesList extends DataStructure {
             }
         }
 
-        return lockAttempt < TRY_LOCK_ATTEMPTS ? 0L : writeLock(page); // Must be explicitly checked further.
+        return lockAttempt < TRY_LOCK_ATTEMPTS ? 0L : writeLock(pageId, page); // Must be explicitly checked further.
     }
 
     /**
@@ -929,12 +978,13 @@ public abstract class PagesList extends DataStructure {
             if (stripe == null)
                 return 0L;
 
-            long tailId = stripe.tailId;
+            final long tailId = stripe.tailId;
+            final long tailPage = acquirePage(tailId);
 
-            try (Page tail = page(tailId)) {
-                long tailPageAddr = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+            try {
+                long tailAddr = writeLockPage(tailId, tailPage, bucket, lockAttempt++); // Explicit check.
 
-                if (tailPageAddr == 0L) {
+                if (tailAddr == 0L) {
                     if (isReuseBucket(bucket) && lockAttempt == TRY_LOCK_ATTEMPTS)
                         addStripeForReuseBucket(bucket);
 
@@ -943,7 +993,7 @@ public abstract class PagesList extends DataStructure {
 
                 if (stripe.empty) {
                     // Another thread took the last page.
-                    writeUnlock(tail, tailPageAddr, false);
+                    writeUnlock(tailId, tailPage, tailAddr, false);
 
                     if (bucketsSize[bucket].get() > 0) {
                         lockAttempt--; // Ignore current attempt.
@@ -954,48 +1004,45 @@ public abstract class PagesList extends DataStructure {
                         return 0L;
                 }
 
-                assert PageIO.getPageId(tailPageAddr) == tailId : "tailId = " + tailId + ", tailPageId = " + PageIO.getPageId(tailPageAddr);
-                assert PageIO.getType(tailPageAddr) == PageIO.T_PAGE_LIST_NODE;
+                assert PageIO.getPageId(tailAddr) == tailId : "tailId = " + tailId + ", tailPageId = " + PageIO.getPageId(tailAddr);
+                assert PageIO.getType(tailAddr) == PageIO.T_PAGE_LIST_NODE;
 
                 boolean dirty = false;
                 long ret;
                 long recycleId = 0L;
 
                 try {
-                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(tailPageAddr);
+                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(tailAddr);
 
-                    if (io.getNextId(tailPageAddr) != 0) {
+                    if (io.getNextId(tailAddr) != 0) {
                         // It is not a tail anymore, retry.
                         continue;
                     }
 
-                    long pageId = io.takeAnyPage(tailPageAddr);
+                    long pageId = io.takeAnyPage(tailAddr);
 
                     if (pageId != 0L) {
                         bucketsSize[bucket].decrementAndGet();
 
-                        if (isWalDeltaRecordNeeded(wal, tail))
+                        if (needWalDeltaRecord(tailId, tailPage, null))
                             wal.log(new PagesListRemovePageRecord(cacheId, tailId, pageId));
 
                         dirty = true;
 
                         ret = pageId;
 
-                        if (io.isEmpty(tailPageAddr)) {
-                            long prevId = io.getPreviousId(tailPageAddr);
+                        if (io.isEmpty(tailAddr)) {
+                            long prevId = io.getPreviousId(tailAddr);
 
                             // If we got an empty page in non-reuse bucket, move it back to reuse list
                             // to prevent empty page leak to data pages.
                             if (!isReuseBucket(bucket)) {
                                 if (prevId != 0L) {
-                                    try (Page prev = page(prevId)) {
-                                        // Lock pages from next to previous.
-                                        Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
+                                    Boolean ok = write(prevId, cutTail, null, bucket, FALSE);
 
-                                        assert ok == TRUE : ok;
-                                    }
+                                    assert ok == TRUE : ok;
 
-                                    recycleId = recyclePage(tailId, tail, tailPageAddr);
+                                    recycleId = recyclePage(tailId, tailPage, tailAddr, null);
                                 }
                                 else
                                     stripe.empty = true;
@@ -1010,37 +1057,32 @@ public abstract class PagesList extends DataStructure {
                         // a previous page, so, the current page can be collected
                         assert isReuseBucket(bucket);
 
-                        long prevId = io.getPreviousId(tailPageAddr);
+                        long prevId = io.getPreviousId(tailAddr);
 
                         assert prevId != 0L;
 
-                        try (Page prev = page(prevId)) {
-                            // Lock pages from next to previous.
-                            Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
+                        Boolean ok = write(prevId, cutTail, bucket, FALSE);
 
-                            assert ok == TRUE : ok;
+                        assert ok == TRUE : ok;
 
-                            bucketsSize[bucket].decrementAndGet();
-                        }
+                        bucketsSize[bucket].decrementAndGet();
 
                         if (initIoVers != null) {
-                            tailId = PageIdUtils.changeType(tailId, FLAG_DATA);
+                            ret = PageIdUtils.changeType(tailId, FLAG_DATA);
 
                             PageIO initIo = initIoVers.latest();
 
-                            initIo.initNewPage(tailPageAddr, tailId, pageSize());
+                            initIo.initNewPage(tailAddr, ret, pageSize());
 
-                            if (isWalDeltaRecordNeeded(wal, tail)) {
-                                wal.log(new InitNewPageRecord(cacheId, tail.id(), initIo.getType(),
-                                    initIo.getVersion(), tailId));
+                            if (needWalDeltaRecord(tailId, tailPage, null)) {
+                                wal.log(new InitNewPageRecord(cacheId, tailId, initIo.getType(),
+                                    initIo.getVersion(), ret));
                             }
                         }
                         else
-                            tailId = recyclePage(tailId, tail, tailPageAddr);
+                            ret = recyclePage(tailId, tailPage, tailAddr, null);
 
                         dirty = true;
-
-                        ret = tailId;
                     }
 
                     // If we do not have a previous page (we are at head), then we still can return
@@ -1049,7 +1091,7 @@ public abstract class PagesList extends DataStructure {
                     // meta page.
                 }
                 finally {
-                    writeUnlock(tail, tailPageAddr, dirty);
+                    writeUnlock(tailId, tailPage, tailAddr, dirty);
                 }
 
                 // Put recycled page (if any) to the reuse bucket after tail is unlocked.
@@ -1061,31 +1103,39 @@ public abstract class PagesList extends DataStructure {
 
                 return ret;
             }
+            finally {
+                releasePage(tailId, tailPage);
+            }
         }
     }
 
     /**
-     * @param dataPage Data page.
-     * @param dataPageAddr Data page address.
+     * @param dataId Data page ID.
+     * @param dataPage Data page pointer.
+     * @param dataAddr Data page address.
      * @param dataIO Data page IO.
      * @param bucket Bucket index.
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if page was removed.
      */
-    protected final boolean removeDataPage(Page dataPage, long dataPageAddr, DataPageIO dataIO, int bucket)
+    protected final boolean removeDataPage(
+        final long dataId,
+        final long dataPage,
+        final long dataAddr,
+        DataPageIO dataIO,
+        int bucket)
         throws IgniteCheckedException {
-        long dataPageId = dataPage.id();
-
-        long pageId = dataIO.getFreeListPageId(dataPageAddr);
+        final long pageId = dataIO.getFreeListPageId(dataAddr);
 
         assert pageId != 0;
 
-        try (Page page = page(pageId)) {
+        final long page = acquirePage(pageId);
+        try {
             long nextId;
 
             long recycleId = 0L;
 
-            long pageAddr = writeLock(page); // Explicit check.
+            long pageAddr = writeLock(pageId, page); // Explicit check.
 
             if (pageAddr == 0L)
                 return false;
@@ -1095,21 +1145,21 @@ public abstract class PagesList extends DataStructure {
             try {
                 PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                rmvd = io.removePage(pageAddr, dataPageId);
+                rmvd = io.removePage(pageAddr, dataId);
 
                 if (!rmvd)
                     return false;
 
                 bucketsSize[bucket].decrementAndGet();
 
-                if (isWalDeltaRecordNeeded(wal, page))
-                    wal.log(new PagesListRemovePageRecord(cacheId, pageId, dataPageId));
+                if (needWalDeltaRecord(pageId, page, null))
+                    wal.log(new PagesListRemovePageRecord(cacheId, pageId, dataId));
 
                 // Reset free list page ID.
-                dataIO.setFreeListPageId(dataPageAddr, 0L);
+                dataIO.setFreeListPageId(dataAddr, 0L);
 
-                if (isWalDeltaRecordNeeded(wal, dataPage))
-                    wal.log(new DataPageSetFreeListPageRecord(cacheId, dataPageId, 0L));
+                if (needWalDeltaRecord(dataId, dataPage, null))
+                    wal.log(new DataPageSetFreeListPageRecord(cacheId, dataId, 0L));
 
                 if (!io.isEmpty(pageAddr))
                     return true; // In optimistic case we still have something in the page and can leave it as is.
@@ -1126,7 +1176,7 @@ public abstract class PagesList extends DataStructure {
                 }
             }
             finally {
-                writeUnlock(page, pageAddr, rmvd);
+                writeUnlock(pageId, page, pageAddr, rmvd);
             }
 
             // Perform a fair merge after lock release (to have a correct locking order).
@@ -1138,65 +1188,40 @@ public abstract class PagesList extends DataStructure {
 
             return true;
         }
-    }
-
-    /**
-     * @param page Page.
-     * @param pageId Page ID.
-     * @param pageAddr Page address.
-     * @param prevId Previous page ID.
-     * @param bucket Bucket index.
-     * @return Page ID to recycle.
-     * @throws IgniteCheckedException If failed.
-     */
-    private long mergeNoNext(long pageId, Page page, long pageAddr, long prevId, int bucket)
-        throws IgniteCheckedException {
-        // If we do not have a next page (we are tail) and we are on reuse bucket,
-        // then we can leave as is as well, because it is normal to have an empty tail page here.
-        if (isReuseBucket(bucket))
-            return 0L;
-
-        if (prevId != 0L) { // Cut tail if we have a previous page.
-            try (Page prev = page(prevId)) {
-                Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
-
-                assert ok == TRUE: ok; // Because we keep lock on current tail and do a world consistency check.
-            }
-        }
-        else {
-            // If we don't have a previous, then we are tail page of free list, just drop the stripe.
-            boolean rmvd = updateTail(bucket, pageId, 0L);
-
-            if (!rmvd)
-                return 0L;
+        finally {
+            releasePage(pageId, page);
         }
-
-        return recyclePage(pageId, page, pageAddr);
     }
 
     /**
      * @param pageId Page ID.
-     * @param page Page.
+     * @param page Page pointer.
      * @param nextId Next page ID.
      * @param bucket Bucket index.
      * @return Page ID to recycle.
      * @throws IgniteCheckedException If failed.
      */
-    private long merge(long pageId, Page page, long nextId, int bucket)
+    private long merge(
+        final long pageId,
+        final long page,
+        long nextId,
+        int bucket)
         throws IgniteCheckedException {
         assert nextId != 0; // We should do mergeNoNext then.
 
         // Lock all the pages in correct order (from next to previous) and do the merge in retry loop.
         for (;;) {
-            try (Page next = nextId == 0L ? null : page(nextId)) {
+            final long curId = nextId;
+            final long curPage = curId == 0L ? 0L : acquirePage(curId);
+            try {
                 boolean write = false;
 
-                long nextPageAddr = next == null ? 0L : writeLock(next); // Explicit check.
-                long pageAddr = writeLock(page); // Explicit check.
+                final long curAddr = curPage == 0L ? 0L : writeLock(curId, curPage); // Explicit check.
+                final long pageAddr = writeLock(pageId, page); // Explicit check.
 
                 if (pageAddr == 0L) {
-                    if (nextPageAddr != 0L) // Unlock next page if needed.
-                        writeUnlock(next, nextPageAddr, false);
+                    if (curAddr != 0L) // Unlock next page if needed.
+                        writeUnlock(curId, curPage, curAddr, false);
 
                     return 0L; // Someone has merged or taken our empty page concurrently. Nothing to do here.
                 }
@@ -1208,8 +1233,8 @@ public abstract class PagesList extends DataStructure {
                         return 0L; // No need to merge anymore.
 
                     // Check if we see a consistent state of the world.
-                    if (io.getNextId(pageAddr) == nextId && (nextId == 0L) == (nextPageAddr == 0L)) {
-                        long recycleId = doMerge(pageId, page, pageAddr, io, next, nextId, nextPageAddr, bucket);
+                    if (io.getNextId(pageAddr) == curId && (curId == 0L) == (curAddr == 0L)) {
+                        long recycleId = doMerge(pageId, page, pageAddr, io, curId, curPage, curAddr, bucket);
 
                         write = true;
 
@@ -1220,35 +1245,76 @@ public abstract class PagesList extends DataStructure {
                     nextId = io.getNextId(pageAddr);
                 }
                 finally {
-                    if (nextPageAddr != 0L)
-                        writeUnlock(next, nextPageAddr, write);
+                    if (curAddr != 0L)
+                        writeUnlock(curId, curPage, curAddr, write);
 
-                    writeUnlock(page, pageAddr, write);
+                    writeUnlock(pageId, page, pageAddr, write);
                 }
             }
+            finally {
+                if(curPage != 0L)
+                    releasePage(curId, curPage);
+            }
         }
     }
 
     /**
-     * @param page Page.
      * @param pageId Page ID.
-     * @param io IO.
+     * @param page Page pointer.
      * @param pageAddr Page address.
-     * @param next Next page.
+     * @param prevId Previous page ID.
+     * @param bucket Bucket index.
+     * @return Page ID to recycle.
+     * @throws IgniteCheckedException If failed.
+     */
+    private long mergeNoNext(
+        long pageId,
+        long page,
+        long pageAddr,
+        long prevId,
+        int bucket)
+        throws IgniteCheckedException {
+        // If we do not have a next page (we are tail) and we are on reuse bucket,
+        // then we can leave as is as well, because it is normal to have an empty tail page here.
+        if (isReuseBucket(bucket))
+            return 0L;
+
+        if (prevId != 0L) { // Cut tail if we have a previous page.
+            Boolean ok = write(prevId, cutTail, null, bucket, FALSE);
+
+            assert ok == TRUE: ok;
+        }
+        else {
+            // If we don't have a previous, then we are tail page of free list, just drop the stripe.
+            boolean rmvd = updateTail(bucket, pageId, 0L);
+
+            if (!rmvd)
+                return 0L;
+        }
+
+        return recyclePage(pageId, page, pageAddr, null);
+    }
+
+    /**
+     * @param pageId Page ID.
+     * @param page Page absolute pointer.
+     * @param pageAddr Page address.
+     * @param io IO.
      * @param nextId Next page ID.
-     * @param nextPageAddr Next page address.
+     * @param nextPage Next page absolute pointer.
+     * @param nextAddr Next page address.
      * @param bucket Bucket index.
      * @return Page to recycle.
      * @throws IgniteCheckedException If failed.
      */
     private long doMerge(
         long pageId,
-        Page page,
+        long page,
         long pageAddr,
         PagesListNodeIO io,
-        Page next,
         long nextId,
-        long nextPageAddr,
+        long nextPage,
+        long nextAddr,
         int bucket
     ) throws IgniteCheckedException {
         long prevId = io.getPreviousId(pageAddr);
@@ -1257,86 +1323,71 @@ public abstract class PagesList extends DataStructure {
             return mergeNoNext(pageId, page, pageAddr, prevId, bucket);
         else {
             // No one must be able to merge it while we keep a reference.
-            assert getPageId(nextPageAddr) == nextId;
+            assert getPageId(nextAddr) == nextId;
 
             if (prevId == 0L) { // No previous page: we are at head.
                 // These references must be updated at the same time in write locks.
-                assert PagesListNodeIO.VERSIONS.forPage(nextPageAddr).getPreviousId(nextPageAddr) == pageId;
+                assert PagesListNodeIO.VERSIONS.forPage(nextAddr).getPreviousId(nextAddr) == pageId;
 
-                PagesListNodeIO nextIO = PagesListNodeIO.VERSIONS.forPage(nextPageAddr);
-                nextIO.setPreviousId(nextPageAddr, 0);
+                PagesListNodeIO nextIO = PagesListNodeIO.VERSIONS.forPage(nextAddr);
+                nextIO.setPreviousId(nextAddr, 0);
 
-                if (isWalDeltaRecordNeeded(wal, next))
+                if (needWalDeltaRecord(nextId, nextPage, null))
                     wal.log(new PagesListSetPreviousRecord(cacheId, nextId, 0L));
             }
             else // Do a fair merge: link previous and next to each other.
-                fairMerge(prevId, pageId, nextId, next, nextPageAddr);
+                fairMerge(prevId, pageId, nextId, nextPage, nextAddr);
 
-            return recyclePage(pageId, page, pageAddr);
+            return recyclePage(pageId, page, pageAddr, null);
         }
     }
 
     /**
      * Link previous and next to each other.
-     *
      * @param prevId Previous Previous page ID.
      * @param pageId Page ID.
-     * @param next Next page.
      * @param nextId Next page ID.
-     * @param nextPageAddr Next page address.
+     * @param nextPage Next page absolute pointer.
+     * @param nextAddr Next page address.
      * @throws IgniteCheckedException If failed.
      */
-    private void fairMerge(long prevId,
+    private void fairMerge(
+        final long prevId,
         long pageId,
         long nextId,
-        Page next,
-        long nextPageAddr)
+        long nextPage,
+        long nextAddr)
         throws IgniteCheckedException {
-        try (Page prev = page(prevId)) {
-            long prevPageAddr = writeLock(prev); // No check, we keep a reference.
-
-            assert prevPageAddr != 0L;
+        long prevPage = acquirePage(prevId);
 
+        try {
+            final long prevAddr = writeLock(prevId, prevPage); // No check, we keep a reference.
+            assert prevAddr != 0L;
             try {
-                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevPageAddr);
-                PagesListNodeIO nextIO = PagesListNodeIO.VERSIONS.forPage(nextPageAddr);
+                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevAddr);
+                PagesListNodeIO nextIO = PagesListNodeIO.VERSIONS.forPage(nextAddr);
 
                 // These references must be updated at the same time in write locks.
-                assert prevIO.getNextId(prevPageAddr) == pageId;
-                assert nextIO.getPreviousId(nextPageAddr) == pageId;
+                assert prevIO.getNextId(prevAddr) == pageId;
+                assert nextIO.getPreviousId(nextAddr) == pageId;
 
-                prevIO.setNextId(prevPageAddr, nextId);
+                prevIO.setNextId(prevAddr, nextId);
 
-                if (isWalDeltaRecordNeeded(wal, prev))
+                if (needWalDeltaRecord(prevId, prevPage, null))
                     wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
 
-                nextIO.setPreviousId(nextPageAddr, prevId);
+                nextIO.setPreviousId(nextAddr, prevId);
 
-                if (isWalDeltaRecordNeeded(wal, next))
+                if (needWalDeltaRecord(nextId, nextPage, null))
                     wal.log(new PagesListSetPreviousRecord(cacheId, nextId, prevId));
             }
             finally {
-                writeUnlock(prev, prevPageAddr, true);
+                writeUnlock(prevId, prevPage, prevAddr, true);
             }
         }
-    }
-
-    /**
-     * @param page Page.
-     * @param pageId Page ID.
-     * @param pageAddr Page address.
-     * @return Rotated page ID.
-     * @throws IgniteCheckedException If failed.
-     */
-    private long recyclePage(long pageId, Page page, long pageAddr) throws IgniteCheckedException {
-        pageId = PageIdUtils.rotatePageId(pageId);
-
-        PageIO.setPageId(pageAddr, pageId);
-
-        if (isWalDeltaRecordNeeded(wal, page))
-            wal.log(new RecycleRecord(cacheId, page.id(), pageId));
-
-        return pageId;
+        finally {
+            releasePage(prevId, prevPage);
+        }
     }
 
     /**