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/01/17 11:52:12 UTC

[5/7] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search row

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/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 4da44bc..e44838d 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
@@ -86,17 +86,17 @@ public abstract class PagesList extends DataStructure {
 
     /** */
     private final PageHandler<Void, Boolean> cutTail = new PageHandler<Void, Boolean>() {
-        @Override public Boolean run(Page page, PageIO pageIo, ByteBuffer buf, Void ignore, int bucket)
+        @Override public Boolean run(Page page, PageIO pageIo, long pageAddr, Void ignore, int bucket)
             throws IgniteCheckedException {
-            assert getPageId(buf) == page.id();
+            assert getPageId(pageAddr) == page.id();
 
             PagesListNodeIO io = (PagesListNodeIO)pageIo;
 
-            long tailId = io.getNextId(buf);
+            long tailId = io.getNextId(pageAddr);
 
             assert tailId != 0;
 
-            io.setNextId(buf, 0L);
+            io.setNextId(pageAddr, 0L);
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new PagesListSetNextRecord(cacheId, page.id(), 0L));
@@ -140,7 +140,7 @@ public abstract class PagesList extends DataStructure {
         if (metaPageId != 0L) {
             if (initNew) {
                 try (Page page = page(metaPageId)) {
-                    initPage(page, this, PagesListMetaIO.VERSIONS.latest(), wal);
+                    initPage(pageMem, page, this, PagesListMetaIO.VERSIONS.latest(), wal);
                 }
             }
             else {
@@ -150,16 +150,16 @@ public abstract class PagesList extends DataStructure {
 
                 while (nextPageId != 0) {
                     try (Page page = page(nextPageId)) {
-                        ByteBuffer buf = readLock(page); // No concurrent recycling on init.
+                        long pageAddr = readLock(page); // No concurrent recycling on init.
 
-                        assert buf != null;
+                        assert pageAddr != 0L;
 
                         try {
-                            PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+                            PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-                            io.getBucketsData(buf, bucketsData);
+                            io.getBucketsData(pageAddr, bucketsData);
 
-                            long next0 = io.getNextMetaPageId(buf);
+                            long next0 = io.getNextMetaPageId(pageAddr);
 
                             assert next0 != nextPageId :
                                 "Loop detected [next=" + U.hexLong(next0) + ", cur=" + U.hexLong(nextPageId) + ']';
@@ -167,7 +167,7 @@ public abstract class PagesList extends DataStructure {
                             nextPageId = next0;
                         }
                         finally {
-                            readUnlock(page, buf);
+                            readUnlock(page, pageAddr);
                         }
                     }
                 }
@@ -199,7 +199,7 @@ public abstract class PagesList extends DataStructure {
         assert metaPageId != 0;
 
         Page curPage = null;
-        ByteBuffer curBuf = null;
+        long curPageAddr = 0L;
         PagesListMetaIO curIo = null;
 
         long nextPageId = metaPageId;
@@ -212,39 +212,39 @@ public abstract class PagesList extends DataStructure {
                     int tailIdx = 0;
 
                     while (tailIdx < tails.length) {
-                        int written = curPage != null ? curIo.addTails(curBuf, bucket, tails, tailIdx) : 0;
+                        int written = curPage != null ? curIo.addTails(pageMem.pageSize(), curPageAddr, bucket, tails, tailIdx) : 0;
 
                         if (written == 0) {
                             if (nextPageId == 0L) {
                                 nextPageId = allocatePageNoReuse();
 
                                 if (curPage != null) {
-                                    curIo.setNextMetaPageId(curBuf, nextPageId);
+                                    curIo.setNextMetaPageId(curPageAddr, nextPageId);
 
-                                    releaseAndClose(curPage, curBuf);
+                                    releaseAndClose(curPage, curPageAddr);
                                     curPage = null;
                                 }
 
                                 curPage = page(nextPageId);
-                                curBuf = writeLock(curPage);
+                                curPageAddr = writeLock(curPage);
 
                                 curIo = PagesListMetaIO.VERSIONS.latest();
 
-                                curIo.initNewPage(curBuf, nextPageId);
+                                curIo.initNewPage(curPageAddr, nextPageId, pageSize());
                             }
                             else {
-                                releaseAndClose(curPage, curBuf);
+                                releaseAndClose(curPage, curPageAddr);
                                 curPage = null;
 
                                 curPage = page(nextPageId);
-                                curBuf = writeLock(curPage);
+                                curPageAddr = writeLock(curPage);
 
-                                curIo = PagesListMetaIO.VERSIONS.forPage(curBuf);
+                                curIo = PagesListMetaIO.VERSIONS.forPage(curPageAddr);
 
-                                curIo.resetCount(curBuf);
+                                curIo.resetCount(curPageAddr);
                             }
 
-                            nextPageId = curIo.getNextMetaPageId(curBuf);
+                            nextPageId = curIo.getNextMetaPageId(curPageAddr);
                         }
                         else
                             tailIdx += written;
@@ -253,25 +253,25 @@ public abstract class PagesList extends DataStructure {
             }
         }
         finally {
-            releaseAndClose(curPage, curBuf);
+            releaseAndClose(curPage, curPageAddr);
         }
 
         while (nextPageId != 0L) {
             try (Page page = page(nextPageId)) {
-                ByteBuffer buf = writeLock(page);
+                long pageAddr = writeLock(page);
 
                 try {
-                    PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+                    PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-                    io.resetCount(buf);
+                    io.resetCount(pageAddr);
 
                     if (PageHandler.isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PageListMetaResetCountRecord(cacheId, nextPageId));
 
-                    nextPageId = io.getNextMetaPageId(buf);
+                    nextPageId = io.getNextMetaPageId(pageAddr);
                 }
                 finally {
-                    writeUnlock(page, buf, true);
+                    writeUnlock(page, pageAddr, true);
                 }
             }
         }
@@ -281,7 +281,7 @@ public abstract class PagesList extends DataStructure {
      * @param page Page.
      * @param buf Buffer.
      */
-    private void releaseAndClose(Page page, ByteBuffer buf) {
+    private void releaseAndClose(Page page, long buf) {
         if (page != null) {
             try {
                 // No special WAL record because we most likely changed the whole page.
@@ -322,10 +322,10 @@ public abstract class PagesList extends DataStructure {
      * @param nextId Next page ID.
      * @param next Next page buffer.
      */
-    private void setupNextPage(PagesListNodeIO io, long prevId, ByteBuffer prev, long nextId, ByteBuffer next) {
+    private void setupNextPage(PagesListNodeIO io, long prevId, long prev, long nextId, long next) {
         assert io.getNextId(prev) == 0L;
 
-        io.initNewPage(next, nextId);
+        io.initNewPage(next, nextId, pageSize());
         io.setPreviousId(next, prevId);
 
         io.setNextId(prev, nextId);
@@ -342,7 +342,7 @@ public abstract class PagesList extends DataStructure {
         long pageId = reuse ? allocatePage(null) : allocatePageNoReuse();
 
         try (Page page = page(pageId)) {
-            initPage(page, this, PagesListNodeIO.VERSIONS.latest(), wal);
+            initPage(pageMem, page, this, PagesListNodeIO.VERSIONS.latest(), wal);
         }
 
         Stripe stripe = new Stripe(pageId);
@@ -466,19 +466,19 @@ public abstract class PagesList extends DataStructure {
                 long pageId = tail.tailId;
 
                 try (Page page = page(pageId)) {
-                    ByteBuffer buf = readLock(page); // No correctness guaranties.
+                    long pageAddr = readLock(page); // No correctness guaranties.
 
                     try {
-                        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                        int cnt = io.getCount(buf);
+                        int cnt = io.getCount(pageAddr);
 
                         assert cnt >= 0;
 
                         res += cnt;
                     }
                     finally {
-                        readUnlock(page, buf);
+                        readUnlock(page, pageAddr);
                     }
                 }
             }
@@ -493,9 +493,9 @@ public abstract class PagesList extends DataStructure {
      * @param bucket Bucket.
      * @throws IgniteCheckedException If failed.
      */
-    protected final void put(ReuseBag bag, Page dataPage, ByteBuffer dataPageBuf, int bucket)
+    protected final void put(ReuseBag bag, Page dataPage, long dataPageBuf, int bucket)
         throws IgniteCheckedException {
-        assert bag == null ^ dataPageBuf == null;
+        assert bag == null ^ dataPageBuf == 0L;
 
         for (int lockAttempt = 0; ;) {
             Stripe stripe = getPageForPut(bucket);
@@ -503,9 +503,9 @@ public abstract class PagesList extends DataStructure {
             long tailId = stripe.tailId;
 
             try (Page tail = page(tailId)) {
-                ByteBuffer buf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+                long buf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
 
-                if (buf == null)
+                if (buf == 0L)
                     continue;
 
                 assert PageIO.getPageId(buf) == tailId : "bufPageId = " + PageIO.getPageId(buf) + ", tailId = " + tailId;
@@ -551,10 +551,10 @@ public abstract class PagesList extends DataStructure {
     private boolean putDataPage(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page dataPage,
-        ByteBuffer dataPageBuf,
+        long dataPageBuf,
         int bucket
     ) throws IgniteCheckedException {
         if (io.getNextId(buf) != 0L)
@@ -562,7 +562,7 @@ public abstract class PagesList extends DataStructure {
 
         long dataPageId = dataPage.id();
 
-        int idx = io.addPage(buf, dataPageId);
+        int idx = io.addPage(buf, dataPageId, pageSize());
 
         if (idx == -1)
             handlePageFull(pageId, page, buf, io, dataPage, dataPageBuf, bucket);
@@ -593,10 +593,10 @@ public abstract class PagesList extends DataStructure {
     private void handlePageFull(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page dataPage,
-        ByteBuffer dataPageBuf,
+        long dataPageBuf,
         int bucket
     ) throws IgniteCheckedException {
         long dataPageId = dataPage.id();
@@ -631,17 +631,17 @@ public abstract class PagesList extends DataStructure {
             long nextId = allocatePage(null);
 
             try (Page next = page(nextId)) {
-                ByteBuffer nextBuf = writeLock(next); // Newly allocated page.
+                long nextPageAddr = writeLock(next); // Newly allocated page.
 
-                assert nextBuf != null;
+                assert nextPageAddr != 0L;
 
                 try {
-                    setupNextPage(io, pageId, buf, nextId, nextBuf);
+                    setupNextPage(io, pageId, buf, nextId, nextPageAddr);
 
                     if (isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PagesListSetNextRecord(cacheId, pageId, nextId));
 
-                    int idx = io.addPage(nextBuf, dataPageId);
+                    int idx = io.addPage(nextPageAddr, dataPageId, pageSize());
 
                     // Here we should never write full page, because it is known to be new.
                     next.fullPageWalRecordPolicy(FALSE);
@@ -667,7 +667,7 @@ public abstract class PagesList extends DataStructure {
                     updateTail(bucket, pageId, nextId);
                 }
                 finally {
-                    writeUnlock(next, nextBuf, true);
+                    writeUnlock(next, nextPageAddr, true);
                 }
             }
         }
@@ -687,7 +687,7 @@ public abstract class PagesList extends DataStructure {
     private boolean putReuseBag(
         final long pageId,
         Page page,
-        final ByteBuffer buf,
+        final long buf,
         PagesListNodeIO io,
         ReuseBag bag,
         int bucket
@@ -696,21 +696,21 @@ public abstract class PagesList extends DataStructure {
             return false; // Splitted.
 
         long nextId;
-        ByteBuffer prevBuf = buf;
+        long prevBuf = buf;
         long prevId = pageId;
 
         List<Page> locked = null; // TODO may be unlock right away and do not keep all these pages locked?
-        List<ByteBuffer> lockedBufs = null;
+        List<Long> lockedBufs = null;
 
         try {
             while ((nextId = bag.pollFreePage()) != 0L) {
-                int idx = io.addPage(prevBuf, nextId);
+                int idx = io.addPage(prevBuf, nextId, pageSize());
 
                 if (idx == -1) { // Attempt to add page failed: the node page is full.
                     try (Page next = page(nextId)) {
-                        ByteBuffer nextBuf = writeLock(next); // Page from reuse bag can't be concurrently recycled.
+                        long nextPageAddr = writeLock(next); // Page from reuse bag can't be concurrently recycled.
 
-                        assert nextBuf != null;
+                        assert nextPageAddr != 0L;
 
                         if (locked == null) {
                             lockedBufs = new ArrayList<>(2);
@@ -718,9 +718,9 @@ public abstract class PagesList extends DataStructure {
                         }
 
                         locked.add(next);
-                        lockedBufs.add(nextBuf);
+                        lockedBufs.add(nextPageAddr);
 
-                        setupNextPage(io, prevId, prevBuf, nextId, nextBuf);
+                        setupNextPage(io, prevId, prevBuf, nextId, nextPageAddr);
 
                         if (isWalDeltaRecordNeeded(wal, page))
                             wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
@@ -741,7 +741,7 @@ 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.
-                        prevBuf = nextBuf;
+                        prevBuf = nextPageAddr;
                         prevId = nextId;
                         page = next;
                     }
@@ -784,15 +784,15 @@ public abstract class PagesList extends DataStructure {
      * @param page Page.
      * @param bucket Bucket.
      * @param lockAttempt Lock attempts counter.
-     * @return Buffer if page is locket of {@code null} if can retry lock.
+     * @return Page address if page is locked of {@code null} if can retry lock.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private ByteBuffer writeLockPage(Page page, int bucket, int lockAttempt)
+    private long writeLockPage(Page page, int bucket, int lockAttempt)
         throws IgniteCheckedException {
-        ByteBuffer buf = tryWriteLock(page);
+        long pageAddr = tryWriteLock(page);
 
-        if (buf != null)
-            return buf;
+        if (pageAddr != 0L)
+            return pageAddr;
 
         if (lockAttempt == TRY_LOCK_ATTEMPTS) {
             Stripe[] stripes = getBucket(bucket);
@@ -800,11 +800,11 @@ public abstract class PagesList extends DataStructure {
             if (stripes == null || stripes.length < MAX_STRIPES_PER_BUCKET) {
                 addStripe(bucket, false);
 
-                return null;
+                return 0L;
             }
         }
 
-        return lockAttempt < TRY_LOCK_ATTEMPTS ? null : writeLock(page); // Must be explicitly checked further.
+        return lockAttempt < TRY_LOCK_ATTEMPTS ? 0L : writeLock(page); // Must be explicitly checked further.
     }
 
     /**
@@ -823,9 +823,9 @@ public abstract class PagesList extends DataStructure {
             long tailId = stripe.tailId;
 
             try (Page tail = page(tailId)) {
-                ByteBuffer tailBuf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+                long tailBuf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
 
-                if (tailBuf == null)
+                if (tailBuf == 0L)
                     continue;
 
                 assert PageIO.getPageId(tailBuf) == tailId : "tailId = " + tailId + ", tailBufId = " + PageIO.getPageId(tailBuf);
@@ -859,7 +859,7 @@ public abstract class PagesList extends DataStructure {
                             if (prevId != 0L) {
                                 try (Page prev = page(prevId)) {
                                     // Lock pages from next to previous.
-                                    Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                                    Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                                     assert ok == TRUE : ok;
                                 }
@@ -878,7 +878,7 @@ public abstract class PagesList extends DataStructure {
 
                             try (Page prev = page(prevId)) {
                                 // Lock pages from next to previous.
-                                Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                                Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                                 assert ok == TRUE : ok;
                             }
@@ -888,7 +888,7 @@ public abstract class PagesList extends DataStructure {
 
                                 PageIO initIo = initIoVers.latest();
 
-                                initIo.initNewPage(tailBuf, tailId);
+                                initIo.initNewPage(tailBuf, tailId, pageSize());
 
                                 if (isWalDeltaRecordNeeded(wal, tail)) {
                                     wal.log(new InitNewPageRecord(cacheId, tail.id(), initIo.getType(),
@@ -933,7 +933,7 @@ public abstract class PagesList extends DataStructure {
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if page was removed.
      */
-    protected final boolean removeDataPage(Page dataPage, ByteBuffer dataPageBuf, DataPageIO dataIO, int bucket)
+    protected final boolean removeDataPage(Page dataPage, long dataPageBuf, DataPageIO dataIO, int bucket)
         throws IgniteCheckedException {
         long dataPageId = dataPage.id();
 
@@ -947,17 +947,17 @@ public abstract class PagesList extends DataStructure {
 
             long recycleId = 0L;
 
-            ByteBuffer buf = writeLock(page); // Explicit check.
+            long pageAddr = writeLock(page); // Explicit check.
 
-            if (buf == null)
+            if (pageAddr == 0L)
                 return false;
 
             boolean rmvd = false;
 
             try {
-                PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                rmvd = io.removePage(buf, dataPageId);
+                rmvd = io.removePage(pageAddr, dataPageId);
 
                 if (!rmvd)
                     return false;
@@ -971,20 +971,20 @@ public abstract class PagesList extends DataStructure {
                 if (isWalDeltaRecordNeeded(wal, dataPage))
                     wal.log(new DataPageSetFreeListPageRecord(cacheId, dataPageId, 0L));
 
-                if (!io.isEmpty(buf))
+                if (!io.isEmpty(pageAddr))
                     return true; // In optimistic case we still have something in the page and can leave it as is.
 
                 // If the page is empty, we have to try to drop it and link next and previous with each other.
-                nextId = io.getNextId(buf);
-                prevId = io.getPreviousId(buf);
+                nextId = io.getNextId(pageAddr);
+                prevId = io.getPreviousId(pageAddr);
 
                 // If there are no next page, then we can try to merge without releasing current write lock,
                 // because if we will need to lock previous page, the locking order will be already correct.
                 if (nextId == 0L)
-                    recycleId = mergeNoNext(pageId, page, buf, prevId, bucket);
+                    recycleId = mergeNoNext(pageId, page, pageAddr, prevId, bucket);
             }
             finally {
-                writeUnlock(page, buf, rmvd);
+                writeUnlock(page, pageAddr, rmvd);
             }
 
             // Perform a fair merge after lock release (to have a correct locking order).
@@ -1007,7 +1007,7 @@ public abstract class PagesList extends DataStructure {
      * @return Page ID to recycle.
      * @throws IgniteCheckedException If failed.
      */
-    private long mergeNoNext(long pageId, Page page, ByteBuffer buf, long prevId, int bucket)
+    private long mergeNoNext(long pageId, Page page, long buf, 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.
@@ -1016,7 +1016,7 @@ public abstract class PagesList extends DataStructure {
 
         if (prevId != 0L) { // Cut tail if we have a previous page.
             try (Page prev = page(prevId)) {
-                Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                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.
             }
@@ -1044,25 +1044,25 @@ public abstract class PagesList extends DataStructure {
             try (Page next = nextId == 0L ? null : page(nextId)) {
                 boolean write = false;
 
-                ByteBuffer nextBuf = next == null ? null : writeLock(next); // Explicit check.
-                ByteBuffer buf = writeLock(page); // Explicit check.
+                long nextPageAddr = next == null ? 0L : writeLock(next); // Explicit check.
+                long pageAddr = writeLock(page); // Explicit check.
 
-                if (buf == null) {
-                    if (nextBuf != null) // Unlock next page if needed.
-                        writeUnlock(next, nextBuf, false);
+                if (pageAddr == 0L) {
+                    if (nextPageAddr != 0L) // Unlock next page if needed.
+                        writeUnlock(next, nextPageAddr, false);
 
                     return 0L; // Someone has merged or taken our empty page concurrently. Nothing to do here.
                 }
 
                 try {
-                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                    if (!io.isEmpty(buf))
+                    if (!io.isEmpty(pageAddr))
                         return 0L; // No need to merge anymore.
 
                     // Check if we see a consistent state of the world.
-                    if (io.getNextId(buf) == nextId && (nextId == 0L) == (nextBuf == null)) {
-                        long recycleId = doMerge(pageId, page, buf, io, next, nextId, nextBuf, bucket);
+                    if (io.getNextId(pageAddr) == nextId && (nextId == 0L) == (nextPageAddr == 0L)) {
+                        long recycleId = doMerge(pageId, page, pageAddr, io, next, nextId, nextPageAddr, bucket);
 
                         write = true;
 
@@ -1070,13 +1070,13 @@ public abstract class PagesList extends DataStructure {
                     }
 
                     // Reread next page ID and go for retry.
-                    nextId = io.getNextId(buf);
+                    nextId = io.getNextId(pageAddr);
                 }
                 finally {
-                    if (nextBuf != null)
-                        writeUnlock(next, nextBuf, write);
+                    if (nextPageAddr != 0L)
+                        writeUnlock(next, nextPageAddr, write);
 
-                    writeUnlock(page, buf, write);
+                    writeUnlock(page, pageAddr, write);
                 }
             }
         }
@@ -1097,11 +1097,11 @@ public abstract class PagesList extends DataStructure {
     private long doMerge(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page next,
         long nextId,
-        ByteBuffer nextBuf,
+        long nextBuf,
         int bucket
     ) throws IgniteCheckedException {
         long prevId = io.getPreviousId(buf);
@@ -1143,22 +1143,22 @@ public abstract class PagesList extends DataStructure {
         long pageId,
         long nextId,
         Page next,
-        ByteBuffer nextBuf)
+        long nextBuf)
         throws IgniteCheckedException {
         try (Page prev = page(prevId)) {
-            ByteBuffer prevBuf = writeLock(prev); // No check, we keep a reference.
+            long prevPageAddr = writeLock(prev); // No check, we keep a reference.
 
-            assert prevBuf != null;
+            assert prevPageAddr != 0L;
 
             try {
-                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevBuf);
+                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevPageAddr);
                 PagesListNodeIO nextIO = PagesListNodeIO.VERSIONS.forPage(nextBuf);
 
                 // These references must be updated at the same time in write locks.
-                assert prevIO.getNextId(prevBuf) == pageId;
+                assert prevIO.getNextId(prevPageAddr) == pageId;
                 assert nextIO.getPreviousId(nextBuf) == pageId;
 
-                prevIO.setNextId(prevBuf, nextId);
+                prevIO.setNextId(prevPageAddr, nextId);
 
                 if (isWalDeltaRecordNeeded(wal, prev))
                     wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
@@ -1169,7 +1169,7 @@ public abstract class PagesList extends DataStructure {
                     wal.log(new PagesListSetPreviousRecord(cacheId, nextId, prevId));
             }
             finally {
-                writeUnlock(prev, prevBuf, true);
+                writeUnlock(prev, prevPageAddr, true);
             }
         }
     }
@@ -1181,7 +1181,7 @@ public abstract class PagesList extends DataStructure {
      * @return Rotated page ID.
      * @throws IgniteCheckedException If failed.
      */
-    private long recyclePage(long pageId, Page page, ByteBuffer buf) throws IgniteCheckedException {
+    private long recyclePage(long pageId, Page page, long buf) throws IgniteCheckedException {
         pageId = PageIdUtils.rotatePageId(pageId);
 
         PageIO.setPageId(buf, pageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
index b525a01..6bd50a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist.io;
 
-import java.nio.ByteBuffer;
 import java.util.Map;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.freelist.PagesList;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -53,66 +53,67 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setCount(buf, 0);
-        setNextMetaPageId(buf, 0L);
+        setCount(pageAddr, 0);
+        setNextMetaPageId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Stored items count.
      */
-    private int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    private int getCount(long pageAddr) {
+        return PageUtils.getShort(pageAddr, CNT_OFF);
     }
 
     /**
-     * @param buf Buffer,
+     * @param pageAddr Page address.
      * @param cnt Stored items count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long pageAddr, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Next meta page ID.
      */
-    public long getNextMetaPageId(ByteBuffer buf) {
-        return buf.getLong(NEXT_META_PAGE_OFF);
+    public long getNextMetaPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_META_PAGE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param metaPageId Next meta page ID.
      */
-    public void setNextMetaPageId(ByteBuffer buf, long metaPageId) {
-        buf.putLong(NEXT_META_PAGE_OFF, metaPageId);
+    public void setNextMetaPageId(long pageAddr, long metaPageId) {
+        PageUtils.putLong(pageAddr, NEXT_META_PAGE_OFF, metaPageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public void resetCount(ByteBuffer buf) {
-        setCount(buf, 0);
+    public void resetCount(long pageAddr) {
+        setCount(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageSize Page size.
+     * @param pageAddr Page address.
      * @param bucket Bucket number.
      * @param tails Tails.
      * @param tailsOff Tails offset.
      * @return Number of items written.
      */
-    public int addTails(ByteBuffer buf, int bucket, PagesList.Stripe[] tails, int tailsOff) {
+    public int addTails(int pageSize, long pageAddr, int bucket, PagesList.Stripe[] tails, int tailsOff) {
         assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
-        int cnt = getCount(buf);
-        int cap = getCapacity(buf);
+        int cnt = getCount(pageAddr);
+        int cap = getCapacity(pageSize, pageAddr);
 
         if (cnt == cap)
             return 0;
@@ -122,25 +123,25 @@ public class PagesListMetaIO extends PageIO {
         int write = Math.min(cap - cnt, tails.length - tailsOff);
 
         for (int i = 0; i < write; i++) {
-            buf.putShort(off, (short)bucket);
-            buf.putLong(off + 2, tails[tailsOff].tailId);
+            PageUtils.putShort(pageAddr, off, (short)bucket);
+            PageUtils.putLong(pageAddr, off + 2, tails[tailsOff].tailId);
 
             tailsOff++;
 
             off += ITEM_SIZE;
         }
 
-        setCount(buf, cnt + write);
+        setCount(pageAddr, cnt + write);
 
         return write;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param res Results map.
      */
-    public void getBucketsData(ByteBuffer buf, Map<Integer, GridLongList> res) {
-        int cnt = getCount(buf);
+    public void getBucketsData(long pageAddr, Map<Integer, GridLongList> res) {
+        int cnt = getCount(pageAddr);
 
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
@@ -150,10 +151,10 @@ public class PagesListMetaIO extends PageIO {
         int off = offset(0);
 
         for (int i = 0; i < cnt; i++) {
-            Integer bucket = (int)buf.getShort(off);
+            Integer bucket = (int)PageUtils.getShort(pageAddr, off);
             assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
-            long tailId = buf.getLong(off + 2);
+            long tailId = PageUtils.getLong(pageAddr, off + 2);
             assert tailId != 0;
 
             GridLongList list = res.get(bucket);
@@ -168,11 +169,11 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Maximum number of items which can be stored in buffer.
      */
-    private int getCapacity(ByteBuffer buf) {
-        return (buf.capacity() - ITEMS_OFF) / ITEM_SIZE;
+    private int getCapacity(int pageSize, long pageAddr) {
+        return (pageSize - ITEMS_OFF) / ITEM_SIZE;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
index 2349fa1..6bd0532 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
 import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.copyMemory;
 
 /**
- * TODO optimize: now we have slow {@link #removePage(ByteBuffer, long)}
+ * TODO optimize: now we have slow {@link #removePage(long, long)}
  */
 public class PagesListNodeIO extends PageIO {
     /** */
@@ -52,84 +52,84 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setEmpty(buf);
+        setEmpty(pageAddr);
 
-        setPreviousId(buf, 0L);
-        setNextId(buf, 0L);
+        setPreviousId(pageAddr, 0L);
+        setNextId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    private void setEmpty(ByteBuffer buf) {
-        setCount(buf, 0);
+    private void setEmpty(long pageAddr) {
+        setCount(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Next page ID.
      */
-    public long getNextId(ByteBuffer buf) {
-        return buf.getLong(NEXT_PAGE_ID_OFF);
+    public long getNextId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextId Next page ID.
      */
-    public void setNextId(ByteBuffer buf, long nextId) {
-        buf.putLong(NEXT_PAGE_ID_OFF, nextId);
+    public void setNextId(long pageAddr, long nextId) {
+        PageUtils.putLong(pageAddr, NEXT_PAGE_ID_OFF, nextId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Previous page ID.
      */
-    public long getPreviousId(ByteBuffer buf) {
-        return buf.getLong(PREV_PAGE_ID_OFF);
+    public long getPreviousId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, PREV_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param prevId Previous  page ID.
      */
-    public void setPreviousId(ByteBuffer buf, long prevId) {
-        buf.putLong(PREV_PAGE_ID_OFF, prevId);
+    public void setPreviousId(long pageAddr, long prevId) {
+        PageUtils.putLong(pageAddr, PREV_PAGE_ID_OFF, prevId);
     }
 
     /**
      * Gets total count of entries in this page. Does not change the buffer state.
      *
-     * @param buf Page buffer to get count from.
+     * @param pageAddr Page address to get count from.
      * @return Total number of entries.
      */
-    public int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    public int getCount(long pageAddr) {
+        return PageUtils.getShort(pageAddr, CNT_OFF);
     }
 
     /**
      * Sets total count of entries in this page. Does not change the buffer state.
      *
-     * @param buf Page buffer to write to.
+     * @param pageAddr Page address to write to.
      * @param cnt Count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long pageAddr, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
     }
 
     /**
      * Gets capacity of this page in items.
      *
-     * @param buf Page buffer to get capacity.
+     * @param pageSize Page size.
      * @return Capacity of this page in items.
      */
-    private int getCapacity(ByteBuffer buf) {
-        return (buf.capacity() - PAGE_IDS_OFF) >>> 3; // /8
+    private int getCapacity(int pageSize) {
+        return (pageSize - PAGE_IDS_OFF) >>> 3; // /8
     }
 
     /**
@@ -141,38 +141,39 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /**
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @return Item at the given index.
      */
-    private long getAt(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    private long getAt(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @param pageId Item value to write.
      */
-    private void setAt(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx), pageId);
+    private void setAt(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx), pageId);
     }
 
     /**
      * Adds page to the end of pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param pageId Page ID.
+     * @param pageSize Page size.
      * @return Total number of items in this page.
      */
-    public int addPage(ByteBuffer buf, long pageId) {
-        int cnt = getCount(buf);
+    public int addPage(long pageAddr, long pageId, int pageSize) {
+        int cnt = getCount(pageAddr);
 
-        if (cnt == getCapacity(buf))
+        if (cnt == getCapacity(pageSize))
             return -1;
 
-        setAt(buf, cnt, pageId);
-        setCount(buf, cnt + 1);
+        setAt(pageAddr, cnt, pageId);
+        setCount(pageAddr, cnt + 1);
 
         return cnt;
     }
@@ -180,38 +181,38 @@ public class PagesListNodeIO extends PageIO {
     /**
      * Removes any page from the pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @return Removed page ID.
      */
-    public long takeAnyPage(ByteBuffer buf) {
-        int cnt = getCount(buf);
+    public long takeAnyPage(long pageAddr) {
+        int cnt = getCount(pageAddr);
 
         if (cnt == 0)
             return 0L;
 
-        setCount(buf, --cnt);
+        setCount(pageAddr, --cnt);
 
-        return getAt(buf, cnt);
+        return getAt(pageAddr, cnt);
     }
 
     /**
      * Removes the given page ID from the pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param dataPageId Page ID to remove.
      * @return {@code true} if page was in the list and was removed, {@code false} otherwise.
      */
-    public boolean removePage(ByteBuffer buf, long dataPageId) {
+    public boolean removePage(long pageAddr, long dataPageId) {
         assert dataPageId != 0;
 
-        int cnt = getCount(buf);
+        int cnt = getCount(pageAddr);
 
         for (int i = 0; i < cnt; i++) {
-            if (getAt(buf, i) == dataPageId) {
+            if (getAt(pageAddr, i) == dataPageId) {
                 if (i != cnt - 1)
-                    copyMemory(buf, buf, offset(i + 1), offset(i), 8 * (cnt - i - 1));
+                    copyMemory(pageAddr, pageAddr, offset(i + 1), offset(i), 8 * (cnt - i - 1));
 
-                setCount(buf, cnt - 1);
+                setCount(pageAddr, cnt - 1);
 
                 return true;
             }
@@ -221,10 +222,10 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /**
-     * @param buf Page buffer.
-     * @return {@code true} if there are no items in this page.
+     * @param pageAddr Page address.
+     * @return {@code True} if there are no items in this page.
      */
-    public boolean isEmpty(ByteBuffer buf) {
-        return getCount(buf) == 0;
+    public boolean isEmpty(long pageAddr) {
+        return getCount(pageAddr) == 0;
     }
 }