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/11 13:22:35 UTC

[01/25] ignite git commit: ignite-gg-11810

Repository: ignite
Updated Branches:
  refs/heads/ignite-gg-11810 [created] b81c30acb


http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index 9642e62..fb8762e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@ -17,8 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
  *
@@ -69,7 +68,7 @@ public class PageMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
+    @Override public void initNewPage(long buf, long pageId) {
         super.initNewPage(buf, pageId);
 
         setTreeRoot(buf, 0);
@@ -85,106 +84,106 @@ public class PageMetaIO extends PageIO {
      * @param buf Buffer.
      * @return Meta store root page.
      */
-    public long getTreeRoot(ByteBuffer buf) {
-        return buf.getLong(METASTORE_ROOT_OFF);
+    public long getTreeRoot(long buf) {
+        return PageUtils.getLong(buf, METASTORE_ROOT_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param metastoreRoot metastore root
      */
-    public void setTreeRoot(@NotNull ByteBuffer buf, long metastoreRoot) {
-        buf.putLong(METASTORE_ROOT_OFF, metastoreRoot);
+    public void setTreeRoot(long buf, long metastoreRoot) {
+        PageUtils.putLong(buf, METASTORE_ROOT_OFF, metastoreRoot);
     }
 
     /**
      * @param buf Buffer.
      * @return Reuse list root page.
      */
-    public long getReuseListRoot(ByteBuffer buf) {
-        return buf.getLong(REUSE_LIST_ROOT_OFF);
+    public long getReuseListRoot(long buf) {
+        return PageUtils.getLong(buf, REUSE_LIST_ROOT_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param pageId Root page ID.
      */
-    public void setReuseListRoot(@NotNull ByteBuffer buf, long pageId) {
-        buf.putLong(REUSE_LIST_ROOT_OFF, pageId);
+    public void setReuseListRoot(long buf, long pageId) {
+        PageUtils.putLong(buf, REUSE_LIST_ROOT_OFF, pageId);
     }
 
     /**
      * @param buf Buffer.
      * @param lastSuccessfulBackupId Last successful backup id.
      */
-    public void setLastSuccessfulBackupId(@NotNull ByteBuffer buf, long lastSuccessfulBackupId) {
-        buf.putLong(LAST_SUCCESSFUL_BACKUP_ID_OFF, lastSuccessfulBackupId);
+    public void setLastSuccessfulBackupId(long buf, long lastSuccessfulBackupId) {
+        PageUtils.putLong(buf, LAST_SUCCESSFUL_BACKUP_ID_OFF, lastSuccessfulBackupId);
     }
 
     /**
      * @param buf Buffer.
      */
-    public long getLastSuccessfulBackupId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_BACKUP_ID_OFF);
+    public long getLastSuccessfulBackupId(long buf) {
+        return PageUtils.getLong(buf, LAST_SUCCESSFUL_BACKUP_ID_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param lastSuccessfulFullBackupId Last successful full backup id.
      */
-    public void setLastSuccessfulFullBackupId(@NotNull ByteBuffer buf, long lastSuccessfulFullBackupId) {
-        buf.putLong(LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF, lastSuccessfulFullBackupId);
+    public void setLastSuccessfulFullBackupId(long buf, long lastSuccessfulFullBackupId) {
+        PageUtils.putLong(buf, LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF, lastSuccessfulFullBackupId);
     }
 
     /**
      * @param buf Buffer.
      */
-    public long getLastSuccessfulFullBackupId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF);
+    public long getLastSuccessfulFullBackupId(long buf) {
+        return PageUtils.getLong(buf, LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param nextBackupId Next backup id.
      */
-    public void setNextBackupId(@NotNull ByteBuffer buf, long nextBackupId) {
-        buf.putLong(NEXT_BACKUP_ID_OFF, nextBackupId);
+    public void setNextBackupId(long buf, long nextBackupId) {
+        PageUtils.putLong(buf, NEXT_BACKUP_ID_OFF, nextBackupId);
     }
 
     /**
      * @param buf Buffer.
      * @param lastAllocatedIdx Last allocated index.
      */
-    public void setLastAllocatedIndex(@NotNull ByteBuffer buf, int lastAllocatedIdx) {
-        buf.putInt(LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
+    public void setLastAllocatedIndex(long buf, int lastAllocatedIdx) {
+        PageUtils.putInt(buf, LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
     }
 
     /**
      * @param buf Buffer.
      */
-    public int getLastAllocatedIndex(@NotNull ByteBuffer buf) {
-        return buf.getInt(LAST_ALLOCATED_INDEX_OFF);
+    public int getLastAllocatedIndex(long buf) {
+        return PageUtils.getInt(buf, LAST_ALLOCATED_INDEX_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param previousAllocatedIdx Last allocated index.
      */
-    public void setCandidateAllocatedIndex(@NotNull ByteBuffer buf, int previousAllocatedIdx) {
-        buf.putInt(CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
+    public void setCandidateAllocatedIndex(long buf, int previousAllocatedIdx) {
+        PageUtils.putInt(buf, CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
     }
 
     /**
      * @param buf Buffer.
      */
-    public int getCandidateAllocatedIndex(@NotNull ByteBuffer buf) {
-        return buf.getInt(CANDIDATE_ALLOCATED_INDEX_OFF);
+    public int getCandidateAllocatedIndex(long buf) {
+        return PageUtils.getInt(buf, CANDIDATE_ALLOCATED_INDEX_OFF);
     }
 
     /**
      * @param buf Buffer.
      */
-    public long getNextBackupId(@NotNull ByteBuffer buf) {
-        return buf.getLong(NEXT_BACKUP_ID_OFF);
+    public long getNextBackupId(long buf) {
+        return PageUtils.getLong(buf, NEXT_BACKUP_ID_OFF);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
index c01d774..72893fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
@@ -18,8 +18,11 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
+/**
+ *
+ */
 public class PagePartitionMetaIO extends PageMetaIO {
     /** */
     private static final int SIZE_OFF = PageMetaIO.END_OF_PAGE_META;
@@ -39,7 +42,7 @@ public class PagePartitionMetaIO extends PageMetaIO {
     );
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
+    @Override public void initNewPage(long buf, long pageId) {
         super.initNewPage(buf, pageId);
 
         setSize(buf, 0);
@@ -59,62 +62,62 @@ public class PagePartitionMetaIO extends PageMetaIO {
      * @param buf Buffer.
      * @return Partition size.
      */
-    public long getSize(ByteBuffer buf) {
-        return buf.getLong(SIZE_OFF);
+    public long getSize(long buf) {
+        return PageUtils.getLong(buf, SIZE_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param size Partition size.
      */
-    public void setSize(ByteBuffer buf, long size) {
-        buf.putLong(SIZE_OFF, size);
+    public void setSize(long buf, long size) {
+        PageUtils.putLong(buf, SIZE_OFF, size);
     }
 
     /**
      * @param buf Buffer.
      * @return Partition update counter.
      */
-    public long getUpdateCounter(ByteBuffer buf) {
-        return buf.getLong(UPDATE_CNTR_OFF);
+    public long getUpdateCounter(long buf) {
+        return PageUtils.getLong(buf, UPDATE_CNTR_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param cntr Partition update counter.
      */
-    public void setUpdateCounter(ByteBuffer buf, long cntr) {
-        buf.putLong(UPDATE_CNTR_OFF, cntr);
+    public void setUpdateCounter(long buf, long cntr) {
+        PageUtils.putLong(buf, UPDATE_CNTR_OFF, cntr);
     }
 
     /**
      * @param buf Buffer.
      * @return Global remove ID.
      */
-    public long getGlobalRemoveId(ByteBuffer buf) {
-        return buf.getLong(GLOBAL_RMV_ID_OFF);
+    public long getGlobalRemoveId(long buf) {
+        return PageUtils.getLong(buf, GLOBAL_RMV_ID_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param rmvId Global remove ID.
      */
-    public void setGlobalRemoveId(ByteBuffer buf, long rmvId) {
-        buf.putLong(GLOBAL_RMV_ID_OFF, rmvId);
+    public void setGlobalRemoveId(long buf, long rmvId) {
+        PageUtils.putLong(buf, GLOBAL_RMV_ID_OFF, rmvId);
     }
 
     /**
      * @param buf Buffer.
      */
-    public byte getPartitionState(ByteBuffer buf) {
-        return buf.get(PARTITION_STATE_OFF);
+    public byte getPartitionState(long buf) {
+        return PageUtils.getByte(buf, PARTITION_STATE_OFF);
     }
 
     /**
      * @param buf
      * @param state
      */
-    public void setPartitionState(ByteBuffer buf, byte state) {
-        buf.put(PARTITION_STATE_OFF, state);
+    public void setPartitionState(long buf, byte state) {
+        PageUtils.putByte(buf, PARTITION_STATE_OFF, state);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
index 8c0d6a6..81feb51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 
 /**
@@ -76,7 +76,7 @@ public class TrackingPageIO extends PageIO {
      * @param nextBackupId id of next backup.
      * @param pageSize Page size.
      */
-    public boolean markChanged(ByteBuffer buf, long pageId, long nextBackupId, long lastSuccessfulBackupId, int pageSize) {
+    public boolean markChanged(long buf, long pageId, long nextBackupId, long lastSuccessfulBackupId, int pageSize) {
         validateBackupId(buf, nextBackupId, lastSuccessfulBackupId, pageSize);
 
         int cntOfPage = countOfPageToTrack(pageSize);
@@ -87,19 +87,19 @@ public class TrackingPageIO extends PageIO {
 
         short newSize = (short)(countOfChangedPage(buf, nextBackupId, pageSize) + 1);
 
-        buf.putShort(sizeOff, newSize);
+        PageUtils.putShort(buf, sizeOff, newSize);
 
         assert newSize == countOfChangedPage(buf, nextBackupId, pageSize);
 
         int idx = sizeOff + SIZE_FIELD_SIZE + (idxToUpdate >> 3);
 
-        byte byteToUpdate = buf.get(idx);
+        byte byteToUpdate = PageUtils.getByte(buf, idx);
 
         int updateTemplate = 1 << (idxToUpdate & 0b111);
 
         byte newVal =  (byte) (byteToUpdate | updateTemplate);
 
-        buf.put(idx, newVal);
+        PageUtils.putByte(buf, idx, newVal);
 
         return newVal != byteToUpdate;
     }
@@ -110,7 +110,7 @@ public class TrackingPageIO extends PageIO {
      * @param lastSuccessfulBackupId Last successful backup id.
      * @param pageSize Page size.
      */
-    private void validateBackupId(ByteBuffer buf, long nextBackupId, long lastSuccessfulBackupId, int pageSize) {
+    private void validateBackupId(long buf, long nextBackupId, long lastSuccessfulBackupId, int pageSize) {
         assert nextBackupId != lastSuccessfulBackupId : "nextBackupId = " + nextBackupId +
             ", lastSuccessfulBackupId = " + lastSuccessfulBackupId;
 
@@ -124,10 +124,11 @@ public class TrackingPageIO extends PageIO {
         int cntOfPage = countOfPageToTrack(pageSize);
 
         if (last <= lastSuccessfulBackupId) { //we can drop our data
-            buf.putLong(LAST_BACKUP_OFFSET, nextBackupId);
+            PageUtils.putLong(buf, LAST_BACKUP_OFFSET, nextBackupId);
 
             PageHandler.zeroMemory(buf, SIZE_FIELD_OFFSET, buf.capacity() - SIZE_FIELD_OFFSET);
-        } else { //we can't drop data, it is still necessary for incremental backups
+        }
+        else { //we can't drop data, it is still necessary for incremental backups
             int len = cntOfPage >> 3;
 
             int sizeOff = useLeftHalf(nextBackupId) ? SIZE_FIELD_OFFSET : BITMAP_OFFSET + len;
@@ -142,32 +143,32 @@ public class TrackingPageIO extends PageIO {
                 int i = 0;
 
                 for (; i < len - 8; i += 8) {
-                    long newVal = buf.getLong(sizeOff + SIZE_FIELD_SIZE + i) | buf.getLong(sizeOff2 + SIZE_FIELD_SIZE + i);
+                    long newVal = PageUtils.getLong(buf, sizeOff + SIZE_FIELD_SIZE + i) | PageUtils.getLong(buf, sizeOff2 + SIZE_FIELD_SIZE + i);
 
                     newSize += Long.bitCount(newVal);
 
-                    buf.putLong(sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
+                    PageUtils.putLong(buf, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
                 }
 
                 for (i -= 8; i < len; i ++) {
-                    byte newVal = (byte) (buf.get(sizeOff + SIZE_FIELD_SIZE + i) | buf.get(sizeOff2 + SIZE_FIELD_SIZE + i));
+                    byte newVal = (byte) (PageUtils.getByte(buf, sizeOff + SIZE_FIELD_SIZE + i) | PageUtils.getByte(buf, sizeOff2 + SIZE_FIELD_SIZE + i));
 
                     newSize += Integer.bitCount(newVal & 0xFF);
 
-                    buf.put(sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
+                    PageUtils.putByte(buf, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
                 }
 
-                buf.putShort(sizeOff2, (short)newSize);
+                PageUtils.putShort(buf, sizeOff2, (short)newSize);
             }
 
-            buf.putLong(LAST_BACKUP_OFFSET, nextBackupId);
+            PageUtils.putLong(buf, LAST_BACKUP_OFFSET, nextBackupId);
 
             PageHandler.zeroMemory(buf, sizeOff, len + SIZE_FIELD_SIZE);
         }
     }
 
-    public long getLastBackupId(ByteBuffer buf) {
-        return buf.getLong(LAST_BACKUP_OFFSET);
+    public long getLastBackupId(long buf) {
+        return PageUtils.getLong(buf, LAST_BACKUP_OFFSET);
     }
 
     /**
@@ -178,7 +179,7 @@ public class TrackingPageIO extends PageIO {
      * @param curBackupId Backup id.
      * @param pageSize Page size.
      */
-    public boolean wasChanged(ByteBuffer buf, long pageId, long curBackupId, long lastSuccessfulBackupId, int pageSize) {
+    public boolean wasChanged(long buf, long pageId, long curBackupId, long lastSuccessfulBackupId, int pageSize) {
         validateBackupId(buf, curBackupId + 1, lastSuccessfulBackupId, pageSize);
 
         if (countOfChangedPage(buf, curBackupId, pageSize) < 1)
@@ -191,9 +192,9 @@ public class TrackingPageIO extends PageIO {
         byte byteToTest;
 
         if (useLeftHalf(curBackupId))
-            byteToTest = buf.get(BITMAP_OFFSET + (idxToTest >> 3));
+            byteToTest = PageUtils.getByte(buf, BITMAP_OFFSET + (idxToTest >> 3));
         else
-            byteToTest = buf.get(BITMAP_OFFSET + SIZE_FIELD_SIZE + ((idxToTest + cntOfPage) >> 3));
+            byteToTest = PageUtils.getByte(buf, BITMAP_OFFSET + SIZE_FIELD_SIZE + ((idxToTest + cntOfPage) >> 3));
 
         int testTemplate = 1 << (idxToTest & 0b111);
 
@@ -207,16 +208,16 @@ public class TrackingPageIO extends PageIO {
      *
      * @return count of pages which were marked as change for given backupId
      */
-    public short countOfChangedPage(ByteBuffer buf, long backupId, int pageSize) {
+    public short countOfChangedPage(long buf, long backupId, int pageSize) {
         long dif = getLastBackupId(buf) - backupId;
 
         if (dif != 0 && dif != 1)
             return -1;
 
         if (useLeftHalf(backupId))
-            return buf.getShort(SIZE_FIELD_OFFSET);
+            return PageUtils.getShort(buf, SIZE_FIELD_OFFSET);
         else
-            return buf.getShort(BITMAP_OFFSET + (countOfPageToTrack(pageSize) >> 3));
+            return PageUtils.getShort(buf, BITMAP_OFFSET + (countOfPageToTrack(pageSize) >> 3));
     }
 
     /**
@@ -262,7 +263,7 @@ public class TrackingPageIO extends PageIO {
      * @param pageSize Page size.
      * @return set pageId if it was changed or next closest one, if there is no changed page null will be returned
      */
-    public Long findNextChangedPage(ByteBuffer buf, long start, long curBackupId, long lastSuccessfulBackupId, int pageSize) {
+    public Long findNextChangedPage(long buf, long start, long curBackupId, long lastSuccessfulBackupId, int pageSize) {
         validateBackupId(buf, curBackupId + 1, lastSuccessfulBackupId, pageSize);
 
         int cntOfPage = countOfPageToTrack(pageSize);
@@ -286,7 +287,7 @@ public class TrackingPageIO extends PageIO {
         int stopIdx = zeroIdx + (cntOfPage >> 3);
 
         while (idx < stopIdx) {
-            byte byteToTest = buf.get(idx);
+            byte byteToTest = PageUtils.getByte(buf, idx);
 
             if (byteToTest != 0) {
                 int foundSetBit;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 2620caf..df25449 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
@@ -35,7 +35,7 @@ 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, ByteBuffer buf, Void arg, int intArg)
+        @Override public Boolean run(Page page, PageIO io, long buf, Void arg, int intArg)
             throws IgniteCheckedException {
             return TRUE;
         }
@@ -50,7 +50,7 @@ public abstract class PageHandler<X, R> {
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract R run(Page page, PageIO io, ByteBuffer buf, X arg, int intArg)
+    public abstract R run(Page page, PageIO io, long buf, X arg, int intArg)
         throws IgniteCheckedException;
 
     /**
@@ -74,15 +74,15 @@ public abstract class PageHandler<X, R> {
      */
     public static <X, R> R readPage(
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        ByteBuffer buf = readLock(page, lockListener);
+        long buf = readLock(page, lockLsnr);
 
-        if (buf == null)
+        if (buf == 0L)
             return lockFailed;
 
         try {
@@ -91,7 +91,7 @@ public abstract class PageHandler<X, R> {
             return h.run(page, io, buf, arg, intArg);
         }
         finally {
-            readUnlock(page, buf, lockListener);
+            readUnlock(page, buf, lockLsnr);
         }
     }
 
@@ -106,43 +106,43 @@ public abstract class PageHandler<X, R> {
      */
     public static <X, R> R writePage(
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        return writePage(page, lockListener, h, null, null, arg, intArg, lockFailed);
+        return writePage(page, lockLsnr, h, null, null, arg, intArg, lockFailed);
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param init IO for new page initialization or {@code null} if it is an existing page.
      * @throws IgniteCheckedException If failed.
      */
     public static void initPage(
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
-        Boolean res = writePage(page, lockListener, NOOP, init, wal, null, 0, FALSE);
+        Boolean res = writePage(page, lockLsnr, NOOP, init, wal, null, 0, FALSE);
 
         assert res == TRUE : res; // It must be newly allocated page, can't be recycled.
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @return Byte buffer or {@code null} if failed to lock due to recycling.
      */
-    public static ByteBuffer readLock(Page page, PageLockListener lockListener) {
-        lockListener.onBeforeReadLock(page);
+    public static long readLock(Page page, PageLockListener lockLsnr) {
+        lockLsnr.onBeforeReadLock(page);
 
-        ByteBuffer buf = page.getForRead();
+        long buf = page.getForReadPointer();
 
-        lockListener.onReadLock(page, buf);
+        lockLsnr.onReadLock(page, buf);
 
         return buf;
     }
@@ -150,26 +150,26 @@ public abstract class PageHandler<X, R> {
     /**
      * @param page Page.
      * @param buf Page buffer.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      */
-    public static void readUnlock(Page page, ByteBuffer buf, PageLockListener lockListener) {
-        lockListener.onReadUnlock(page, buf);
+    public static void readUnlock(Page page, long buf, PageLockListener lockLsnr) {
+        lockLsnr.onReadUnlock(page, buf);
 
         page.releaseRead();
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param tryLock Only try to lock without waiting.
      * @return Byte buffer or {@code null} if failed to lock due to recycling.
      */
-    public static ByteBuffer writeLock(Page page, PageLockListener lockListener, boolean tryLock) {
-        lockListener.onBeforeWriteLock(page);
+    public static long writeLock(Page page, PageLockListener lockLsnr, boolean tryLock) {
+        lockLsnr.onBeforeWriteLock(page);
 
-        ByteBuffer buf = tryLock ? page.tryGetForWrite() : page.getForWrite();
+        long buf = tryLock ? page.tryGetForWritePointer() : page.getForWritePointer();
 
-        lockListener.onWriteLock(page, buf);
+        lockLsnr.onWriteLock(page, buf);
 
         return buf;
     }
@@ -177,18 +177,18 @@ public abstract class PageHandler<X, R> {
     /**
      * @param page Page.
      * @param buf Page buffer.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param dirty Page is dirty.
      */
-    public static void writeUnlock(Page page, ByteBuffer buf, PageLockListener lockListener, boolean dirty) {
-        lockListener.onWriteUnlock(page, buf);
+    public static void writeUnlock(Page page, long buf, PageLockListener lockLsnr, boolean dirty) {
+        lockLsnr.onWriteUnlock(page, buf);
 
         page.releaseWrite(dirty);
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param init IO for new page initialization or {@code null} if it is an existing page.
      * @param arg Argument.
@@ -199,7 +199,7 @@ public abstract class PageHandler<X, R> {
      */
     public static <X, R> R writePage(
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         PageIO init,
         IgniteWriteAheadLogManager wal,
@@ -207,9 +207,9 @@ public abstract class PageHandler<X, R> {
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        ByteBuffer buf = writeLock(page, lockListener, false);
+        long buf = writeLock(page, lockLsnr, false);
 
-        if (buf == null)
+        if (buf == 0L)
             return lockFailed;
 
         R res;
@@ -230,7 +230,7 @@ public abstract class PageHandler<X, R> {
             assert PageIO.getCrc(buf) == 0; //TODO GG-11480
 
             if (h.releaseAfterWrite(page, arg, intArg))
-                writeUnlock(page, buf, lockListener, ok);
+                writeUnlock(page, buf, lockLsnr, ok);
         }
 
         return res;
@@ -245,7 +245,7 @@ public abstract class PageHandler<X, R> {
      */
     private static void doInitPage(
         Page page,
-        ByteBuffer buf,
+        long buf,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
@@ -294,14 +294,11 @@ public abstract class PageHandler<X, R> {
         GridUnsafe.copyMemory(srcArr, srcPtr + srcArrOff + srcOff, dstArr, dstPtr + dstArrOff + dstOff, cnt);
     }
 
-    public static void zeroMemory(ByteBuffer buf, int off, int length) {
-        if (buf.isDirect())
-            GridUnsafe.setMemory(((DirectBuffer)buf).address() + off, length, (byte)0);
-
-        else {
-            for (int i = off; i < off + length; i++)
-                buf.put(i, (byte)0);
+    public static void copyMemory(long src, long dst, long srcOff, long dstOff, long cnt) {
+        GridUnsafe.copyMemory(null, src + srcOff, null, dst + dstOff, cnt);
+    }
 
-        }
+    public static void zeroMemory(long buf, int off, int len) {
+        GridUnsafe.setMemory(buf + off, len, (byte)0);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 216bdd3..8ee4e01 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
@@ -33,13 +33,13 @@ public interface PageLockListener {
      * @param page Page.
      * @param buf Buffer or {@code null} if attempt to lock failed.
      */
-    public void onWriteLock(Page page, ByteBuffer buf);
+    public void onWriteLock(Page page, long buf);
 
     /**
      * @param page Page.
      * @param buf Buffer.
      */
-    public void onWriteUnlock(Page page, ByteBuffer buf);
+    public void onWriteUnlock(Page page, long buf);
 
     /**
      * @param page Page.
@@ -50,11 +50,11 @@ public interface PageLockListener {
      * @param page Page.
      * @param buf Buffer or {@code null} if attempt to lock failed.
      */
-    public void onReadLock(Page page, ByteBuffer buf);
+    public void onReadLock(Page page, long buf);
 
     /**
      * @param page Page.
      * @param buf Buffer.
      */
-    public void onReadUnlock(Page page, ByteBuffer buf);
+    public void onReadUnlock(Page page, long buf);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
index 1f7a53c..aaf10f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
@@ -44,7 +44,7 @@ import sun.misc.Unsafe;
  */
 public abstract class GridUnsafe {
     /** Unsafe. */
-    private static final Unsafe UNSAFE = unsafe();
+    public static final Unsafe UNSAFE = unsafe();
 
     /** Unaligned flag. */
     private static final boolean UNALIGNED = unaligned();


[13/25] ignite git commit: test

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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 6eb7fe1..4a7a68a 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
@@ -126,15 +126,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 return null;
 
             try (Page page = page(pageId)) {
-                long buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
-                    BPlusIO io = io(buf);
+                    BPlusIO io = io(pageAddr);
 
                     if (io.isLeaf())
                         return null;
 
-                    int cnt = io.getCount(buf);
+                    int cnt = io.getCount(pageAddr);
 
                     assert cnt >= 0 : cnt;
 
@@ -144,12 +144,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                         res = new ArrayList<>(cnt + 1);
 
                         for (int i = 0; i < cnt; i++)
-                            res.add(inner(io).getLeft(buf, i));
+                            res.add(inner(io).getLeft(pageAddr, i));
 
-                        res.add(inner(io).getRight(buf, cnt - 1));
+                        res.add(inner(io).getRight(pageAddr, cnt - 1));
                     }
                     else {
-                        long left = inner(io).getLeft(buf, 0);
+                        long left = inner(io).getLeft(pageAddr, 0);
 
                         res = left == 0 ? Collections.<Long>emptyList() : Collections.singletonList(left);
                     }
@@ -157,7 +157,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                     return res;
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -173,15 +173,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 return "<Zero>";
 
             try (Page page = page(pageId)) {
-                long buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
-                    BPlusIO<L> io = io(buf);
+                    BPlusIO<L> io = io(pageAddr);
 
-                    return printPage(io, buf, keys);
+                    return printPage(io, pageAddr, keys);
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -192,17 +192,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Get> askNeighbor = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Get g, int isBack) {
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int isBack) {
             assert !io.isLeaf(); // Inner page.
 
             boolean back = isBack == TRUE.ordinal();
 
-            long res = doAskNeighbor(io, buf, back);
+            long res = doAskNeighbor(io, pageAddr, back);
 
             if (back) {
                 assert g.getClass() == Remove.class;
 
-                if (io.getForward(buf) != g.backId) // See how g.backId is setup in removeDown for this check.
+                if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in removeDown for this check.
                     return RETRY;
 
                 g.backId = res;
@@ -219,25 +219,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Get> search = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Get g, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
-            if (io.getForward(buf) != g.fwdId)
+            if (io.getForward(pageAddr) != g.fwdId)
                 return RETRY;
 
             boolean needBackIfRouting = g.backId != 0;
 
             g.backId = 0; // Usually we'll go left down and don't need it.
 
-            int cnt = io.getCount(buf);
-            int idx = findInsertionPoint(io, buf, 0, cnt, g.row, g.shift);
+            int cnt = io.getCount(pageAddr);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, g.row, g.shift);
 
             boolean found = idx >= 0;
 
             if (found) { // Found exact match.
                 assert g.getClass() != GetCursor.class;
 
-                if (g.found(io, buf, idx, lvl))
+                if (g.found(io, pageAddr, idx, lvl))
                     return FOUND;
 
                 // Else we need to reach leaf page, go left down.
@@ -245,20 +245,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             else {
                 idx = fix(idx);
 
-                if (g.notFound(io, buf, idx, lvl)) // No way down, stop here.
+                if (g.notFound(io, pageAddr, idx, lvl)) // No way down, stop here.
                     return NOT_FOUND;
             }
 
             assert !io.isLeaf();
 
             // If idx == cnt then we go right down, else left down: getLeft(cnt) == getRight(cnt - 1).
-            g.pageId = inner(io).getLeft(buf, idx);
+            g.pageId = inner(io).getLeft(pageAddr, idx);
 
             // If we see the tree in consistent state, then our right down page must be forward for our left down page,
             // we need to setup fwdId and/or backId to be able to check this invariant on lower level.
             if (idx < cnt) {
                 // Go left down here.
-                g.fwdId = inner(io).getRight(buf, idx);
+                g.fwdId = inner(io).getRight(pageAddr, idx);
             }
             else {
                 // Go right down here or it is an empty branch.
@@ -267,7 +267,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 // Here child's forward is unknown to us (we either go right or it is an empty "routing" page),
                 // need to ask our forward about the child's forward (it must be leftmost child of our forward page).
                 // This is ok from the locking standpoint because we take all locks in the forward direction.
-                long fwdId = io.getForward(buf);
+                long fwdId = io.getForward(pageAddr);
 
                 // Setup fwdId.
                 if (fwdId == 0)
@@ -282,7 +282,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                 // Setup backId.
                 if (cnt != 0) // It is not a routing page and we are going to the right, can get backId here.
-                    g.backId = inner(io).getLeft(buf, cnt - 1);
+                    g.backId = inner(io).getLeft(pageAddr, cnt - 1);
                 else if (needBackIfRouting) {
                     // Can't get backId here because of possible deadlock and it is only needed for remove operation.
                     return GO_DOWN_X;
@@ -295,16 +295,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Put> replace = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
-            if (io.getForward(buf) != p.fwdId)
+            if (io.getForward(pageAddr) != p.fwdId)
                 return RETRY;
 
             assert p.btmLvl == 0 : "split is impossible with replace";
 
-            final int cnt = io.getCount(buf);
-            final int idx = findInsertionPoint(io, buf, 0, cnt, p.row, 0);
+            final int cnt = io.getCount(pageAddr);
+            final int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx < 0) // Not found, split or merge happened.
                 return RETRY;
@@ -318,7 +318,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 assert p.oldRow == null;
 
                 // Get old row in leaf page to reduce contention at upper level.
-                p.oldRow = getRow(io, buf, idx);
+                p.oldRow = getRow(io, pageAddr, idx);
 
                 p.finish();
 
@@ -326,7 +326,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 assert p.needReplaceInner == FALSE || p.needReplaceInner == DONE : p.needReplaceInner;
             }
 
-            io.store(buf, idx, newRow, null);
+            io.store(pageAddr, idx, newRow, null);
 
             if (needWalDeltaRecord(page))
                 wal.log(new ReplaceRecord<>(cacheId, page.id(), io, newRow, null, idx));
@@ -337,16 +337,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Put> insert = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page 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;
 
             // Check triangle invariant.
-            if (io.getForward(buf) != p.fwdId)
+            if (io.getForward(pageAddr) != p.fwdId)
                 return RETRY;
 
-            int cnt = io.getCount(buf);
-            int idx = findInsertionPoint(io, buf, 0, cnt, p.row, 0);
+            int cnt = io.getCount(pageAddr);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx >= 0) // We do not support concurrent put of the same key.
                 throw new IllegalStateException("Duplicate row in index.");
@@ -354,7 +354,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             idx = fix(idx);
 
             // Do insert.
-            L moveUpRow = p.insert(page, io, buf, idx, lvl);
+            L moveUpRow = p.insert(page, io, pageAddr, idx, lvl);
 
             // Check if split happened.
             if (moveUpRow != null) {
@@ -363,8 +363,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                 // 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(buf);
-                p.tail(page, buf);
+                p.rightId = io.getForward(pageAddr);
+                p.tail(page, pageAddr);
 
                 assert p.rightId != 0;
             }
@@ -376,16 +376,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     };
 
     /** */
-    private final GetPageHandler<Remove> removeFromLeaf = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page leaf, long buf, BPlusIO<L> io, Remove r, int lvl)
+    private final GetPageHandler<Remove> rmvFromLeaf = new GetPageHandler<Remove>() {
+        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
-            final int cnt = io.getCount(buf);
+            final int cnt = io.getCount(pageAddr);
 
             assert cnt <= Short.MAX_VALUE: cnt;
 
-            int idx = findInsertionPoint(io, buf, 0, cnt, r.row, 0);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, r.row, 0);
 
             if (idx < 0) {
                 if (!r.ceil) // We've found exact match on search but now it's gone.
@@ -401,14 +401,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             // 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(buf) != 0;
+            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(pageAddr) != 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(buf, pageSize())))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(pageAddr, 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);
@@ -432,7 +432,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 if (needReplaceInner)
                     r.needReplaceInner = TRUE;
 
-                Tail<L> t = r.addTail(leaf, buf, io, 0, Tail.EXACT);
+                Tail<L> t = r.addTail(leaf, pageAddr, io, 0, Tail.EXACT);
 
                 t.idx = (short)idx;
 
@@ -441,18 +441,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 return FOUND;
             }
 
-            r.removeDataRowFromLeaf(leaf, io, buf, cnt, idx);
+            r.removeDataRowFromLeaf(leaf, io, pageAddr, cnt, idx);
 
             return FOUND;
         }
     };
 
     /** */
-    private final GetPageHandler<Remove> lockBackAndRemoveFromLeaf = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page back, long buf, BPlusIO<L> io, Remove r, int lvl)
+    private final GetPageHandler<Remove> lockBackAndRmvFromLeaf = new GetPageHandler<Remove>() {
+        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(buf) != r.pageId)
+            if (io.getForward(pageAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
@@ -460,7 +460,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             // Keep locks on back and leaf pages for subsequent merges.
             if (res == FOUND && r.tail != null)
-                r.addTail(back, buf, io, lvl, Tail.BACK);
+                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -468,17 +468,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockBackAndTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page back, long buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(buf) != r.pageId)
+            if (io.getForward(pageAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
             Result res = r.doLockTail(lvl);
 
             if (res == FOUND)
-                r.addTail(back, buf, io, lvl, Tail.BACK);
+                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -486,9 +486,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockTailForward = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page page, long buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
-            r.addTail(page, buf, io, lvl, Tail.FORWARD);
+            r.addTail(page, pageAddr, io, lvl, Tail.FORWARD);
 
             return FOUND;
         }
@@ -496,12 +496,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
             // Check that we have a correct view of the world.
-            if (io.getForward(buf) != r.fwdId)
+            if (io.getForward(pageAddr) != r.fwdId)
                 return RETRY;
 
             // We don't have a back page, need to lock our forward and become a back for it.
@@ -512,7 +512,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                     return res; // Retry.
             }
 
-            r.addTail(page, buf, io, lvl, Tail.EXACT);
+            r.addTail(page, pageAddr, io, lvl, Tail.EXACT);
 
             return FOUND;
         }
@@ -520,14 +520,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Void, Bool> cutRoot = new PageHandler<Void, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, long buf, Void ignore, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long pageAddr, 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(buf); // Can drop only root.
+            assert lvl == io.getRootLevel(pageAddr); // Can drop only root.
 
-            io.cutRoot(pageSize(), buf);
+            io.cutRoot(pageSize(), pageAddr);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
@@ -538,16 +538,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Long, Bool> addRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, long buf, Long rootPageId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootPageId, int lvl)
             throws IgniteCheckedException {
             assert rootPageId != null;
 
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            assert lvl == io.getLevelsCount(buf);
+            assert lvl == io.getLevelsCount(pageAddr);
 
-            io.addRoot(pageSize(), buf, rootPageId);
+            io.addRoot(pageSize(), pageAddr, rootPageId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
@@ -558,14 +558,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Long, Bool> initRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, long buf, Long rootId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootId, int lvl)
             throws IgniteCheckedException {
             assert rootId != null;
 
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            io.initRoot(pageSize(), buf, rootId);
+            io.initRoot(pageSize(), pageAddr, rootId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageInitRootRecord(cacheId, meta.id(), rootId));
@@ -579,10 +579,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param cacheId Cache ID.
      * @param pageMem Page memory.
      * @param wal Write ahead log manager.
+     * @param globalRmvId Remove ID.
      * @param metaPageId Meta page ID.
      * @param reuseList Reuse list.
      * @param innerIos Inner IO versions.
      * @param leafIos Leaf IO versions.
+     * @throws IgniteCheckedException If failed.
      */
     public BPlusTree(
         String name,
@@ -653,16 +655,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     }
 
     /**
+     * @param meta Meta page.
      * @return Root level.
      */
     private int getRootLevel(Page meta) {
-        long buf = readLock(meta); // Meta can't be removed.
+        long pageAddr = readLock(meta); // Meta can't be removed.
 
         try {
-            return BPlusMetaIO.VERSIONS.forPage(buf).getRootLevel(buf);
+            return BPlusMetaIO.VERSIONS.forPage(pageAddr).getRootLevel(pageAddr);
         }
         finally {
-            readUnlock(meta, buf);
+            readUnlock(meta, pageAddr);
         }
     }
 
@@ -672,27 +675,28 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Page ID.
      */
     private long getFirstPageId(Page meta, int lvl) {
-        long buf = readLock(meta); // Meta can't be removed.
+        long pageAddr = readLock(meta); // Meta can't be removed.
 
         try {
-            BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+            BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
             if (lvl < 0)
-                lvl = io.getRootLevel(buf);
+                lvl = io.getRootLevel(pageAddr);
 
-            if (lvl >= io.getLevelsCount(buf))
+            if (lvl >= io.getLevelsCount(pageAddr))
                 return 0;
 
-            return io.getFirstPageId(buf, lvl);
+            return io.getFirstPageId(pageAddr, lvl);
         }
         finally {
-            readUnlock(meta, buf);
+            readUnlock(meta, pageAddr);
         }
     }
 
     /**
      * @param upper Upper bound.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
     private GridCursor<T> findLowerUnbounded(L upper) throws IgniteCheckedException {
         ForwardCursor cursor = new ForwardCursor(null, upper);
@@ -704,13 +708,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         try (Page first = page(firstPageId)) {
-            long buf = readLock(first); // We always merge pages backwards, the first page is never removed.
+            long pageAddr = readLock(first); // We always merge pages backwards, the first page is never removed.
 
             try {
-                cursor.init(buf, io(buf), 0);
+                cursor.init(pageAddr, io(pageAddr), 0);
             }
             finally {
-                readUnlock(first, buf);
+                readUnlock(first, pageAddr);
             }
         }
 
@@ -758,6 +762,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     /**
      * @param row Lookup row for exact match.
      * @return Found row.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
     public final T findOne(L row) throws IgniteCheckedException {
@@ -783,6 +788,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /**
      * @param g Get.
+     * @throws IgniteCheckedException If failed.
      */
     private void doFind(Get g) throws IgniteCheckedException {
         try {
@@ -921,22 +927,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            long buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (cnt < 0)
                     fail("Negative count: " + cnt);
 
                 if (io.isLeaf()) {
                     for (int i = 0; i < cnt; i++) {
-                        if (minRow != null && compare(io, buf, i, minRow) <= 0)
+                        if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
                             fail("Wrong sort order: " + U.hexLong(pageId) + " , at " + i + " , minRow: " + minRow);
 
-                        minRow = io.getLookupRow(this, buf, i);
+                        minRow = io.getLookupRow(this, pageAddr, i);
                     }
 
                     return;
@@ -944,16 +950,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                 // To find our inner key we have to go left and then always go to the right.
                 for (int i = 0; i < cnt; i++) {
-                    L row = io.getLookupRow(this, buf, i);
+                    L row = io.getLookupRow(this, pageAddr, i);
 
-                    if (minRow != null && compare(io, buf, i, minRow) <= 0)
+                    if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
                         fail("Min row violated: " + row + " , minRow: " + minRow);
 
-                    long leftId = inner(io).getLeft(buf, i);
+                    long leftId = inner(io).getLeft(pageAddr, i);
 
                     L leafRow = getGreatestRowInSubTree(leftId);
 
-                    int cmp = compare(io, buf, i, leafRow);
+                    int cmp = compare(io, pageAddr, i, leafRow);
 
                     if (cmp < 0 || (cmp != 0 && canGetRowFromInner))
                         fail("Wrong inner row: " + U.hexLong(pageId) + " , at: " + i + " , leaf:  " + leafRow +
@@ -965,12 +971,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 }
 
                 // Need to handle the rightmost child subtree separately or handle empty routing page.
-                long rightId = inner(io).getLeft(buf, cnt); // The same as getRight(cnt - 1)
+                long rightId = inner(io).getLeft(pageAddr, cnt); // The same as getRight(cnt - 1)
 
                 validateDownKeys(rightId, minRow);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
@@ -982,26 +988,26 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private L getGreatestRowInSubTree(long pageId) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            long buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (io.isLeaf()) {
                     if (cnt <= 0) // This code is called only if the tree is not empty, so we can't see empty leaf.
                         fail("Invalid leaf count: " + cnt + " " + U.hexLong(pageId));
 
-                    return io.getLookupRow(this, buf, cnt - 1);
+                    return io.getLookupRow(this, pageAddr, cnt - 1);
                 }
 
-                long rightId = inner(io).getLeft(buf, cnt);// The same as getRight(cnt - 1), but good for routing pages.
+                long rightId = inner(io).getLeft(pageAddr, cnt);// The same as getRight(cnt - 1), but good for routing pages.
 
                 return getGreatestRowInSubTree(rightId);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
@@ -1067,25 +1073,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private void validateDownPages(Page meta, long pageId, long fwdId, final int lvl) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            long buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                long realPageId = BPlusIO.getPageId(buf);
+                long realPageId = BPlusIO.getPageId(pageAddr);
 
                 if (realPageId != pageId)
                     fail(new SB("ABA on page ID: ref ").appendHex(pageId).a(", buf ").appendHex(realPageId));
 
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
                 if (io.isLeaf() != (lvl == 0)) // Leaf pages only at the level 0.
                     fail("Leaf level mismatch: " + lvl);
 
-                long actualFwdId = io.getForward(buf);
+                long actualFwdId = io.getForward(pageAddr);
 
                 if (actualFwdId != fwdId)
                     fail(new SB("Triangle: expected fwd ").appendHex(fwdId).a(", actual fwd ").appendHex(actualFwdId));
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (cnt < 0)
                     fail("Negative count: " + cnt);
@@ -1097,32 +1103,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 else {
                     // Recursively go down if we are on inner level.
                     for (int i = 0; i < cnt; i++)
-                        validateDownPages(meta, inner(io).getLeft(buf, i), inner(io).getRight(buf, i), lvl - 1);
+                        validateDownPages(meta, inner(io).getLeft(pageAddr, i), inner(io).getRight(pageAddr, i), lvl - 1);
 
                     if (fwdId != 0) {
                         // For the rightmost child ask neighbor.
                         try (Page fwd = page(fwdId)) {
-                            long fwdBuf = readLock(fwd); // No correctness guaranties.
+                            long fwdPageAddr = readLock(fwd); // No correctness guaranties.
 
                             try {
-                                if (io(fwdBuf) != io)
+                                if (io(fwdPageAddr) != io)
                                     fail("IO on the same level must be the same");
 
-                                fwdId = inner(io).getLeft(fwdBuf, 0);
+                                fwdId = inner(io).getLeft(fwdPageAddr, 0);
                             }
                             finally {
-                                readUnlock(fwd, fwdBuf);
+                                readUnlock(fwd, fwdPageAddr);
                             }
                         }
                     }
 
-                    pageId = inner(io).getLeft(buf, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
+                    pageId = inner(io).getLeft(pageAddr, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
 
                     validateDownPages(meta, pageId, fwdId, lvl - 1);
                 }
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
@@ -1487,21 +1493,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
         while (pageId != 0) {
             try (Page page = page(pageId)) {
-                long buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
                     if (io == null) {
-                        io = io(buf);
+                        io = io(pageAddr);
 
                         assert io.isLeaf();
                     }
 
-                    cnt += io.getCount(buf);
+                    cnt += io.getCount(pageAddr);
 
-                    pageId = io.getForward(buf);
+                    pageId = io.getForward(pageAddr);
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
         }
@@ -1533,9 +1539,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             for (;;) { // Go down with retries.
                 p.init();
 
-                Result result = putDown(p, p.rootId, 0L, p.rootLvl);
+                Result res = putDown(p, p.rootId, 0L, p.rootLvl);
 
-                switch (result) {
+                switch (res) {
                     case RETRY:
                     case RETRY_ROOT:
                         checkInterrupted();
@@ -1558,7 +1564,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                         return p.oldRow;
 
                     default:
-                        throw new IllegalStateException("Result: " + result);
+                        throw new IllegalStateException("Result: " + res);
                 }
             }
         }
@@ -1597,28 +1603,28 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         long pagesCnt = 0;
 
         try (Page meta = page(metaPageId)) {
-            long metaBuf = writeLock(meta); // No checks, we must be out of use.
+            long metaPageAddr = writeLock(meta); // No checks, we must be out of use.
 
             try {
-                for (long pageId : getFirstPageIds(metaBuf)) {
+                for (long pageId : getFirstPageIds(metaPageAddr)) {
                     assert pageId != 0;
 
                     do {
                         try (Page page = page(pageId)) {
-                            long buf = writeLock(page); // No checks, we must be out of use.
+                            long pageAddr = writeLock(page); // No checks, we must be out of use.
 
                             try {
-                                BPlusIO<L> io = io(buf);
+                                BPlusIO<L> io = io(pageAddr);
 
-                                long fwdPageId = io.getForward(buf);
+                                long fwdPageId = io.getForward(pageAddr);
 
-                                bag.addFreePage(recyclePage(pageId, page, buf));
+                                bag.addFreePage(recyclePage(pageId, page, pageAddr));
                                 pagesCnt++;
 
                                 pageId = fwdPageId;
                             }
                             finally {
-                                writeUnlock(page, buf, true);
+                                writeUnlock(page, pageAddr, true);
                             }
                         }
 
@@ -1631,11 +1637,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                     while (pageId != 0);
                 }
 
-                bag.addFreePage(recyclePage(metaPageId, meta, metaBuf));
+                bag.addFreePage(recyclePage(metaPageId, meta, metaPageAddr));
                 pagesCnt++;
             }
             finally {
-                writeUnlock(meta, metaBuf, true);
+                writeUnlock(meta, metaPageAddr, true);
             }
         }
 
@@ -1937,19 +1943,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             int rootLvl;
             long rootId;
 
-            long buf = readLock(meta); // Meta can't be removed.
+            long pageAddr = readLock(meta); // Meta can't be removed.
 
-            assert buf != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert pageAddr != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
-                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-                rootLvl = io.getRootLevel(buf);
-                rootId = io.getFirstPageId(buf, rootLvl);
+                rootLvl = io.getRootLevel(pageAddr);
+                rootId = io.getFirstPageId(pageAddr, rootLvl);
             }
             finally {
-                readUnlock(meta, buf);
+                readUnlock(meta, pageAddr);
             }
 
             restartFromRoot(rootId, rootLvl, globalRmvId.get());
@@ -2227,15 +2233,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 // Need to check this before the actual split, because after the split we will have new forward page here.
                 boolean hadFwd = io.getForward(buf) != 0;
 
-                long fwdBuf = writeLock(fwd); // Initial write, no need to check for concurrent modification.
+                long fwdPageAddr = writeLock(fwd); // Initial write, no need to check for concurrent modification.
 
-                assert fwdBuf != 0L;
+                assert fwdPageAddr != 0L;
 
                 try {
                     // Never write full forward page, because it is known to be new.
                     fwd.fullPageWalRecordPolicy(Boolean.FALSE);
 
-                    boolean midShift = splitPage(io, page, buf, fwdId, fwd, fwdBuf, idx);
+                    boolean midShift = splitPage(io, page, buf, fwdId, fwd, fwdPageAddr, idx);
 
                     // Do insert.
                     int cnt = io.getCount(buf);
@@ -2245,14 +2251,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                         // Fix leftmost child of forward page, because newly inserted row will go up.
                         if (idx == cnt && !io.isLeaf()) {
-                            inner(io).setLeft(fwdBuf, 0, rightId);
+                            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));
                         }
                     }
                     else // Insert into newly allocated forward page.
-                        insertSimple(fwd, io, fwdBuf, idx - cnt);
+                        insertSimple(fwd, io, fwdPageAddr, idx - cnt);
 
                     // Do move up.
                     cnt = io.getCount(buf);
@@ -2274,9 +2280,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                             if (io.isLeaf())
                                 io = latestInnerIO();
 
-                            long newRootBuf = writeLock(newRoot); // Initial write.
+                            long newRootPageAddr = writeLock(newRoot); // Initial write.
 
-                            assert newRootBuf != 0L;
+                            assert newRootPageAddr != 0L;
 
                             try {
                                 // Never write full new root page, because it is known to be new.
@@ -2284,14 +2290,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                                 long pageId = PageIO.getPageId(buf);
 
-                                inner(io).initNewRoot(newRootBuf, newRootId, pageId, moveUpRow, null, fwdId, pageSize());
+                                inner(io).initNewRoot(newRootPageAddr,
+                                    newRootId,
+                                    pageId,
+                                    moveUpRow,
+                                    null,
+                                    fwdId,
+                                    pageSize());
 
                                 if (needWalDeltaRecord(newRoot))
                                     wal.log(new NewRootInitRecord<>(cacheId, newRoot.id(), newRootId,
                                         inner(io), pageId, moveUpRow, null, fwdId));
                             }
                             finally {
-                                writeUnlock(newRoot, newRootBuf, true);
+                                writeUnlock(newRoot, newRootPageAddr, true);
                             }
                         }
 
@@ -2306,7 +2318,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                     return moveUpRow;
                 }
                 finally {
-                    writeUnlock(fwd, fwdBuf, true);
+                    writeUnlock(fwd, fwdPageAddr, true);
                 }
             }
         }
@@ -2680,7 +2692,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             Page back = page(backId);
 
             try {
-                return writePage(back, BPlusTree.this, lockBackAndRemoveFromLeaf, this, 0, RETRY);
+                return writePage(back, BPlusTree.this, lockBackAndRmvFromLeaf, this, 0, RETRY);
             }
             finally {
                 if (canRelease(back, 0))
@@ -2695,7 +2707,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         private Result doRemoveFromLeaf() throws IgniteCheckedException {
             assert page != null;
 
-            return writePage(page, BPlusTree.this, removeFromLeaf, this, 0, RETRY);
+            return writePage(page, BPlusTree.this, rmvFromLeaf, this, 0, RETRY);
         }
 
         /**
@@ -3664,22 +3676,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 }
 
                 try (Page next = page(nextPageId)) {
-                    long buf = readLock(next); // Doing explicit null check.
+                    long pageAddr = readLock(next); // Doing explicit null check.
 
                     // If concurrent merge occurred we have to reinitialize cursor from the last returned row.
-                    if (buf == 0L)
+                    if (pageAddr == 0L)
                         break;
 
                     try {
-                        BPlusIO<L> io = io(buf);
+                        BPlusIO<L> io = io(pageAddr);
 
-                        if (fillFromBuffer(buf, io, 0, io.getCount(buf)))
+                        if (fillFromBuffer(pageAddr, io, 0, io.getCount(pageAddr)))
                             return true;
 
                         // Continue fetching forward.
                     }
                     finally {
-                        readUnlock(next, buf);
+                        readUnlock(next, pageAddr);
                     }
                 }
             }
@@ -3714,31 +3726,31 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     private abstract class GetPageHandler<G extends Get> extends PageHandler<G, Result> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override public final Result run(Page page, PageIO iox, long buf, G g, int lvl)
+        @Override public final Result run(Page page, PageIO iox, long pageAddr, G g, int lvl)
             throws IgniteCheckedException {
-            assert PageIO.getPageId(buf) == page.id();
+            assert PageIO.getPageId(pageAddr) == page.id();
 
             // If we've passed the check for correct page ID, we can safely cast.
             BPlusIO<L> io = (BPlusIO<L>)iox;
 
             // In case of intersection with inner replace in remove operation
             // we need to restart our operation from the tree root.
-            if (lvl == 0 && g.rmvId < io.getRemoveId(buf))
+            if (lvl == 0 && g.rmvId < io.getRemoveId(pageAddr))
                 return RETRY_ROOT;
 
-            return run0(page, buf, io, g, lvl);
+            return run0(page, pageAddr, io, g, lvl);
         }
 
         /**
          * @param page Page.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param g Operation.
          * @param lvl Level.
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        protected abstract Result run0(Page page, long buf, BPlusIO<L> io, G g, int lvl)
+        protected abstract Result run0(Page page, long pageAddr, BPlusIO<L> io, G g, int lvl)
             throws IgniteCheckedException;
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
index 10df6f1..ff3efb8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
@@ -72,56 +72,56 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId, int pageSize) {
-        super.initNewPage(buf, pageId, pageSize);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setCount(buf, 0);
-        setForward(buf, 0);
-        setRemoveId(buf, 0);
+        setCount(pageAddr, 0);
+        setForward(pageAddr, 0);
+        setRemoveId(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Forward page ID.
      */
-    public final long getForward(long buf) {
-        return PageUtils.getLong(buf, FORWARD_OFF);
+    public final long getForward(long pageAddr) {
+        return PageUtils.getLong(pageAddr, FORWARD_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Forward page ID.
      */
-    public final void setForward(long buf, long pageId) {
-        PageUtils.putLong(buf, FORWARD_OFF, pageId);
+    public final void setForward(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, FORWARD_OFF, pageId);
 
-        assert getForward(buf) == pageId;
+        assert getForward(pageAddr) == pageId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Remove ID.
      */
-    public final long getRemoveId(long buf) {
-        return PageUtils.getLong(buf, REMOVE_ID_OFF);
+    public final long getRemoveId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, REMOVE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rmvId Remove ID.
      */
-    public final void setRemoveId(long buf, long rmvId) {
-        PageUtils.putLong(buf, REMOVE_ID_OFF, rmvId);
+    public final void setRemoveId(long pageAddr, long rmvId) {
+        PageUtils.putLong(pageAddr, REMOVE_ID_OFF, rmvId);
 
-        assert getRemoveId(buf) == rmvId;
+        assert getRemoveId(pageAddr) == rmvId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Items count in the page.
      */
-    public final int getCount(long buf) {
-        int cnt = PageUtils.getShort(buf, CNT_OFF) & 0xFFFF;
+    public final int getCount(long pageAddr) {
+        int cnt = PageUtils.getShort(pageAddr, CNT_OFF) & 0xFFFF;
 
         assert cnt >= 0: cnt;
 
@@ -129,15 +129,15 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Count.
      */
-    public final void setCount(long buf, int cnt) {
+    public final void setCount(long pageAddr, int cnt) {
         assert cnt >= 0: cnt;
 
-        PageUtils.putShort(buf, CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
 
-        assert getCount(buf) == cnt;
+        assert getCount(pageAddr) == cnt;
     }
 
     /**
@@ -157,27 +157,28 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return Max items count.
      */
-    public abstract int getMaxCount(long buf, int pageSize);
+    public abstract int getMaxCount(long pageAddr, int pageSize);
 
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param row Lookup or full row.
      * @param rowBytes Row bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public final void store(long buf, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
+    public final void store(long pageAddr, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
         int off = offset(idx);
 
         if (rowBytes == null)
-            storeByOffset(buf, off, row);
+            storeByOffset(pageAddr, off, row);
         else
-            putBytes(buf, off, rowBytes);
+            putBytes(pageAddr, off, rowBytes);
     }
 
     /**
@@ -189,50 +190,50 @@ public abstract class BPlusIO<L> extends PageIO {
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param off Offset in bytes.
      * @param row Lookup or full row.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void storeByOffset(long buf, int off, L row) throws IgniteCheckedException;
+    public abstract void storeByOffset(long pageAddr, int off, L row) throws IgniteCheckedException;
 
     /**
      * Store row info from the given source.
      *
-     * @param dst Destination buffer
+     * @param dstPageAddr Destination page address.
      * @param dstIdx Destination index.
      * @param srcIo Source IO.
-     * @param src Source buffer.
+     * @param srcPageAddr Source page address.
      * @param srcIdx Source index.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void store(long dst, int dstIdx, BPlusIO<L> srcIo, long src, int srcIdx)
+    public abstract void store(long dstPageAddr, int dstIdx, BPlusIO<L> srcIo, long srcPageAddr, int srcIdx)
         throws IgniteCheckedException;
 
     /**
      * Get lookup row.
      *
      * @param tree Tree.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Lookup row.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract L getLookupRow(BPlusTree<L, ?> tree, long buf, int idx) throws IgniteCheckedException;
+    public abstract L getLookupRow(BPlusTree<L, ?> tree, long pageAddr, int idx) throws IgniteCheckedException;
 
     /**
      * Copy items from source buffer to destination buffer.
      * Both pages must be of the same type and the same version.
      *
-     * @param src Source buffer.
-     * @param dst Destination buffer.
+     * @param srcPageAddr Source page address.
+     * @param dstPageAddr Destination page address.
      * @param srcIdx Source begin index.
      * @param dstIdx Destination begin index.
      * @param cnt Items count.
      * @param cpLeft Copy leftmost link (makes sense only for inner pages).
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void copyItems(long src, long dst, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
+    public abstract void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
         throws IgniteCheckedException;
 
     // Methods for B+Tree logic.
@@ -289,50 +290,51 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param mid Bisection index.
      * @param fwdId New forward page ID.
      */
-    public void splitExistingPage(long buf, int mid, long fwdId) {
-        setCount(buf, mid);
-        setForward(buf, fwdId);
+    public void splitExistingPage(long pageAddr, int mid, long fwdId) {
+        setCount(pageAddr, mid);
+        setForward(pageAddr, fwdId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param cnt Count.
      * @throws IgniteCheckedException If failed.
      */
-    public void remove(long buf, int idx, int cnt) throws IgniteCheckedException {
+    public void remove(long pageAddr, int idx, int cnt) throws IgniteCheckedException {
         cnt--;
 
-        copyItems(buf, buf, idx + 1, idx, cnt - idx, false);
-        setCount(buf, cnt);
+        copyItems(pageAddr, pageAddr, idx + 1, idx, cnt - idx, false);
+        setCount(pageAddr, cnt);
     }
 
     /**
      * @param prntIo Parent IO.
-     * @param prnt Parent buffer.
+     * @param prntPageAddr Parent page address.
      * @param prntIdx Split key index in parent.
-     * @param left Left buffer.
-     * @param right Right buffer.
+     * @param leftPageAddr Left page address.
+     * @param rightPageAddr Right page address.
      * @param emptyBranch We are merging an empty branch.
+     * @param pageSize Page size.
      * @return {@code false} If we were not able to merge.
      * @throws IgniteCheckedException If failed.
      */
     public boolean merge(
         BPlusIO<L> prntIo,
-        long prnt,
+        long prntPageAddr,
         int prntIdx,
-        long left,
-        long right,
+        long leftPageAddr,
+        long rightPageAddr,
         boolean emptyBranch,
         int pageSize
     ) throws IgniteCheckedException {
-        int prntCnt = prntIo.getCount(prnt);
-        int leftCnt = getCount(left);
-        int rightCnt = getCount(right);
+        int prntCnt = prntIo.getCount(prntPageAddr);
+        int leftCnt = getCount(leftPageAddr);
+        int rightCnt = getCount(rightPageAddr);
 
         int newCnt = leftCnt + rightCnt;
 
@@ -340,13 +342,13 @@ public abstract class BPlusIO<L> extends PageIO {
         if (!isLeaf() && !emptyBranch)
             newCnt++;
 
-        if (newCnt > getMaxCount(left, pageSize)) {
+        if (newCnt > getMaxCount(leftPageAddr, pageSize)) {
             assert !emptyBranch;
 
             return false;
         }
 
-        setCount(left, newCnt);
+        setCount(leftPageAddr, newCnt);
 
         // Move down split key in inner pages.
         if (!isLeaf() && !emptyBranch) {
@@ -354,19 +356,19 @@ public abstract class BPlusIO<L> extends PageIO {
 
             // We can be sure that we have enough free space to store split key here,
             // because we've done remove already and did not release child locks.
-            store(left, leftCnt, prntIo, prnt, prntIdx);
+            store(leftPageAddr, leftCnt, prntIo, prntPageAddr, prntIdx);
 
             leftCnt++;
         }
 
-        copyItems(right, left, 0, leftCnt, rightCnt, !emptyBranch);
-        setForward(left, getForward(right));
+        copyItems(rightPageAddr, leftPageAddr, 0, leftCnt, rightCnt, !emptyBranch);
+        setForward(leftPageAddr, getForward(rightPageAddr));
 
-        long rmvId = getRemoveId(right);
+        long rmvId = getRemoveId(rightPageAddr);
 
         // Need to have maximum remove ID.
-        if (rmvId > getRemoveId(left))
-            setRemoveId(left, rmvId);
+        if (rmvId > getRemoveId(leftPageAddr))
+            setRemoveId(leftPageAddr, rmvId);
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
index 7158770..cac7f05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
@@ -45,7 +45,7 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(long buf, int pageSize) {
+    @Override public int getMaxCount(long pageAddr, int pageSize) {
         // The structure of the page is the following:
         // |ITEMS_OFF|w|A|x|B|y|C|z|
         // where capital letters are data items, lowercase letters are 8 byte page references.
@@ -53,63 +53,63 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getLeft(long buf, int idx) {
-        return PageUtils.getLong(buf, (offset(idx, SHIFT_LEFT)));
+    public final long getLeft(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, (offset(idx, SHIFT_LEFT)));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setLeft(long buf, int idx, long pageId) {
-        PageUtils.putLong(buf, offset(idx, SHIFT_LEFT), pageId);
+    public final void setLeft(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx, SHIFT_LEFT), pageId);
 
-        assert pageId == getLeft(buf, idx);
+        assert pageId == getLeft(pageAddr, idx);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getRight(long buf, int idx) {
-        return PageUtils.getLong(buf, offset(idx, SHIFT_RIGHT));
+    public final long getRight(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx, SHIFT_RIGHT));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setRight(long buf, int idx, long pageId) {
-        PageUtils.putLong(buf, offset(idx, SHIFT_RIGHT), pageId);
+    private void setRight(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx, SHIFT_RIGHT), pageId);
 
-        assert pageId == getRight(buf, idx);
+        assert pageId == getRight(pageAddr, idx);
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(long src, long dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
-        assert srcIdx != dstIdx || src != dst;
+        assert srcIdx != dstIdx || srcPageAddr != dstPageAddr;
 
         cnt *= itemSize + 8; // From items to bytes.
 
         if (dstIdx > srcIdx) {
-            PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
+            PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt);
 
             if (cpLeft)
-                PageUtils.putLong(dst, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(src, (offset(srcIdx, SHIFT_LEFT))));
+                PageUtils.putLong(dstPageAddr, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(srcPageAddr, (offset(srcIdx, SHIFT_LEFT))));
         }
         else {
             if (cpLeft)
-                PageUtils.putLong(dst, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(src, (offset(srcIdx, SHIFT_LEFT))));
+                PageUtils.putLong(dstPageAddr, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(srcPageAddr, (offset(srcIdx, SHIFT_LEFT))));
 
-            PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
+            PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt);
         }
     }
 
@@ -148,7 +148,9 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
      * @param newRootId New root ID.
      * @param leftChildId Left child ID.
      * @param row Moved up row.
+     * @param rowBytes Bytes.
      * @param rightChildId Right child ID.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void initNewRoot(

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
index cf97764..8b3a4f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
@@ -35,16 +35,16 @@ public abstract class BPlusLeafIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(long buf, int pageSize) {
+    @Override public int getMaxCount(long pageAddr, int pageSize) {
         return (pageSize - ITEMS_OFF) / itemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(long src, long dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
-        assert srcIdx != dstIdx || src != dst;
+        assert srcIdx != dstIdx || srcPageAddr != dstPageAddr;
 
-        PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt * itemSize);
+        PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt * itemSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index 1034220..c365063 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -149,8 +149,6 @@ public class DataPageIO extends PageIO {
      * @return Data page entry size.
      */
     private int getPageEntrySize(int payloadLen, int show) {
-        if (payloadLen == 0)
-            System.out.println();
         assert payloadLen > 0 : payloadLen;
 
         int res = payloadLen;
@@ -720,8 +718,6 @@ public class DataPageIO extends PageIO {
         final int rowSize,
         final int pageSize
     ) throws IgniteCheckedException {
-        System.out.println("Start add row " + buf + " " + printPageLayout(buf, pageSize));
-
         assert rowSize <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(rowSize, SHOW_PAYLOAD_LEN | SHOW_ITEM);
@@ -736,8 +732,6 @@ public class DataPageIO extends PageIO {
         int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         setLink(row, buf, itemId);
-
-        System.out.println("Add row " + buf + " " + printPageLayout(buf, pageSize));
     }
 
     /**
@@ -929,18 +923,12 @@ public class DataPageIO extends PageIO {
 
             short p = (short)(payloadSize | FRAGMENTED_FLAG);
 
-            System.out.println("Start add row fragment " + buf + " " + dataOff + " " + payloadSize + " " + p);
-
             buf0.putShort(p);
             buf0.putLong(lastLink);
 
             int rowOff = rowSize - written - payloadSize;
 
-            System.out.println("Size1: " + PageUtils.getShort(buf, dataOff));
-
             writeFragmentData(row, buf0, rowOff, payloadSize);
-
-            System.out.println("Size2: " + PageUtils.getShort(buf, dataOff));
         }
         else {
             PageUtils.putShort(buf, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
@@ -955,8 +943,6 @@ public class DataPageIO extends PageIO {
         if (row != null)
             setLink(row, buf, itemId);
 
-        System.out.println("Add fragment " + buf + " " + printPageLayout(buf, pageSize));
-
         return payloadSize;
     }
 
@@ -1274,36 +1260,18 @@ public class DataPageIO extends PageIO {
         int payloadSize,
         CacheDataRow row
     ) throws IgniteCheckedException {
-        PageUtils.putShort(buf, dataOff, (short)payloadSize);
-        dataOff += 2;
-
-        byte[] bytes = row.key().valueBytes(null);
-
-        PageUtils.putInt(buf, dataOff, bytes.length);
-        dataOff += 4;
-
-        PageUtils.putByte(buf, dataOff, row.key().cacheObjectType());
-        dataOff++;
-
-        PageUtils.putBytes(buf, dataOff, bytes);
-        dataOff += bytes.length;
-
-        bytes = row.value().valueBytes(null);
-
-        PageUtils.putInt(buf, dataOff, bytes.length);
-        dataOff += 4;
-
-        PageUtils.putByte(buf, dataOff, row.value().cacheObjectType());
-        dataOff++;
+        long addr = buf + dataOff;
 
-        PageUtils.putBytes(buf, dataOff, bytes);
-        dataOff += bytes.length;
+        PageUtils.putShort(addr, 0, (short)payloadSize);
+        addr += 2;
 
-        CacheVersionIO.write(buf + dataOff, row.version(), false);
+        addr += row.key().putValue(addr);
+        addr += row.value().putValue(addr);
 
-        dataOff += CacheVersionIO.size(row.version(), false);
+        CacheVersionIO.write(addr, row.version(), false);
+        addr += CacheVersionIO.size(row.version(), false);
 
-        PageUtils.putLong(buf, dataOff, row.expireTime());
+        PageUtils.putLong(addr, 0, row.expireTime());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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 39bb44b..480c940 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
@@ -17,14 +17,12 @@
 
 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.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.GridUnsafe;
-import sun.nio.ch.DirectBuffer;
 
 import static java.lang.Boolean.FALSE;
 import static java.lang.Boolean.TRUE;
@@ -35,7 +33,7 @@ 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 buf, Void arg, int intArg)
+        @Override public Boolean run(Page page, PageIO io, long pageAddr, Void arg, int intArg)
             throws IgniteCheckedException {
             return TRUE;
         }
@@ -44,13 +42,13 @@ public abstract class PageHandler<X, R> {
     /**
      * @param page Page.
      * @param io IO.
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @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 buf, X arg, int intArg)
+    public abstract R run(Page page, PageIO io, long pageAddr, X arg, int intArg)
         throws IgniteCheckedException;
 
     /**
@@ -65,6 +63,7 @@ public abstract class PageHandler<X, R> {
 
     /**
      * @param page Page.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -80,23 +79,24 @@ public abstract class PageHandler<X, R> {
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        long buf = readLock(page, lockLsnr);
+        long pageAddr = readLock(page, lockLsnr);
 
-        if (buf == 0L)
+        if (pageAddr == 0L)
             return lockFailed;
 
         try {
-            PageIO io = PageIO.getPageIO(buf);
+            PageIO io = PageIO.getPageIO(pageAddr);
 
-            return h.run(page, io, buf, arg, intArg);
+            return h.run(page, io, pageAddr, arg, intArg);
         }
         finally {
-            readUnlock(page, buf, lockLsnr);
+            readUnlock(page, pageAddr, lockLsnr);
         }
     }
 
     /**
      * @param page Page.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -119,6 +119,7 @@ public abstract class PageHandler<X, R> {
      * @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.
      */
     public static void initPage(
@@ -135,25 +136,25 @@ public abstract class PageHandler<X, R> {
     /**
      * @param page Page.
      * @param lockLsnr Lock listener.
-     * @return Byte buffer or {@code null} if failed to lock due to recycling.
+     * @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 buf = page.getForReadPointer();
+        long pageAddr = page.getForReadPointer();
 
-        lockLsnr.onReadLock(page, buf);
+        lockLsnr.onReadLock(page, pageAddr);
 
-        return buf;
+        return pageAddr;
     }
 
     /**
      * @param page Page.
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param lockLsnr Lock listener.
      */
-    public static void readUnlock(Page page, long buf, PageLockListener lockLsnr) {
-        lockLsnr.onReadUnlock(page, buf);
+    public static void readUnlock(Page page, long pageAddr, PageLockListener lockLsnr) {
+        lockLsnr.onReadUnlock(page, pageAddr);
 
         page.releaseRead();
     }
@@ -162,16 +163,16 @@ public abstract class PageHandler<X, R> {
      * @param page Page.
      * @param lockLsnr Lock listener.
      * @param tryLock Only try to lock without waiting.
-     * @return Byte buffer or {@code null} if failed to lock due to recycling.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
      */
     public static long writeLock(Page page, PageLockListener lockLsnr, boolean tryLock) {
         lockLsnr.onBeforeWriteLock(page);
 
-        long buf = tryLock ? page.tryGetForWritePointer() : page.getForWritePointer();
+        long pageAddr = tryLock ? page.tryGetForWritePointer() : page.getForWritePointer();
 
-        lockLsnr.onWriteLock(page, buf);
+        lockLsnr.onWriteLock(page, pageAddr);
 
-        return buf;
+        return pageAddr;
     }
 
     /**
@@ -191,6 +192,7 @@ public abstract class PageHandler<X, R> {
      * @param lockLsnr 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 arg Argument.
      * @param intArg Argument of type {@code int}.
      * @param lockFailed Result in case of lock failure due to page recycling.
@@ -207,9 +209,9 @@ public abstract class PageHandler<X, R> {
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        long buf = writeLock(page, lockLsnr, false);
+        long pageAddr = writeLock(page, lockLsnr, false);
 
-        if (buf == 0L)
+        if (pageAddr == 0L)
             return lockFailed;
 
         R res;
@@ -218,19 +220,19 @@ public abstract class PageHandler<X, R> {
 
         try {
             if (init != null) // It is a new page and we have to initialize it.
-                doInitPage(page, buf, init, wal);
+                doInitPage(page, pageAddr, init, wal);
             else
-                init = PageIO.getPageIO(buf);
+                init = PageIO.getPageIO(pageAddr);
 
-            res = h.run(page, init, buf, arg, intArg);
+            res = h.run(page, init, pageAddr, arg, intArg);
 
             ok = true;
         }
         finally {
-            assert PageIO.getCrc(buf) == 0; //TODO GG-11480
+            assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
             if (h.releaseAfterWrite(page, arg, intArg))
-                writeUnlock(page, buf, lockLsnr, ok);
+                writeUnlock(page, pageAddr, lockLsnr, ok);
         }
 
         return res;
@@ -238,22 +240,22 @@ public abstract class PageHandler<X, R> {
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param init Initial IO.
      * @param wal Write ahead log.
      * @throws IgniteCheckedException If failed.
      */
     private static void doInitPage(
         Page page,
-        long buf,
+        long pageAddr,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
-        assert PageIO.getCrc(buf) == 0; //TODO GG-11480
+        assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
         long pageId = page.id();
 
-        init.initNewPage(buf, pageId, page.size());
+        init.initNewPage(pageAddr, pageId, page.size());
 
         // Here we should never write full page, because it is known to be new.
         page.fullPageWalRecordPolicy(FALSE);
@@ -275,30 +277,11 @@ public abstract class PageHandler<X, R> {
             (page.fullPageWalRecordPolicy() == FALSE || page.isDirty());
     }
 
-    /**
-     * @param src Source.
-     * @param dst Destination.
-     * @param srcOff Source offset in bytes.
-     * @param dstOff Destination offset in bytes.
-     * @param cnt Bytes count to copy.
-     */
-    public static void copyMemory(ByteBuffer src, ByteBuffer dst, long srcOff, long dstOff, long cnt) {
-        byte[] srcArr = src.hasArray() ? src.array() : null;
-        byte[] dstArr = dst.hasArray() ? dst.array() : null;
-        long srcArrOff = src.hasArray() ? src.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
-        long dstArrOff = dst.hasArray() ? dst.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
-
-        long srcPtr = src.isDirect() ? ((DirectBuffer)src).address() : 0;
-        long dstPtr = dst.isDirect() ? ((DirectBuffer)dst).address() : 0;
-
-        GridUnsafe.copyMemory(srcArr, srcPtr + srcArrOff + srcOff, dstArr, dstPtr + dstArrOff + dstOff, cnt);
-    }
-
-    public static void copyMemory(long src, long dst, long srcOff, long dstOff, long cnt) {
-        GridUnsafe.copyMemory(null, src + srcOff, null, dst + dstOff, cnt);
+    public static void copyMemory(long srcAddr, long dstAddr, long srcOff, long dstOff, long cnt) {
+        GridUnsafe.copyMemory(null, srcAddr + srcOff, null, dstAddr + dstOff, cnt);
     }
 
-    public static void zeroMemory(long buf, int off, int len) {
-        GridUnsafe.setMemory(buf + off, len, (byte)0);
+    public static void zeroMemory(long addr, int off, int len) {
+        GridUnsafe.setMemory(addr + off, len, (byte)0);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
index ef8b4b4..0052de9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
@@ -267,6 +267,11 @@ public class GridAffinityNoCacheSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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 3e78b9c..483b228 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
@@ -18,12 +18,10 @@
 package org.apache.ignite.internal.pagemem.impl;
 
 import java.io.File;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
@@ -211,17 +209,17 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that updated pages are inaccessible using old IDs.
             for (FullPageId id : old) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long pageAddr = page.getForWritePointer();
 
-                    if (buf != null) {
+                    if (pageAddr != 0L) {
                         page.releaseWrite(false);
 
                         fail("Was able to acquire page write lock.");
                     }
 
-                    buf = page.getForRead();
+                    pageAddr = page.getForReadPointer();
 
-                    if (buf != null) {
+                    if (pageAddr != 0) {
                         page.releaseRead();
 
                         fail("Was able to acquire page read lock.");
@@ -303,13 +301,13 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
     private void readPage(Page page, int expVal) {
         expVal &= 0xFF;
 
-        ByteBuffer bytes = page.getForRead();
+        long pageAddr = page.getForReadPointer();
 
-        assertNotNull(bytes);
+        assert(pageAddr != 0);
 
         try {
             for (int i = PageIO.COMMON_HEADER_END; i < PAGE_SIZE; i++) {
-                int val = bytes.get(i) & 0xFF;
+                int val = PageUtils.getByte(pageAddr, i) & 0xFF;
 
                 assertEquals("Unexpected value at position: " + i, expVal, val);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
index 0224a88..4aa72cb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
@@ -122,6 +122,11 @@ public class IgniteIncompleteCacheObjectSelfTest extends GridCommonAbstractTest
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index fd3e0f5..ad5e78f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -237,17 +238,13 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
     protected void checkInsertDeleteSingleThreaded(int pageSize) throws Exception {
         FreeList list = createFreeList(pageSize);
 
-        long seed = 1482500591478L;//System.currentTimeMillis();
-
-        Random rnd = new Random(seed);
-
-        System.out.println("Seed " + seed);
+        Random rnd = new Random();
 
         Map<Long, TestDataRow> stored = new HashMap<>();
 
-        {
-            int keySize = 749;
-            int valSize = 2426;
+        for (int i = 0; i < 100; i++) {
+            int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
+            int valSize = rnd.nextInt(pageSize * 5 / 2) + 10;
 
             TestDataRow row = new TestDataRow(keySize, valSize);
 
@@ -259,88 +256,55 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
 
             assertNull(old);
         }
-        {
-            int keySize = 472;
-            int valSize = 83;
 
-            TestDataRow row = new TestDataRow(keySize, valSize);
+        boolean grow = true;
 
-            list.insertDataRow(row);
+        for (int i = 0; i < 1_000_000; i++) {
+            if (grow) {
+                if (stored.size() > 20_000) {
+                    grow = false;
 
-            assertTrue(row.link() != 0L);
+                    info("Shrink... [" + stored.size() + ']');
+                }
+            }
+            else {
+                if (stored.size() < 1_000) {
+                    grow = true;
 
-            TestDataRow old = stored.put(row.link(), row);
+                    info("Grow... [" + stored.size() + ']');
+                }
+            }
 
-            assertNull(old);
-        }
+            boolean insert = rnd.nextInt(100) < 70 == grow;
 
+            if (insert) {
+                int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
+                int valSize = rnd.nextInt(pageSize * 3 / 2) + 10;
 
+                TestDataRow row = new TestDataRow(keySize, valSize);
 
-//        for (int i = 0; i < 100; i++) {
-//            int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
-//            int valSize = rnd.nextInt(pageSize * 5 / 2) + 10;
-//
-//            System.out.println("Add " + keySize + " " + valSize);
-//
-//            TestDataRow row = new TestDataRow(keySize, valSize);
-//
-//            list.insertDataRow(row);
-//
-//            assertTrue(row.link() != 0L);
-//
-//            TestDataRow old = stored.put(row.link(), row);
-//
-//            assertNull(old);
-//        }
-
-//        boolean grow = true;
-//
-//        for (int i = 0; i < 1_000_000; i++) {
-//            if (grow) {
-//                if (stored.size() > 20_000) {
-//                    grow = false;
-//
-//                    info("Shrink... [" + stored.size() + ']');
-//                }
-//            }
-//            else {
-//                if (stored.size() < 1_000) {
-//                    grow = true;
-//
-//                    info("Grow... [" + stored.size() + ']');
-//                }
-//            }
-//
-//            boolean insert = rnd.nextInt(100) < 70 == grow;
-//
-//            if (insert) {
-//                int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
-//                int valSize = rnd.nextInt(pageSize * 3 / 2) + 10;
-//
-//                TestDataRow row = new TestDataRow(keySize, valSize);
-//
-//                list.insertDataRow(row);
-//
-//                assertTrue(row.link() != 0L);
-//
-//                TestDataRow old = stored.put(row.link(), row);
-//
-//                assertNull(old);
-//            }
-//            else {
-//                Iterator<TestDataRow> it = stored.values().iterator();
-//
-//                if (it.hasNext()) {
-//                    TestDataRow row = it.next();
-//
-//                    TestDataRow rmvd = stored.remove(row.link);
-//
-//                    assertTrue(rmvd == row);
-//
-//                    list.removeDataRowByLink(row.link);
-//                }
-//            }
-//        }
+                list.insertDataRow(row);
+
+                assertTrue(row.link() != 0L);
+
+                TestDataRow old = stored.put(row.link(), row);
+
+                assertNull(old);
+            }
+            else {
+                Iterator<TestDataRow> it = stored.values().iterator();
+
+                if (it.hasNext()) {
+                    TestDataRow row = it.next();
+
+                    TestDataRow rmvd = stored.remove(row.link);
+
+                    assertTrue(rmvd == row);
+
+                    list.removeDataRowByLink(row.link);
+                }
+            }
+        }
     }
 
     /**
@@ -495,6 +459,13 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            PageUtils.putBytes(addr, 0, data);
+
+            return data.length;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(ByteBuffer buf, int off, int len) throws IgniteCheckedException {
             buf.put(data, off, len);
 


[09/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 80bf5806b7d6945c09af3b8f024acfd8f814123f
Parents: 0a556cb
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 23 15:46:44 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 23 15:46:44 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/PageUtils.java      | 38 ++++++++++++++++++++
 .../cache/database/CacheDataRowAdapter.java     |  2 +-
 .../cache/database/tree/io/DataPageIO.java      | 12 +++++--
 3 files changed, 48 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/80bf5806/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index b6faeb1..6f46cd5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -29,10 +29,17 @@ public class PageUtils {
     private static final Unsafe unsafe = GridUnsafe.UNSAFE;
 
     public static byte getByte(long buf, int off) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         return unsafe.getByte(buf + off);
     }
 
     public static byte[] getBytes(long buf, int off, int len) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+        assert len >= 0;
+
         byte[] bytes = new byte[len];
 
         unsafe.copyMemory(null, buf + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
@@ -41,38 +48,69 @@ public class PageUtils {
     }
 
     public static void getBytes(long src, int srcOff, byte[] dst, int dstOff, int len) {
+        assert src > 0;
+        assert srcOff > 0;
+        assert dst != null;
+        assert dstOff >= 0;
+        assert len >= 0;
+
         unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
     }
 
     public static short getShort(long buf, int off) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         return unsafe.getShort(buf + off);
     }
 
     public static int getInt(long buf, int off) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         return unsafe.getInt(buf + off);
     }
 
     public static long getLong(long buf, int off) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         return unsafe.getLong(buf + off);
     }
 
     public static void putBytes(long buf, int off, byte[] bytes) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+        assert bytes != null;
+
         unsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, buf + off, bytes.length);
     }
 
     public static void putByte(long buf, int off, byte v) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         unsafe.putByte(buf + off, v);
     }
 
     public static void putShort(long buf, int off, short v) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         unsafe.putShort(buf + off, v);
     }
 
     public static void putInt(long buf, int off, int v) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         unsafe.putInt(buf + off, v);
     }
 
     public static void putLong(long buf, int off, long v) {
+        assert buf > 0 : buf;
+        assert off >= 0;
+
         unsafe.putLong(buf + off, v);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/80bf5806/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 59e2fc4..1e7a99b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
@@ -104,7 +104,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
                     if (first) {
                         if (nextLink == 0) {
                             // Fast path for a single page row.
-                            readFullRow(coctx, data.offset(), keyOnly);
+                            readFullRow(coctx, buf + data.offset(), keyOnly);
 
                             return;
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/80bf5806/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index c9baf05..fcb7fb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -29,6 +29,8 @@ import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.typedef.internal.SB;
+import sun.misc.JavaNioAccess;
+import sun.misc.SharedSecrets;
 
 /**
  * Data pages IO.
@@ -184,9 +186,10 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param freeSpace Free space.
+     * @param pageSize Page size.
      */
     private void setRealFreeSpace(long buf, int freeSpace, int pageSize) {
-        assert freeSpace == actualFreeSpace(buf, pageSize): freeSpace + " != " + actualFreeSpace(buf, pageSize);
+        assert freeSpace == actualFreeSpace(buf, pageSize) : freeSpace + " != " + actualFreeSpace(buf, pageSize);
 
         PageUtils.putShort(buf, FREE_SPACE_OFF, (short)freeSpace);
     }
@@ -872,6 +875,9 @@ public class DataPageIO extends PageIO {
         addRowFragment(buf, 0, 0, lastLink, null, payload, pageSize);
     }
 
+    // TODO GG-11810.
+    private JavaNioAccess nioAccess = SharedSecrets.getJavaNioAccess();
+
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
@@ -905,11 +911,11 @@ public class DataPageIO extends PageIO {
         int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         if (payload == null) {
-            ByteBuffer buf0 = null;
+            ByteBuffer buf0 = nioAccess.newDirectByteBuffer(buf, pageSize, null);
 
             buf0.position(dataOff);
 
-            buf0.putShort((short) (payloadSize | FRAGMENTED_FLAG));
+            buf0.putShort((short)(payloadSize | FRAGMENTED_FLAG));
             buf0.putLong(lastLink);
 
             int rowOff = rowSize - written - payloadSize;


[23/25] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-gg-8.0.2.ea2' into ignite-gg-11810

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-gg-8.0.2.ea2' into ignite-gg-11810

# Conflicts:
#	modules/pds/src/main/java/org/gridgain/grid/cache/db/GridCacheOffheapManager.java
#	modules/pds/src/main/java/org/gridgain/grid/internal/processors/cache/database/GridCacheDatabaseSharedManager.java


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

Branch: refs/heads/ignite-gg-11810
Commit: 1b56558cae1c75ffb0e011a82916cd10a5829c3b
Parents: 9876950
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 14:40:09 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 14:40:09 2016 +0300

----------------------------------------------------------------------
 modules/yardstick/config/ignite-base-config.xml | 13 -------------
 1 file changed, 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1b56558c/modules/yardstick/config/ignite-base-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-base-config.xml b/modules/yardstick/config/ignite-base-config.xml
index 047738d..8ce3c98 100644
--- a/modules/yardstick/config/ignite-base-config.xml
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -125,19 +125,6 @@
                     <property name="cacheMode" value="PARTITIONED"/>
 
                     <property name="atomicityMode" value="ATOMIC"/>
-
-                    <property name="typeMetadata">
-                        <list>
-                            <value>java.lang.Integer</value>
-                            <value>org.apache.ignite.yardstick.cache.model.Person1</value>
-
-                            <value>java.lang.Integer</value>
-                            <value>org.apache.ignite.yardstick.cache.model.Person2</value>
-
-                            <value>java.lang.Integer</value>
-                            <value>org.apache.ignite.yardstick.cache.model.Person8</value>
-                        </list>
-                    </property>
                 </bean>
 
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">


[10/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 6afd2e795b409f20218991253ccd068a44409902
Parents: 80bf580
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 23 17:06:25 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 23 17:06:25 2016 +0300

----------------------------------------------------------------------
 .../cache/database/tree/io/DataPageIO.java      |  41 +++++-
 .../database/FreeListImplSelfTest.java          | 127 ++++++++++++-------
 2 files changed, 116 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6afd2e79/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index fcb7fb9..0aaa2a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.HashSet;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
@@ -148,7 +149,9 @@ public class DataPageIO extends PageIO {
      * @return Data page entry size.
      */
     private int getPageEntrySize(int payloadLen, int show) {
-        assert payloadLen > 0: payloadLen;
+        if (payloadLen == 0)
+            System.out.println();
+        assert payloadLen > 0 : payloadLen;
 
         int res = payloadLen;
 
@@ -400,6 +403,7 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
+     * @param pageSize Page size.
      * @return Data entry offset in bytes.
      */
     private int getDataOffset(long buf, int itemId, int pageSize) {
@@ -711,11 +715,13 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        long buf,
+        final long buf,
         CacheDataRow row,
-        int rowSize,
-        int pageSize
+        final int rowSize,
+        final int pageSize
     ) throws IgniteCheckedException {
+        System.out.println("Start add row " + buf + " " + printPageLayout(buf, pageSize));
+
         assert rowSize <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(rowSize, SHOW_PAYLOAD_LEN | SHOW_ITEM);
@@ -730,6 +736,8 @@ public class DataPageIO extends PageIO {
         int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         setLink(row, buf, itemId);
+
+        System.out.println("Add row " + buf + " " + printPageLayout(buf, pageSize));
     }
 
     /**
@@ -845,6 +853,7 @@ public class DataPageIO extends PageIO {
      * @param row Cache data row.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
+     * @param pageSize Page size.
      * @return Written payload size.
      * @throws IgniteCheckedException If failed.
      */
@@ -864,6 +873,7 @@ public class DataPageIO extends PageIO {
      * @param buf Byte buffer.
      * @param payload Payload bytes.
      * @param lastLink Link to the previous written fragment (link to the tail).
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRowFragment(
@@ -887,6 +897,7 @@ public class DataPageIO extends PageIO {
      * @param lastLink Link to the previous written fragment (link to the tail).
      * @param row Row.
      * @param payload Payload bytes.
+     * @param pageSize Page size.
      * @return Written payload size.
      * @throws IgniteCheckedException If failed.
      */
@@ -912,24 +923,40 @@ public class DataPageIO extends PageIO {
 
         if (payload == null) {
             ByteBuffer buf0 = nioAccess.newDirectByteBuffer(buf, pageSize, null);
+            buf0.order(PageMemory.NATIVE_BYTE_ORDER);
 
             buf0.position(dataOff);
 
-            buf0.putShort((short)(payloadSize | FRAGMENTED_FLAG));
+            short p = (short)(payloadSize | FRAGMENTED_FLAG);
+
+            System.out.println("Start add row fragment " + buf + " " + dataOff + " " + payloadSize + " " + p);
+
+            buf0.putShort(p);
             buf0.putLong(lastLink);
 
             int rowOff = rowSize - written - payloadSize;
 
+            System.out.println("Size1: " + PageUtils.getShort(buf, dataOff));
+
             writeFragmentData(row, buf0, rowOff, payloadSize);
+
+            System.out.println("Size2: " + PageUtils.getShort(buf, dataOff));
+        }
+        else {
+            PageUtils.putShort(buf, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
+
+            PageUtils.putLong(buf, dataOff + 2, lastLink);
+
+            PageUtils.putBytes(buf, dataOff + 10, payload);
         }
-        else
-            PageUtils.putBytes(buf, dataOff, payload);
 
         int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         if (row != null)
             setLink(row, buf, itemId);
 
+        System.out.println("Add fragment " + buf + " " + printPageLayout(buf, pageSize));
+
         return payloadSize;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6afd2e79/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 2b93d48..fd3e0f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -237,13 +237,17 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
     protected void checkInsertDeleteSingleThreaded(int pageSize) throws Exception {
         FreeList list = createFreeList(pageSize);
 
-        Random rnd = new Random();
+        long seed = 1482500591478L;//System.currentTimeMillis();
+
+        Random rnd = new Random(seed);
+
+        System.out.println("Seed " + seed);
 
         Map<Long, TestDataRow> stored = new HashMap<>();
 
-        for (int i = 0; i < 100; i++) {
-            int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
-            int valSize = rnd.nextInt(pageSize * 5 / 2) + 10;
+        {
+            int keySize = 749;
+            int valSize = 2426;
 
             TestDataRow row = new TestDataRow(keySize, valSize);
 
@@ -255,55 +259,88 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
 
             assertNull(old);
         }
+        {
+            int keySize = 472;
+            int valSize = 83;
 
-        boolean grow = true;
-
-        for (int i = 0; i < 1_000_000; i++) {
-            if (grow) {
-                if (stored.size() > 20_000) {
-                    grow = false;
-
-                    info("Shrink... [" + stored.size() + ']');
-                }
-            }
-            else {
-                if (stored.size() < 1_000) {
-                    grow = true;
-
-                    info("Grow... [" + stored.size() + ']');
-                }
-            }
-
-            boolean insert = rnd.nextInt(100) < 70 == grow;
-
-            if (insert) {
-                int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
-                int valSize = rnd.nextInt(pageSize * 3 / 2) + 10;
-
-                TestDataRow row = new TestDataRow(keySize, valSize);
-
-                list.insertDataRow(row);
+            TestDataRow row = new TestDataRow(keySize, valSize);
 
-                assertTrue(row.link() != 0L);
+            list.insertDataRow(row);
 
-                TestDataRow old = stored.put(row.link(), row);
+            assertTrue(row.link() != 0L);
 
-                assertNull(old);
-            }
-            else {
-                Iterator<TestDataRow> it = stored.values().iterator();
+            TestDataRow old = stored.put(row.link(), row);
 
-                if (it.hasNext()) {
-                    TestDataRow row = it.next();
+            assertNull(old);
+        }
 
-                    TestDataRow rmvd = stored.remove(row.link);
 
-                    assertTrue(rmvd == row);
 
-                    list.removeDataRowByLink(row.link);
-                }
-            }
-        }
+//        for (int i = 0; i < 100; i++) {
+//            int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
+//            int valSize = rnd.nextInt(pageSize * 5 / 2) + 10;
+//
+//            System.out.println("Add " + keySize + " " + valSize);
+//
+//            TestDataRow row = new TestDataRow(keySize, valSize);
+//
+//            list.insertDataRow(row);
+//
+//            assertTrue(row.link() != 0L);
+//
+//            TestDataRow old = stored.put(row.link(), row);
+//
+//            assertNull(old);
+//        }
+
+//        boolean grow = true;
+//
+//        for (int i = 0; i < 1_000_000; i++) {
+//            if (grow) {
+//                if (stored.size() > 20_000) {
+//                    grow = false;
+//
+//                    info("Shrink... [" + stored.size() + ']');
+//                }
+//            }
+//            else {
+//                if (stored.size() < 1_000) {
+//                    grow = true;
+//
+//                    info("Grow... [" + stored.size() + ']');
+//                }
+//            }
+//
+//            boolean insert = rnd.nextInt(100) < 70 == grow;
+//
+//            if (insert) {
+//                int keySize = rnd.nextInt(pageSize * 3 / 2) + 10;
+//                int valSize = rnd.nextInt(pageSize * 3 / 2) + 10;
+//
+//                TestDataRow row = new TestDataRow(keySize, valSize);
+//
+//                list.insertDataRow(row);
+//
+//                assertTrue(row.link() != 0L);
+//
+//                TestDataRow old = stored.put(row.link(), row);
+//
+//                assertNull(old);
+//            }
+//            else {
+//                Iterator<TestDataRow> it = stored.values().iterator();
+//
+//                if (it.hasNext()) {
+//                    TestDataRow row = it.next();
+//
+//                    TestDataRow rmvd = stored.remove(row.link);
+//
+//                    assertTrue(rmvd == row);
+//
+//                    list.removeDataRowByLink(row.link);
+//                }
+//            }
+//        }
     }
 
     /**


[05/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
index 6c739a5..10df6f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
@@ -72,8 +72,8 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setCount(buf, 0);
         setForward(buf, 0);
@@ -160,7 +160,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param buf Buffer.
      * @return Max items count.
      */
-    public abstract int getMaxCount(int pageSize, long buf);
+    public abstract int getMaxCount(long buf, int pageSize);
 
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
@@ -263,6 +263,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param fwdBuf Forward buffer.
      * @param mid Bisection index.
      * @param cnt Initial elements count in the page being split.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void splitForwardPage(
@@ -270,9 +271,10 @@ public abstract class BPlusIO<L> extends PageIO {
         long fwdId,
         long fwdBuf,
         int mid,
-        int cnt
+        int cnt,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(fwdBuf, fwdId);
+        initNewPage(fwdBuf, fwdId, pageSize);
 
         cnt -= mid;
 
@@ -320,13 +322,13 @@ public abstract class BPlusIO<L> extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public boolean merge(
-        int pageSize,
         BPlusIO<L> prntIo,
         long prnt,
         int prntIdx,
         long left,
         long right,
-        boolean emptyBranch
+        boolean emptyBranch,
+        int pageSize
     ) throws IgniteCheckedException {
         int prntCnt = prntIo.getCount(prnt);
         int leftCnt = getCount(left);
@@ -338,7 +340,7 @@ public abstract class BPlusIO<L> extends PageIO {
         if (!isLeaf() && !emptyBranch)
             newCnt++;
 
-        if (newCnt > getMaxCount(pageSize, left)) {
+        if (newCnt > getMaxCount(left, pageSize)) {
             assert !emptyBranch;
 
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
index d0ead51..7158770 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
@@ -45,7 +45,7 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(int pageSize, long buf) {
+    @Override public int getMaxCount(long buf, int pageSize) {
         // The structure of the page is the following:
         // |ITEMS_OFF|w|A|x|B|y|C|z|
         // where capital letters are data items, lowercase letters are 8 byte page references.
@@ -157,9 +157,10 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
         long leftChildId,
         L row,
         byte[] rowBytes,
-        long rightChildId
+        long rightChildId,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(newRootBuf, newRootId);
+        initNewPage(newRootBuf, newRootId, pageSize);
 
         setCount(newRootBuf, 1);
         setLeft(newRootBuf, 0, leftChildId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
index 76c621c..cf97764 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
@@ -35,7 +35,7 @@ public abstract class BPlusLeafIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(int pageSize, long buf) {
+    @Override public int getMaxCount(long buf, int pageSize) {
         return (pageSize - ITEMS_OFF) / itemSize;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
index ed75d12..fac3e20 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
+import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -58,6 +59,29 @@ public class CacheVersionIO {
      * @param ver Version to write.
      * @param allowNull Is {@code null} version allowed.
      */
+    public static void write(ByteBuffer buf, GridCacheVersion ver, boolean allowNull) {
+        if (ver == null) {
+            if (allowNull)
+                buf.put(NULL_PROTO_VER);
+            else
+                throw new IllegalStateException("Cache version is null");
+        }
+        else {
+            byte protoVer = 1; // Version of serialization protocol.
+
+            buf.put(protoVer);
+            buf.putInt(ver.topologyVersion());
+            buf.putInt(ver.nodeOrderAndDrIdRaw());
+            buf.putLong(ver.globalTime());
+            buf.putLong(ver.order());
+        }
+    }
+
+    /**
+     * @param buf Byte buffer.
+     * @param ver Version to write.
+     * @param allowNull Is {@code null} version allowed.
+     */
     public static void write(long buf, GridCacheVersion ver, boolean allowNull) {
         if (ver == null) {
             if (allowNull)
@@ -102,6 +126,53 @@ public class CacheVersionIO {
      * @return Size of serialized version.
      * @throws IgniteCheckedException If failed.
      */
+    public static int readSize(ByteBuffer buf, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(buf.get(buf.position()), allowNull);
+
+        switch (protoVer) {
+            case NULL_PROTO_VER:
+                return NULL_SIZE;
+
+            case 1:
+                return SIZE_V1;
+
+            default:
+                throw new IllegalStateException();
+        }
+    }
+
+    /**
+     * Reads GridCacheVersion instance from the given buffer. Moves buffer's position by the number of used
+     * bytes.
+     *
+     * @param buf Byte buffer.
+     * @param allowNull Is {@code null} version allowed.
+     * @return Version.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static GridCacheVersion read(ByteBuffer buf, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(buf.get(), allowNull);
+
+        if (protoVer == NULL_PROTO_VER)
+            return null;
+
+        int topVer = buf.getInt();
+        int nodeOrderDrId = buf.getInt();
+        long globalTime = buf.getLong();
+        long order = buf.getLong();
+
+        return new GridCacheVersion(topVer, nodeOrderDrId, globalTime, order);
+    }
+
+    /**
+     * Gets needed buffer size to read the whole version instance.
+     * Does not change buffer position.
+     *
+     * @param buf Buffer.
+     * @param allowNull Is {@code null} version allowed.
+     * @return Size of serialized version.
+     * @throws IgniteCheckedException If failed.
+     */
     public static int readSize(long buf, boolean allowNull) throws IgniteCheckedException {
         byte protoVer = checkProtocolVersion(PageUtils.getByte(buf, 0), allowNull);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index ca8ea39..c9baf05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -88,21 +89,22 @@ public class DataPageIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
-        setEmptyPage(buf);
+        setEmptyPage(buf, pageSize);
         setFreeListPageId(buf, 0L);
     }
 
     /**
      * @param buf Buffer.
+     * @param pageSize Page size.
      */
-    private void setEmptyPage(long buf) {
+    private void setEmptyPage(long buf, int pageSize) {
         setDirectCount(buf, 0);
         setIndirectCount(buf, 0);
-        setFirstEntryOffset(buf, buf.capacity());
-        setRealFreeSpace(buf, buf.capacity() - ITEMS_OFF);
+        setFirstEntryOffset(buf, pageSize, pageSize);
+        setRealFreeSpace(buf, pageSize - ITEMS_OFF, pageSize);
     }
 
     /**
@@ -163,9 +165,10 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param dataOff Entry data offset.
+     * @param pageSize Page size.
      */
-    private void setFirstEntryOffset(long buf, int dataOff) {
-        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= buf.capacity(): dataOff;
+    private void setFirstEntryOffset(long buf, int dataOff, int pageSize) {
+        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= pageSize : dataOff;
 
         PageUtils.putShort(buf, FIRST_ENTRY_OFF, (short)dataOff);
     }
@@ -182,8 +185,8 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param freeSpace Free space.
      */
-    private void setRealFreeSpace(long buf, int freeSpace) {
-        assert freeSpace == actualFreeSpace(buf): freeSpace + " != " + actualFreeSpace(buf);
+    private void setRealFreeSpace(long buf, int freeSpace, int pageSize) {
+        assert freeSpace == actualFreeSpace(buf, pageSize): freeSpace + " != " + actualFreeSpace(buf, pageSize);
 
         PageUtils.putShort(buf, FREE_SPACE_OFF, (short)freeSpace);
     }
@@ -218,7 +221,7 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * Equivalent for {@link #actualFreeSpace(long)} but reads saved value.
+     * Equivalent for {@link #actualFreeSpace(long, int)} but reads saved value.
      *
      * @param buf Buffer.
      * @return Free space.
@@ -316,9 +319,10 @@ public class DataPageIO extends PageIO {
 
     /**
      * @param buf Buffer.
+     * @param pageSize Page size.
      * @return String representation.
      */
-    public String printPageLayout(long buf) {
+    public String printPageLayout(long buf, int pageSize) {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
         int free = getRealFreeSpace(buf);
@@ -337,7 +341,7 @@ public class DataPageIO extends PageIO {
 
             short item = getItem(buf, i);
 
-            if (item < ITEMS_OFF || item >= buf.capacity())
+            if (item < ITEMS_OFF || item >= pageSize)
                 valid = false;
 
             entriesSize += getPageEntrySize(buf, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
@@ -375,7 +379,7 @@ public class DataPageIO extends PageIO {
 
         b.a("][free=").a(free);
 
-        int actualFree = buf.capacity() - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
+        int actualFree = pageSize - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
 
         if (free != actualFree) {
             b.a(", actualFree=").a(actualFree);
@@ -395,19 +399,19 @@ public class DataPageIO extends PageIO {
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @return Data entry offset in bytes.
      */
-    private int getDataOffset(long buf, int itemId) {
+    private int getDataOffset(long buf, int itemId, int pageSize) {
         assert checkIndex(itemId): itemId;
 
         int directCnt = getDirectCount(buf);
 
-        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(buf);
+        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(buf, pageSize);
 
         if (itemId >= directCnt) { // Need to do indirect lookup.
             int indirectCnt = getIndirectCount(buf);
 
             // Must have indirect items here.
             assert indirectCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", indirectCnt=" + indirectCnt +
-                ", page=" + printPageLayout(buf);
+                ", page=" + printPageLayout(buf, pageSize);
 
             int indirectItemIdx = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
 
@@ -449,18 +453,16 @@ public class DataPageIO extends PageIO {
      * @param itemId Item to position on.
      * @return Link to the next fragment or {@code 0} if it is the last fragment or the data row is not fragmented.
      */
-    public long setPositionAndLimitOnPayload(final long buf, final int itemId) {
-        int dataOff = getDataOffset(buf, itemId);
+    public DataPagePayload readPayload(final long buf, final int itemId, final int pageSize) {
+        int dataOff = getDataOffset(buf, itemId, pageSize);
 
         boolean fragmented = isFragmented(buf, dataOff);
         long nextLink = fragmented ? getNextFragmentLink(buf, dataOff) : 0;
         int payloadSize = getPageEntrySize(buf, dataOff, 0);
 
-        buf.position(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0));
-
-        buf.limit(buf.position() + payloadSize);
-
-        return nextLink;
+        return new DataPagePayload(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0),
+            payloadSize,
+            nextLink);
     }
 
     /**
@@ -593,13 +595,14 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
+     * @param pageSize Page size.
      * @return Next link for fragmented entries or {@code 0} if none.
      * @throws IgniteCheckedException If failed.
      */
-    public long removeRow(long buf, int itemId) throws IgniteCheckedException {
+    public long removeRow(long buf, int itemId, int pageSize) throws IgniteCheckedException {
         assert checkIndex(itemId) : itemId;
 
-        final int dataOff = getDataOffset(buf, itemId);
+        final int dataOff = getDataOffset(buf, itemId, pageSize);
         final long nextLink = isFragmented(buf, dataOff) ? getNextFragmentLink(buf, dataOff) : 0;
 
         // Record original counts to calculate delta in free space in the end of remove.
@@ -615,7 +618,7 @@ public class DataPageIO extends PageIO {
             assert (indirectCnt == 0 && itemId == 0) ||
                 (indirectCnt == 1 && itemId == itemId(getItem(buf, 1))) : itemId;
 
-            setEmptyPage(buf);
+            setEmptyPage(buf, pageSize);
         }
         else {
             // Get the entry size before the actual remove.
@@ -642,15 +645,15 @@ public class DataPageIO extends PageIO {
 
             if (indirectId == 0) {// For the last direct item with no indirect item.
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectCnt, -1);
+                    moveItems(buf, directCnt, indirectCnt, -1, pageSize);
                 else
                     curIndirectCnt++;
             }
             else {
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectId - directCnt, -1);
+                    moveItems(buf, directCnt, indirectId - directCnt, -1, pageSize);
 
-                moveItems(buf, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1);
+                moveItems(buf, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1, pageSize);
 
                 if (dropLast)
                     curIndirectCnt--;
@@ -662,8 +665,9 @@ public class DataPageIO extends PageIO {
             assert getIndirectCount(buf) <= getDirectCount(buf);
 
             // Increase free space.
-            setRealFreeSpace(buf, getRealFreeSpace(buf) + rmvEntrySize +
-                ITEM_SIZE * (directCnt - getDirectCount(buf) + indirectCnt - getIndirectCount(buf)));
+            setRealFreeSpace(buf,
+                getRealFreeSpace(buf) + rmvEntrySize + ITEM_SIZE * (directCnt - getDirectCount(buf) + indirectCnt - getIndirectCount(buf)),
+                pageSize);
         }
 
         return nextLink;
@@ -674,12 +678,13 @@ public class DataPageIO extends PageIO {
      * @param idx Index.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveItems(long buf, int idx, int cnt, int step) {
+    private void moveItems(long buf, int idx, int cnt, int step, int pageSize) {
         assert cnt >= 0: cnt;
 
         if (cnt != 0)
-            moveBytes(buf, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE);
+            moveBytes(buf, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE, pageSize);
     }
 
     /**
@@ -699,12 +704,14 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param row Cache data row.
      * @param rowSize Row size.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
         long buf,
         CacheDataRow row,
-        int rowSize
+        int rowSize,
+        int pageSize
     ) throws IgniteCheckedException {
         assert rowSize <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
 
@@ -713,11 +720,11 @@ public class DataPageIO extends PageIO {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         writeRowData(buf, dataOff, rowSize, row);
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         setLink(row, buf, itemId);
     }
@@ -726,11 +733,14 @@ public class DataPageIO extends PageIO {
      * Adds row to this data page and sets respective link to the given row object.
      *
      * @param buf Buffer.
+     * @param payload Payload.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
         long buf,
-        byte[] payload
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
         assert payload.length <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
 
@@ -739,11 +749,11 @@ public class DataPageIO extends PageIO {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         writeRowData(buf, dataOff, payload);
 
-        addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
     }
 
     /**
@@ -752,6 +762,7 @@ public class DataPageIO extends PageIO {
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff First entry offset.
+     * @param pageSize Page size.
      * @return First entry offset after compaction.
      */
     private int compactIfNeed(
@@ -759,10 +770,11 @@ public class DataPageIO extends PageIO {
         final int entryFullSize,
         final int directCnt,
         final int indirectCnt,
-        int dataOff
+        int dataOff,
+        int pageSize
     ) {
         if (!isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt)) {
-            dataOff = compactDataEntries(buf, directCnt);
+            dataOff = compactDataEntries(buf, directCnt, pageSize);
 
             assert isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt);
         }
@@ -778,20 +790,27 @@ public class DataPageIO extends PageIO {
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff Data offset.
+     * @param pageSize Page size.
      * @return Item ID.
      */
-    private int addItem(final long buf, final int fullEntrySize, final int directCnt,
-        final int indirectCnt, final int dataOff) {
-        setFirstEntryOffset(buf, dataOff);
+    private int addItem(final long buf,
+        final int fullEntrySize,
+        final int directCnt,
+        final int indirectCnt,
+        final int dataOff,
+        final int pageSize)
+    {
+        setFirstEntryOffset(buf, dataOff, pageSize);
 
-        int itemId = insertItem(buf, dataOff, directCnt, indirectCnt);
+        int itemId = insertItem(buf, dataOff, directCnt, indirectCnt, pageSize);
 
         assert checkIndex(itemId): itemId;
         assert getIndirectCount(buf) <= getDirectCount(buf);
 
         // Update free space. If number of indirect items changed, then we were able to reuse an item slot.
-        setRealFreeSpace(buf, getRealFreeSpace(buf) - fullEntrySize +
-            (getIndirectCount(buf) != indirectCnt ? ITEM_SIZE : 0));
+        setRealFreeSpace(buf,
+            getRealFreeSpace(buf) - fullEntrySize + (getIndirectCount(buf) != indirectCnt ? ITEM_SIZE : 0),
+            pageSize);
 
         return itemId;
     }
@@ -801,13 +820,14 @@ public class DataPageIO extends PageIO {
      * @param fullEntrySize Full entry size.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Offset in the buffer where the entry must be written.
      */
-    private int getDataOffsetForWrite(long buf, int fullEntrySize, int directCnt, int indirectCnt) {
+    private int getDataOffsetForWrite(long buf, int fullEntrySize, int directCnt, int indirectCnt, int pageSize) {
         int dataOff = getFirstEntryOffset(buf);
 
         // Compact if we do not have enough space for entry.
-        dataOff = compactIfNeed(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        dataOff = compactIfNeed(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         // We will write data right before the first entry.
         dataOff -= fullEntrySize - ITEM_SIZE;
@@ -829,9 +849,10 @@ public class DataPageIO extends PageIO {
         long buf,
         CacheDataRow row,
         int written,
-        int rowSize
+        int rowSize,
+        int pageSize
     ) throws IgniteCheckedException {
-        return addRowFragment(buf, written, rowSize, row.link(), row, null);
+        return addRowFragment(buf, written, rowSize, row.link(), row, null, pageSize);
     }
 
     /**
@@ -845,9 +866,10 @@ public class DataPageIO extends PageIO {
     public void addRowFragment(
         long buf,
         byte[] payload,
-        long lastLink
+        long lastLink,
+        int pageSize
     ) throws IgniteCheckedException {
-        addRowFragment(buf, 0, 0, lastLink, null, payload);
+        addRowFragment(buf, 0, 0, lastLink, null, payload, pageSize);
     }
 
     /**
@@ -868,7 +890,8 @@ public class DataPageIO extends PageIO {
         int rowSize,
         long lastLink,
         CacheDataRow row,
-        byte[] payload
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
         assert payload == null ^ row == null;
 
@@ -879,20 +902,24 @@ public class DataPageIO extends PageIO {
             Math.min(rowSize - written, getFreeSpace(buf));
 
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
-
-        PageUtils.putShort(buf, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
-        PageUtils.putLong(buf, dataOff + 2, lastLink);
+        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         if (payload == null) {
+            ByteBuffer buf0 = null;
+
+            buf0.position(dataOff);
+
+            buf0.putShort((short) (payloadSize | FRAGMENTED_FLAG));
+            buf0.putLong(lastLink);
+
             int rowOff = rowSize - written - payloadSize;
 
-            writeFragmentData(row, buf + dataOff + 10, rowOff, payloadSize);
+            writeFragmentData(row, buf0, rowOff, payloadSize);
         }
         else
-            PageUtils.putBytes(buf, dataOff + 10, payload);
+            PageUtils.putBytes(buf, dataOff, payload);
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         if (row != null)
             setLink(row, buf, itemId);
@@ -920,7 +947,7 @@ public class DataPageIO extends PageIO {
      */
     private void writeFragmentData(
         final CacheDataRow row,
-        final long buf,
+        final ByteBuffer buf,
         final int rowOff,
         final int payloadSize
     ) throws IgniteCheckedException {
@@ -946,7 +973,7 @@ public class DataPageIO extends PageIO {
      */
     private int writeFragment(
         final CacheDataRow row,
-        final long buf,
+        final ByteBuffer buf,
         final int rowOff,
         final int payloadSize,
         final EntryPart type,
@@ -1080,9 +1107,10 @@ public class DataPageIO extends PageIO {
      * @param dataOff Data offset.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Item ID (insertion index).
      */
-    private int insertItem(long buf, int dataOff, int directCnt, int indirectCnt) {
+    private int insertItem(long buf, int dataOff, int directCnt, int indirectCnt, int pageSize) {
         if (indirectCnt > 0) {
             // If the first indirect item is on correct place to become the last direct item, do the transition
             // and insert the new item into the free slot which was referenced by this first indirect item.
@@ -1102,7 +1130,7 @@ public class DataPageIO extends PageIO {
         }
 
         // Move all the indirect items forward to make a free slot and insert new item at the end of direct items.
-        moveItems(buf, directCnt, indirectCnt, +1);
+        moveItems(buf, directCnt, indirectCnt, +1, pageSize);
 
         setItem(buf, directCnt, directItemFromOffset(dataOff));
 
@@ -1115,9 +1143,10 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param directCnt Direct items count.
+     * @param pageSize Page size.
      * @return New first entry offset.
      */
-    private int compactDataEntries(long buf, int directCnt) {
+    private int compactDataEntries(long buf, int directCnt, int pageSize) {
         assert checkCount(directCnt): directCnt;
 
         int[] offs = new int[directCnt];
@@ -1131,7 +1160,7 @@ public class DataPageIO extends PageIO {
         Arrays.sort(offs);
 
         // Move right all of the entries if possible to make the page as compact as possible to its tail.
-        int prevOff = buf.capacity();
+        int prevOff = pageSize;
 
         for (int i = directCnt - 1; i >= 0; i--) {
             int off = offs[i] >>> 8;
@@ -1145,7 +1174,7 @@ public class DataPageIO extends PageIO {
             if (delta != 0) { // Move right.
                 assert delta > 0: delta;
 
-                moveBytes(buf, off, entrySize, delta);
+                moveBytes(buf, off, entrySize, delta, pageSize);
 
                 int itemId = offs[i] & 0xFF;
 
@@ -1164,9 +1193,10 @@ public class DataPageIO extends PageIO {
      * Full-scan free space calculation procedure.
      *
      * @param buf Buffer to scan.
+     * @param pageSize Page size.
      * @return Actual free space in the buffer.
      */
-    private int actualFreeSpace(long buf) {
+    private int actualFreeSpace(long buf, int pageSize) {
         int directCnt = getDirectCount(buf);
 
         int entriesSize = 0;
@@ -1179,7 +1209,7 @@ public class DataPageIO extends PageIO {
             entriesSize += entrySize;
         }
 
-        return buf.capacity() - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(buf)) * ITEM_SIZE;
+        return pageSize - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(buf)) * ITEM_SIZE;
     }
 
     /**
@@ -1187,12 +1217,13 @@ public class DataPageIO extends PageIO {
      * @param off Offset.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveBytes(long buf, int off, int cnt, int step) {
+    private void moveBytes(long buf, int off, int cnt, int step, int pageSize) {
         assert step != 0: step;
         assert off + step >= 0;
-        assert off + step + cnt <= buf.capacity() : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
-            ", cap=" + buf.capacity() + ']';
+        assert off + step + cnt <= pageSize : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
+            ", cap=" + pageSize + ']';
 
         PageHandler.copyMemory(buf, buf, off, off + step, cnt);
     }
@@ -1210,26 +1241,36 @@ public class DataPageIO extends PageIO {
         int payloadSize,
         CacheDataRow row
     ) throws IgniteCheckedException {
-        try {
-            buf.position(dataOff);
+        PageUtils.putShort(buf, dataOff, (short)payloadSize);
+        dataOff += 2;
 
-            buf.putShort((short)payloadSize);
+        byte[] bytes = row.key().valueBytes(null);
 
-            boolean ok = row.key().putValue(buf);
+        PageUtils.putInt(buf, dataOff, bytes.length);
+        dataOff += 4;
 
-            assert ok;
+        PageUtils.putBytes(buf, dataOff, bytes);
+        dataOff += bytes.length;
 
-            ok = row.value().putValue(buf);
+        PageUtils.putByte(buf, dataOff, row.key().cacheObjectType());
+        dataOff++;
 
-            assert ok;
+        bytes = row.value().valueBytes(null);
 
-            CacheVersionIO.write(buf, row.version(), false);
+        PageUtils.putInt(buf, dataOff, bytes.length);
+        dataOff += 4;
 
-            buf.putLong(row.expireTime());
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putBytes(buf, dataOff, bytes);
+        dataOff += bytes.length;
+
+        PageUtils.putByte(buf, dataOff, row.value().cacheObjectType());
+        dataOff++;
+
+        CacheVersionIO.write(buf + dataOff, row.version(), false);
+
+        dataOff += CacheVersionIO.size(row.version(), false);
+
+        PageUtils.putLong(buf, dataOff, row.expireTime());
     }
 
     /**
@@ -1242,15 +1283,9 @@ public class DataPageIO extends PageIO {
         int dataOff,
         byte[] payload
     ) {
-        try {
-            buf.position(dataOff);
+        PageUtils.putShort(buf, dataOff, (short)payload.length);
+        dataOff += 2;
 
-            buf.putShort((short)payload.length);
-
-            buf.put(payload);
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putBytes(buf, dataOff, payload);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
new file mode 100644
index 0000000..203429e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.database.tree.io;
+
+/**
+ *
+ */
+public class DataPagePayload {
+    /** */
+    private final int off;
+
+    /** */
+    private final int payloadSize;
+
+    /** */
+    private final long nextLink;
+
+    DataPagePayload(int off, int payloadSize, long nextLink) {
+        this.off = off;
+        this.payloadSize = payloadSize;
+        this.nextLink = nextLink;
+    }
+
+    public int offset() {
+        return off;
+    }
+
+    public int payloadSize() {
+        return payloadSize;
+    }
+
+    public long nextLink() {
+        return nextLink;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 12646ae..3aee268 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
@@ -275,7 +275,7 @@ public abstract class PageIO {
      * @param buf Buffer.
      * @param pageId Page ID.
      */
-    public void initNewPage(long buf, long pageId) {
+    public void initNewPage(long buf, long pageId, int pageSize) {
         setType(buf, getType());
         setVersion(buf, getVersion());
         setPageId(buf, pageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index fb8762e..7ef4db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@ -68,8 +68,8 @@ public class PageMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setTreeRoot(buf, 0);
         setReuseListRoot(buf, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
index 72893fb..531ba1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
@@ -42,8 +42,8 @@ public class PagePartitionMetaIO extends PageMetaIO {
     );
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setSize(buf, 0);
         setUpdateCounter(buf, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
index 81feb51..4beef7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
@@ -126,7 +126,7 @@ public class TrackingPageIO extends PageIO {
         if (last <= lastSuccessfulBackupId) { //we can drop our data
             PageUtils.putLong(buf, LAST_BACKUP_OFFSET, nextBackupId);
 
-            PageHandler.zeroMemory(buf, SIZE_FIELD_OFFSET, buf.capacity() - SIZE_FIELD_OFFSET);
+            PageHandler.zeroMemory(buf, SIZE_FIELD_OFFSET, pageSize - SIZE_FIELD_OFFSET);
         }
         else { //we can't drop data, it is still necessary for incremental backups
             int len = cntOfPage >> 3;

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 df25449..39bb44b 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
@@ -253,7 +253,7 @@ public abstract class PageHandler<X, R> {
 
         long pageId = page.id();
 
-        init.initNewPage(buf, pageId);
+        init.initNewPage(buf, pageId, page.size());
 
         // Here we should never write full page, because it is known to be new.
         page.fullPageWalRecordPolicy(FALSE);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 97debf1..3e78b9c 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
@@ -32,6 +32,7 @@ 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.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -190,7 +191,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that initial pages are accessible.
             for (FullPageId id : old) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long buf = page.getForWritePointer();
 
                     assertNotNull(buf);
 
@@ -231,7 +232,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that updated pages are accessible using new IDs.
             for (FullPageId id : updated) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long buf = page.getForWritePointer();
 
                     assertNotNull(buf);
 
@@ -242,7 +243,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                         page.releaseWrite(false);
                     }
 
-                    buf = page.getForRead();
+                    buf = page.getForReadPointer();
 
                     assertNotNull(buf);
 
@@ -282,13 +283,13 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
      * @param val Value to write.
      */
     private void writePage(Page page, int val) {
-        ByteBuffer bytes = page.getForWrite();
+        long bytes = page.getForWritePointer();
 
         try {
             PageIO.setPageId(bytes, page.id());
 
             for (int i = PageIO.COMMON_HEADER_END; i < PAGE_SIZE; i++)
-                bytes.put(i, (byte)val);
+                PageUtils.putByte(bytes, i, (byte)val);
         }
         finally {
             page.releaseWrite(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
index 1495cc0..f4b0cc9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
@@ -8,6 +8,10 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ThreadLocalRandom;
 import junit.framework.TestCase;
+import org.apache.ignite.internal.util.GridUnsafe;
+import sun.misc.JavaNioAccess;
+import sun.misc.SharedSecrets;
+import sun.nio.ch.DirectBuffer;
 
 /**
  *
@@ -16,14 +20,29 @@ public class TrackingPageIOTest extends TestCase {
     /** Page size. */
     public static final int PAGE_SIZE = 2048;
 
+    /** */
     private final TrackingPageIO io = TrackingPageIO.VERSIONS.latest();
 
+    /** */
+    private long buf;
+
+    @Override protected void setUp() throws Exception {
+        super.setUp();
+
+        buf = GridUnsafe.allocateMemory(PAGE_SIZE);
+    }
+
+    @Override protected void tearDown() throws Exception {
+        if (buf != 0L)
+            GridUnsafe.freeMemory(buf);
+
+        super.tearDown();
+    }
+
     /**
      *
      */
     public void testBasics() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
-
         io.markChanged(buf, 2, 0, -1, PAGE_SIZE);
 
         assertTrue(io.wasChanged(buf, 2, 0, -1, PAGE_SIZE));
@@ -33,12 +52,18 @@ public class TrackingPageIOTest extends TestCase {
         assertFalse(io.wasChanged(buf, 2, 1,  0, PAGE_SIZE));
     }
 
+    private long allocate() {
+        return GridUnsafe.allocateMemory(PAGE_SIZE);
+    }
+
+    private void free(long addr) {
+        GridUnsafe.freeMemory(addr);
+    }
+
     /**
      *
      */
     public void testMarkingRandomly() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
-
         int cntOfPageToTrack = io.countOfPageToTrack(PAGE_SIZE);
 
         for (int i = 0; i < 1001; i++)
@@ -49,8 +74,6 @@ public class TrackingPageIOTest extends TestCase {
      *
      */
     public void testZeroingRandomly() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
-
         for (int i = 0; i < 1001; i++)
             checkMarkingRandomly(buf, i, true);
     }
@@ -59,7 +82,7 @@ public class TrackingPageIOTest extends TestCase {
      * @param buf Buffer.
      * @param backupId Backup id.
      */
-    private void checkMarkingRandomly(ByteBuffer buf, int backupId, boolean testZeroing) {
+    private void checkMarkingRandomly(long buf, int backupId, boolean testZeroing) {
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
         int track = io.countOfPageToTrack(PAGE_SIZE);
@@ -106,8 +129,6 @@ public class TrackingPageIOTest extends TestCase {
     }
 
     public void testFindNextChangedPage() throws Exception {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
-
         for (int i = 0; i < 101; i++)
             checkFindingRandomly(buf, i);
     }
@@ -116,7 +137,7 @@ public class TrackingPageIOTest extends TestCase {
      * @param buf Buffer.
      * @param backupId Backup id.
      */
-    private void checkFindingRandomly(ByteBuffer buf, int backupId) {
+    private void checkFindingRandomly(long buf, int backupId) {
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
         int track = io.countOfPageToTrack(PAGE_SIZE);
@@ -158,8 +179,6 @@ public class TrackingPageIOTest extends TestCase {
     }
 
     public void testMerging() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
-
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
         int track = io.countOfPageToTrack(PAGE_SIZE);
@@ -193,8 +212,6 @@ public class TrackingPageIOTest extends TestCase {
     }
 
     public void testMerging_MarksShouldBeDropForSuccessfulBackup() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
-
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
         int track = io.countOfPageToTrack(PAGE_SIZE);
@@ -226,7 +243,7 @@ public class TrackingPageIOTest extends TestCase {
     }
 
     private void generateMarking(
-        ByteBuffer buf,
+        long buf,
         int track,
         long basePageId,
         long maxPageId,

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 cfc58b0..4996eef 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
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.database;
 
-import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
@@ -88,14 +87,14 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, ByteBuffer buf) {
+        @Override public void onReadLock(Page page, long buf) {
             checkPageId(page, buf);
 
             assertTrue(readLocks.get().add(page.id()));
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, ByteBuffer buf) {
+        @Override public void onReadUnlock(Page page, long buf) {
             checkPageId(page, buf);
 
             assertTrue(readLocks.get().remove(page.id()));
@@ -107,8 +106,8 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, ByteBuffer buf) {
-            if (buf == null)
+        @Override public void onWriteLock(Page page, long buf) {
+            if (buf == 0L)
                 return; // Failed to lock.
 
             checkPageId(page, buf);
@@ -117,7 +116,7 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
+        @Override public void onWriteUnlock(Page page, long buf) {
             assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(buf)));
 
             assertTrue(writeLocks.get().remove(page.id()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 4bc39ea..b524b45 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
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.database;
 
-import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -38,6 +37,7 @@ 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;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.database.DataStructure;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
@@ -1134,7 +1134,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
      * @param page Page.
      * @param buf Buffer.
      */
-    public static void checkPageId(Page page, ByteBuffer buf) {
+    public static void checkPageId(Page page, long buf) {
         long pageId = PageIO.getPageId(buf);
 
         // Page ID must be 0L for newly allocated page, for reused page effective ID must remain the same.
@@ -1199,7 +1199,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<Long> io, ByteBuffer buf, int idx, Long n2)
+        @Override protected int compare(BPlusIO<Long> io, long buf, int idx, Long n2)
             throws IgniteCheckedException {
             Long n1 = io.getLookupRow(this, buf, idx);
 
@@ -1207,7 +1207,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override protected Long getRow(BPlusIO<Long> io, ByteBuffer buf, int idx) throws IgniteCheckedException {
+        @Override protected Long getRow(BPlusIO<Long> io, long buf, int idx) throws IgniteCheckedException {
             assert io.canGetRow() : io;
 
             return io.getLookupRow(this, buf, idx);
@@ -1247,8 +1247,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, ByteBuffer buf) {
-            if (buf != null) {
+        @Override public void onReadLock(Page page, long buf) {
+            if (buf != 0L) {
                 long pageId = PageIO.getPageId(buf);
 
                 checkPageId(page, buf);
@@ -1260,7 +1260,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, ByteBuffer buf) {
+        @Override public void onReadUnlock(Page page, long buf) {
             checkPageId(page, buf);
 
             long pageId = PageIO.getPageId(buf);
@@ -1274,8 +1274,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, ByteBuffer buf) {
-            if (buf != null) {
+        @Override public void onWriteLock(Page page, long buf) {
+            if (buf != 0L) {
                 checkPageId(page, buf);
 
                 long pageId = PageIO.getPageId(buf);
@@ -1290,7 +1290,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
+        @Override public void onWriteUnlock(Page page, long buf) {
             assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(buf)));
 
             assertEquals(Long.valueOf(page.id()), locks(false).remove(page.id()));
@@ -1358,15 +1358,15 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public int getMaxCount(ByteBuffer buf) {
+        @Override public int getMaxCount(long buf, int pageSize) {
             if (MAX_PER_PAGE != 0)
                 return MAX_PER_PAGE;
 
-            return super.getMaxCount(buf);
+            return super.getMaxCount(buf, pageSize);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<Long> srcIo, ByteBuffer src, int srcIdx)
+        @Override public void store(long dst, int dstIdx, BPlusIO<Long> srcIo, long src, int srcIdx)
             throws IgniteCheckedException {
             Long row = srcIo.getLookupRow(null, src, srcIdx);
 
@@ -1382,16 +1382,16 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) {
+        @Override public void storeByOffset(long buf, int off, Long row) {
             checkNotRemoved(row);
 
-            buf.putLong(off, row);
+            PageUtils.putLong(buf, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, ByteBuffer buf, int idx)
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long buf, int idx)
             throws IgniteCheckedException {
-            Long row = buf.getLong(offset(idx));
+            Long row = PageUtils.getLong(buf, offset(idx));
 
             checkNotRemoved(row);
 
@@ -1433,29 +1433,29 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public int getMaxCount(ByteBuffer buf) {
+        @Override public int getMaxCount(long buf, int pageSize) {
             if (MAX_PER_PAGE != 0)
                 return MAX_PER_PAGE;
 
-            return super.getMaxCount(buf);
+            return super.getMaxCount(buf, pageSize);
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) {
-            buf.putLong(off, row);
+        @Override public void storeByOffset(long buf, int off, Long row) {
+            PageUtils.putLong(buf, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<Long> srcIo, ByteBuffer src, int srcIdx) {
+        @Override public void store(long dst, int dstIdx, BPlusIO<Long> srcIo, long src, int srcIdx) {
             assert srcIo == this;
 
-            dst.putLong(offset(dstIdx), src.getLong(offset(srcIdx)));
+            PageUtils.putLong(dst, offset(dstIdx), PageUtils.getLong(src, offset(srcIdx)));
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, ByteBuffer buf, int idx)
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long buf, int idx)
             throws IgniteCheckedException {
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(buf, offset(idx));
         }
     }
 }


[12/25] ignite git commit: test

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 8bd8ebe..ac39850 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.database;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -88,7 +87,7 @@ public class H2TreeIndex extends GridH2IndexBase {
         tree = new H2Tree(name, cctx.offheap().reuseListForIndex(name), cctx.cacheId(),
             dbMgr.pageMemory(), cctx.shared().wal(), cctx.offheap().globalRemoveId(),
             tbl.rowFactory(), page.pageId().pageId(), page.isAllocated()) {
-            @Override protected int compare(BPlusIO<SearchRow> io, ByteBuffer buf, int idx, SearchRow row)
+            @Override protected int compare(BPlusIO<SearchRow> io, long buf, int idx, SearchRow row)
                 throws IgniteCheckedException {
                 return compareRows(getRow(io, buf, idx), row);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
index 6868205..e56691e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.query.h2.database.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
@@ -44,16 +44,16 @@ public class H2InnerIO extends BPlusInnerIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void storeByOffset(ByteBuffer buf, int off, SearchRow row) {
+    @Override public void storeByOffset(long buf, int off, SearchRow row) {
         GridH2Row row0 = (GridH2Row)row;
 
         assert row0.link != 0;
 
-        buf.putLong(off, row0.link);
+        PageUtils.putLong(buf, off, row0.link);
     }
 
     /** {@inheritDoc} */
-    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, ByteBuffer buf, int idx)
+    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, long buf, int idx)
         throws IgniteCheckedException {
         long link = getLink(buf, idx);
 
@@ -61,14 +61,14 @@ public class H2InnerIO extends BPlusInnerIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<SearchRow> srcIo, ByteBuffer src, int srcIdx) {
+    @Override public void store(long dst, int dstIdx, BPlusIO<SearchRow> srcIo, long src, int srcIdx) {
         long link = ((H2RowLinkIO)srcIo).getLink(src, srcIdx);
 
-        dst.putLong(offset(dstIdx), link);
+        PageUtils.putLong(dst, offset(dstIdx), link);
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    @Override public long getLink(long buf, int idx) {
+        return PageUtils.getLong(buf, offset(idx));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
index ce517a8..67fe337 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.query.h2.database.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
@@ -44,23 +44,23 @@ public class H2LeafIO extends BPlusLeafIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void storeByOffset(ByteBuffer buf, int off, SearchRow row) {
+    @Override public void storeByOffset(long buf, int off, SearchRow row) {
         GridH2Row row0 = (GridH2Row)row;
 
         assert row0.link != 0;
 
-        buf.putLong(off, row0.link);
+        PageUtils.putLong(buf, off, row0.link);
     }
 
     /** {@inheritDoc} */
-    @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<SearchRow> srcIo, ByteBuffer src, int srcIdx) {
+    @Override public void store(long dst, int dstIdx, BPlusIO<SearchRow> srcIo, long src, int srcIdx) {
         assert srcIo == this;
 
-        dst.putLong(offset(dstIdx), getLink(src, srcIdx));
+        PageUtils.putLong(dst, offset(dstIdx), getLink(src, srcIdx));
     }
 
     /** {@inheritDoc} */
-    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, ByteBuffer buf, int idx)
+    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, long buf, int idx)
         throws IgniteCheckedException {
         long link = getLink(buf, idx);
 
@@ -68,7 +68,7 @@ public class H2LeafIO extends BPlusLeafIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    @Override public long getLink(long buf, int idx) {
+        return PageUtils.getLong(buf, offset(idx));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
index 9c4737d..852f46d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.database.io;
 
-import java.nio.ByteBuffer;
-
 /**
  * Row link IO.
  */
@@ -28,5 +26,5 @@ public interface H2RowLinkIO {
      * @param idx Index.
      * @return Row link.
      */
-    public long getLink(ByteBuffer buf, int idx);
+    public long getLink(long buf, int idx);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index e72cfa6..f8b77af 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -659,6 +659,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;


[24/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 75a9a078d6cc6026498a452bb86472770a419c8d
Parents: 1b56558
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 15:51:20 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 16:07:52 2016 +0300

----------------------------------------------------------------------
 .../cache/database/CacheDataRowAdapter.java     |  6 ++--
 .../cache/database/tree/io/BPlusMetaIO.java     | 34 ++++++++++----------
 2 files changed, 19 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/75a9a078/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 f559fd4..bb606cc 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
@@ -80,7 +80,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @return compare result.
      * @throws IgniteCheckedException if fails.
      */
-    public int compareKey(GridCacheContext cctx, long link) throws IgniteCheckedException {
+    public int compareKey(GridCacheContext cctx, final long link) throws IgniteCheckedException {
         byte[] bytes = key().valueBytes(cctx.cacheObjectContext());
 
         PageMemory pageMem = cctx.shared().database().pageMemory();
@@ -97,9 +97,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
                     itemId(link),
                     pageMem.pageSize());
 
-                link = data.nextLink();
-
-                if (link == 0) {
+                if (data.nextLink() == 0) {
                     long addr = pageAddr + data.offset();
 
                     int len = PageUtils.getInt(addr, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/75a9a078/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
index 700da39..b5f655f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
@@ -90,25 +90,25 @@ public class BPlusMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lvl Level.
      * @return First page ID at that level.
      */
-    public long getFirstPageId(long buf, int lvl) {
-        return PageUtils.getLong(buf, offset(lvl));
+    public long getFirstPageId(long pageAddr, int lvl) {
+        return PageUtils.getLong(pageAddr, offset(lvl));
     }
 
     /**
-     * @param buf    Buffer.
+     * @param pageAddr Page address.
      * @param lvl    Level.
      * @param pageId Page ID.
      */
-    private void setFirstPageId(long buf, int lvl, long pageId) {
-        assert lvl >= 0 && lvl < getLevelsCount(buf);
+    private void setFirstPageId(long pageAddr, int lvl, long pageId) {
+        assert lvl >= 0 && lvl < getLevelsCount(pageAddr);
 
-        PageUtils.putLong(buf, offset(lvl), pageId);
+        PageUtils.putLong(pageAddr, offset(lvl), pageId);
 
-        assert getFirstPageId(buf, lvl) == pageId;
+        assert getFirstPageId(pageAddr, lvl) == pageId;
     }
 
     /**
@@ -124,24 +124,24 @@ public class BPlusMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rootPageId New root page ID.
      * @param pageSize Page size.
      */
-    public void addRoot(long buf, long rootPageId, int pageSize) {
-        int lvl = getLevelsCount(buf);
+    public void addRoot(long pageAddr, long rootPageId, int pageSize) {
+        int lvl = getLevelsCount(pageAddr);
 
-        setLevelsCount(buf, lvl + 1, pageSize);
-        setFirstPageId(buf, lvl, rootPageId);
+        setLevelsCount(pageAddr, lvl + 1, pageSize);
+        setFirstPageId(pageAddr, lvl, rootPageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageSize Page size.
      */
-    public void cutRoot(long buf, int pageSize) {
-        int lvl = getRootLevel(buf);
+    public void cutRoot(long pageAddr, int pageSize) {
+        int lvl = getRootLevel(pageAddr);
 
-        setLevelsCount(buf, lvl, pageSize); // Decrease tree height.
+        setLevelsCount(pageAddr, lvl, pageSize); // Decrease tree height.
     }
 }


[20/25] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-gg-8.0.2.ea2' into ignite-gg-11810

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index 413f306,8d12f7c..5e6dd87
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@@ -68,16 -72,17 +71,17 @@@ public class PageMetaIO 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);
  
-         setTreeRoot(pageAddr, 0);
-         setReuseListRoot(pageAddr, 0);
-         setLastSuccessfulFullBackupId(pageAddr, 0);
-         setLastSuccessfulBackupId(pageAddr, 0);
-         setNextBackupId(pageAddr, 1);
-         setLastAllocatedIndex(pageAddr, 0);
-         setCandidateAllocatedIndex(pageAddr, 0);
+         setTreeRoot(buf, 0);
+         setReuseListRoot(buf, 0);
+         setLastSuccessfulFullSnapshotId(buf, 0);
+         setLastSuccessfulSnapshotId(buf, 0);
+         setNextSnapshotTag(buf, 1);
+         setLastSuccessfulSnapshotTag(buf, 0);
+         setLastAllocatedIndex(buf, 0);
+         setCandidateAllocatedIndex(buf, 0);
      }
  
      /**
@@@ -121,10 -126,10 +125,10 @@@
      }
  
      /**
 -     * @param buf Buffer.
 +     * @param pageAddr Page address.
       */
-     public long getLastSuccessfulBackupId(long pageAddr) {
-         return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_BACKUP_ID_OFF);
+     public long getLastSuccessfulSnapshotId(@NotNull ByteBuffer buf) {
+         return buf.getLong(LAST_SUCCESSFUL_SNAPSHOT_ID_OFF);
      }
  
      /**
@@@ -136,10 -141,10 +140,10 @@@
      }
  
      /**
 -     * @param buf Buffer.
 +     * @param pageAddr Page address.
       */
-     public long getLastSuccessfulFullBackupId(long pageAddr) {
-         return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF);
+     public long getLastSuccessfulFullSnapshotId(@NotNull ByteBuffer buf) {
+         return buf.getLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF);
      }
  
      /**
@@@ -174,16 -201,9 +200,9 @@@
      }
  
      /**
 -     * @param buf Buffer.
 +     * @param pageAddr Page address.
       */
 -    public int getCandidateAllocatedIndex(@NotNull ByteBuffer buf) {
 -        return buf.getInt(CANDIDATE_ALLOCATED_INDEX_OFF);
 +    public int getCandidateAllocatedIndex(long pageAddr) {
 +        return PageUtils.getInt(pageAddr, CANDIDATE_ALLOCATED_INDEX_OFF);
      }
- 
-     /**
-      * @param pageAddr Page address.
-      */
-     public long getNextBackupId(long pageAddr) {
-         return PageUtils.getLong(pageAddr, NEXT_BACKUP_ID_OFF);
-     }
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
index 4beef7d,136ebff..a2fed21
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
@@@ -147,21 -149,21 +149,21 @@@ public class TrackingPageIO extends Pag
  
                      newSize += Long.bitCount(newVal);
  
 -                    buf.putLong(sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
 +                    PageUtils.putLong(buf, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
                  }
  
-                 for (i -= 8; i < len; i ++) {
-                     byte newVal = (byte) (PageUtils.getByte(buf, sizeOff + SIZE_FIELD_SIZE + i) | PageUtils.getByte(buf, sizeOff2 + SIZE_FIELD_SIZE + i));
+                 for (; i < len; i ++) {
+                     byte newVal = (byte) (buf.get(sizeOff + SIZE_FIELD_SIZE + i) | buf.get(sizeOff2 + SIZE_FIELD_SIZE + i));
  
                      newSize += Integer.bitCount(newVal & 0xFF);
  
 -                    buf.put(sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
 +                    PageUtils.putByte(buf, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
                  }
  
 -                buf.putShort(sizeOff2, (short)newSize);
 +                PageUtils.putShort(buf, sizeOff2, (short)newSize);
              }
  
-             PageUtils.putLong(buf, LAST_BACKUP_OFFSET, nextBackupId);
+             buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag);
  
              PageHandler.zeroMemory(buf, sizeOff, len + SIZE_FIELD_SIZE);
          }
@@@ -191,10 -196,10 +196,10 @@@
  
          byte byteToTest;
  
-         if (useLeftHalf(curBackupId))
-             byteToTest = PageUtils.getByte(buf, BITMAP_OFFSET + (idxToTest >> 3));
+         if (useLeftHalf(curSnapshotTag))
+             byteToTest = buf.get(BITMAP_OFFSET + (idxToTest >> 3));
          else
 -            byteToTest = buf.get(BITMAP_OFFSET + SIZE_FIELD_SIZE + ((idxToTest + cntOfPage) >> 3));
 +            byteToTest = PageUtils.getByte(buf, BITMAP_OFFSET + SIZE_FIELD_SIZE + ((idxToTest + cntOfPage) >> 3));
  
          int testTemplate = 1 << (idxToTest & 0b111);
  
@@@ -214,10 -219,10 +219,10 @@@
          if (dif != 0 && dif != 1)
              return -1;
  
-         if (useLeftHalf(backupId))
-             return PageUtils.getShort(buf, SIZE_FIELD_OFFSET);
+         if (useLeftHalf(snapshotTag))
+             return buf.getShort(SIZE_FIELD_OFFSET);
          else
 -            return buf.getShort(BITMAP_OFFSET + (countOfPageToTrack(pageSize) >> 3));
 +            return PageUtils.getShort(buf, BITMAP_OFFSET + (countOfPageToTrack(pageSize) >> 3));
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
index ccd8dee,fadf74d..222a6a0
--- 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
@@@ -285,11 -275,38 +285,16 @@@ public abstract class PageHandler<X, R
              (page.fullPageWalRecordPolicy() == FALSE || page.isDirty());
      }
  
 -    /**
 -     * @param src Source.
 -     * @param dst Destination.
 -     * @param srcOff Source offset in bytes.
 -     * @param dstOff Destination offset in bytes.
 -     * @param cnt Bytes count to copy.
 -     */
 -    public static void copyMemory(ByteBuffer src, ByteBuffer dst, long srcOff, long dstOff, long cnt) {
 -        byte[] srcArr = src.hasArray() ? src.array() : null;
 -        byte[] dstArr = dst.hasArray() ? dst.array() : null;
 -        long srcArrOff = src.hasArray() ? src.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
 -        long dstArrOff = dst.hasArray() ? dst.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
 -
 -        long srcPtr = src.isDirect() ? ((DirectBuffer)src).address() : 0;
 -        long dstPtr = dst.isDirect() ? ((DirectBuffer)dst).address() : 0;
 -
 -        GridUnsafe.copyMemory(srcArr, srcPtr + srcArrOff + srcOff, dstArr, dstPtr + dstArrOff + dstOff, cnt);
 +    public static void copyMemory(long srcAddr, long dstAddr, long srcOff, long dstOff, long cnt) {
 +        GridUnsafe.copyMemory(null, srcAddr + srcOff, null, dstAddr + dstOff, cnt);
      }
  
+     /**
 -     * Will zero memory in buf
 -     * @param buf Buffer.
++     * @param addr Address.
+      * @param off Offset.
+      * @param len Length.
+      */
 -    public static void zeroMemory(ByteBuffer buf, int off, int len) {
 -        if (buf.isDirect())
 -            GridUnsafe.setMemory(((DirectBuffer)buf).address() + off, len, (byte)0);
 -
 -        else {
 -            for (int i = off; i < off + len; i++)
 -                buf.put(i, (byte)0); //TODO Optimize!
 -        }
 +    public static void zeroMemory(long addr, int off, int len) {
 +        GridUnsafe.setMemory(addr + off, len, (byte)0);
      }
  }

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------


[19/25] ignite git commit: 11810

Posted by sb...@apache.org.
11810


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

Branch: refs/heads/ignite-gg-11810
Commit: df4f0f062eec78322482c4d1741f897272a2adf0
Parents: a481704
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 13:55:58 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 13:55:58 2016 +0300

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/ignite/internal/pagemem/Page.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df4f0f06/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
index 4f32a7b..4ecad9e 100644
--- 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
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.pagemem;
 
 import java.nio.ByteBuffer;
-import org.jetbrains.annotations.Nullable;
 
 /**
  *


[11/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: e3420e9fc7114b9ad3343f9eeaa3593efae14810
Parents: 6afd2e7
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 26 13:25:57 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 26 13:25:57 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/database/CacheDataRowAdapter.java  | 12 ++++++------
 .../processors/cache/database/tree/io/DataPageIO.java   | 12 ++++++------
 2 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e3420e9f/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 1e7a99b..38b19a1 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
@@ -196,12 +196,12 @@ public class CacheDataRowAdapter implements CacheDataRow {
         if (len == 0)
             key = null;
         else {
-            byte[] bytes = PageUtils.getBytes(buf, off, len);
-            off += len;
-
             byte type = PageUtils.getByte(buf, off);
             off++;
 
+            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            off += len;
+
             key = coctx.processor().toKeyCacheObject(coctx, type, bytes);
         }
 
@@ -217,12 +217,12 @@ public class CacheDataRowAdapter implements CacheDataRow {
         if (len == 0)
             val = null;
         else {
-            byte[] bytes = PageUtils.getBytes(buf, off, len);
-            off += len;
-
             byte type = PageUtils.getByte(buf, off);
             off++;
 
+            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            off += len;
+
             val = coctx.processor().toCacheObject(coctx, type, bytes);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3420e9f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index 0aaa2a4..1034220 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -1282,23 +1282,23 @@ public class DataPageIO extends PageIO {
         PageUtils.putInt(buf, dataOff, bytes.length);
         dataOff += 4;
 
-        PageUtils.putBytes(buf, dataOff, bytes);
-        dataOff += bytes.length;
-
         PageUtils.putByte(buf, dataOff, row.key().cacheObjectType());
         dataOff++;
 
+        PageUtils.putBytes(buf, dataOff, bytes);
+        dataOff += bytes.length;
+
         bytes = row.value().valueBytes(null);
 
         PageUtils.putInt(buf, dataOff, bytes.length);
         dataOff += 4;
 
-        PageUtils.putBytes(buf, dataOff, bytes);
-        dataOff += bytes.length;
-
         PageUtils.putByte(buf, dataOff, row.value().cacheObjectType());
         dataOff++;
 
+        PageUtils.putBytes(buf, dataOff, bytes);
+        dataOff += bytes.length;
+
         CacheVersionIO.write(buf + dataOff, row.version(), false);
 
         dataOff += CacheVersionIO.size(row.version(), false);


[15/25] ignite git commit: 11810

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index 7ef4db0..413f306 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@ -68,122 +68,122 @@ public class PageMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId, int pageSize) {
-        super.initNewPage(buf, pageId, pageSize);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setTreeRoot(buf, 0);
-        setReuseListRoot(buf, 0);
-        setLastSuccessfulFullBackupId(buf, 0);
-        setLastSuccessfulBackupId(buf, 0);
-        setNextBackupId(buf, 1);
-        setLastAllocatedIndex(buf, 0);
-        setCandidateAllocatedIndex(buf, 0);
+        setTreeRoot(pageAddr, 0);
+        setReuseListRoot(pageAddr, 0);
+        setLastSuccessfulFullBackupId(pageAddr, 0);
+        setLastSuccessfulBackupId(pageAddr, 0);
+        setNextBackupId(pageAddr, 1);
+        setLastAllocatedIndex(pageAddr, 0);
+        setCandidateAllocatedIndex(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Meta store root page.
      */
-    public long getTreeRoot(long buf) {
-        return PageUtils.getLong(buf, METASTORE_ROOT_OFF);
+    public long getTreeRoot(long pageAddr) {
+        return PageUtils.getLong(pageAddr, METASTORE_ROOT_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param metastoreRoot metastore root
      */
-    public void setTreeRoot(long buf, long metastoreRoot) {
-        PageUtils.putLong(buf, METASTORE_ROOT_OFF, metastoreRoot);
+    public void setTreeRoot(long pageAddr, long metastoreRoot) {
+        PageUtils.putLong(pageAddr, METASTORE_ROOT_OFF, metastoreRoot);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Reuse list root page.
      */
-    public long getReuseListRoot(long buf) {
-        return PageUtils.getLong(buf, REUSE_LIST_ROOT_OFF);
+    public long getReuseListRoot(long pageAddr) {
+        return PageUtils.getLong(pageAddr, REUSE_LIST_ROOT_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Root page ID.
      */
-    public void setReuseListRoot(long buf, long pageId) {
-        PageUtils.putLong(buf, REUSE_LIST_ROOT_OFF, pageId);
+    public void setReuseListRoot(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, REUSE_LIST_ROOT_OFF, pageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulBackupId Last successful backup id.
      */
-    public void setLastSuccessfulBackupId(long buf, long lastSuccessfulBackupId) {
-        PageUtils.putLong(buf, LAST_SUCCESSFUL_BACKUP_ID_OFF, lastSuccessfulBackupId);
+    public void setLastSuccessfulBackupId(long pageAddr, long lastSuccessfulBackupId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_BACKUP_ID_OFF, lastSuccessfulBackupId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulBackupId(long buf) {
-        return PageUtils.getLong(buf, LAST_SUCCESSFUL_BACKUP_ID_OFF);
+    public long getLastSuccessfulBackupId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_BACKUP_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulFullBackupId Last successful full backup id.
      */
-    public void setLastSuccessfulFullBackupId(long buf, long lastSuccessfulFullBackupId) {
-        PageUtils.putLong(buf, LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF, lastSuccessfulFullBackupId);
+    public void setLastSuccessfulFullBackupId(long pageAddr, long lastSuccessfulFullBackupId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF, lastSuccessfulFullBackupId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulFullBackupId(long buf) {
-        return PageUtils.getLong(buf, LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF);
+    public long getLastSuccessfulFullBackupId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_BACKUP_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextBackupId Next backup id.
      */
-    public void setNextBackupId(long buf, long nextBackupId) {
-        PageUtils.putLong(buf, NEXT_BACKUP_ID_OFF, nextBackupId);
+    public void setNextBackupId(long pageAddr, long nextBackupId) {
+        PageUtils.putLong(pageAddr, NEXT_BACKUP_ID_OFF, nextBackupId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastAllocatedIdx Last allocated index.
      */
-    public void setLastAllocatedIndex(long buf, int lastAllocatedIdx) {
-        PageUtils.putInt(buf, LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
+    public void setLastAllocatedIndex(long pageAddr, int lastAllocatedIdx) {
+        PageUtils.putInt(pageAddr, LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public int getLastAllocatedIndex(long buf) {
-        return PageUtils.getInt(buf, LAST_ALLOCATED_INDEX_OFF);
+    public int getLastAllocatedIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, LAST_ALLOCATED_INDEX_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param previousAllocatedIdx Last allocated index.
      */
-    public void setCandidateAllocatedIndex(long buf, int previousAllocatedIdx) {
-        PageUtils.putInt(buf, CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
+    public void setCandidateAllocatedIndex(long pageAddr, int previousAllocatedIdx) {
+        PageUtils.putInt(pageAddr, CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public int getCandidateAllocatedIndex(long buf) {
-        return PageUtils.getInt(buf, CANDIDATE_ALLOCATED_INDEX_OFF);
+    public int getCandidateAllocatedIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, CANDIDATE_ALLOCATED_INDEX_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getNextBackupId(long buf) {
-        return PageUtils.getLong(buf, NEXT_BACKUP_ID_OFF);
+    public long getNextBackupId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_BACKUP_ID_OFF);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
index 531ba1e..aca0725 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
@@ -42,13 +42,13 @@ public class PagePartitionMetaIO extends PageMetaIO {
     );
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId, int pageSize) {
-        super.initNewPage(buf, pageId, pageSize);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setSize(buf, 0);
-        setUpdateCounter(buf, 0);
-        setGlobalRemoveId(buf, 0);
-        setPartitionState(buf, (byte)-1);
+        setSize(pageAddr, 0);
+        setUpdateCounter(pageAddr, 0);
+        setGlobalRemoveId(pageAddr, 0);
+        setPartitionState(pageAddr, (byte)-1);
     }
 
     /**
@@ -59,65 +59,65 @@ public class PagePartitionMetaIO extends PageMetaIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Partition size.
      */
-    public long getSize(long buf) {
-        return PageUtils.getLong(buf, SIZE_OFF);
+    public long getSize(long pageAddr) {
+        return PageUtils.getLong(pageAddr, SIZE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param size Partition size.
      */
-    public void setSize(long buf, long size) {
-        PageUtils.putLong(buf, SIZE_OFF, size);
+    public void setSize(long pageAddr, long size) {
+        PageUtils.putLong(pageAddr, SIZE_OFF, size);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Partition update counter.
      */
-    public long getUpdateCounter(long buf) {
-        return PageUtils.getLong(buf, UPDATE_CNTR_OFF);
+    public long getUpdateCounter(long pageAddr) {
+        return PageUtils.getLong(pageAddr, UPDATE_CNTR_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cntr Partition update counter.
      */
-    public void setUpdateCounter(long buf, long cntr) {
-        PageUtils.putLong(buf, UPDATE_CNTR_OFF, cntr);
+    public void setUpdateCounter(long pageAddr, long cntr) {
+        PageUtils.putLong(pageAddr, UPDATE_CNTR_OFF, cntr);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Global remove ID.
      */
-    public long getGlobalRemoveId(long buf) {
-        return PageUtils.getLong(buf, GLOBAL_RMV_ID_OFF);
+    public long getGlobalRemoveId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, GLOBAL_RMV_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rmvId Global remove ID.
      */
-    public void setGlobalRemoveId(long buf, long rmvId) {
-        PageUtils.putLong(buf, GLOBAL_RMV_ID_OFF, rmvId);
+    public void setGlobalRemoveId(long pageAddr, long rmvId) {
+        PageUtils.putLong(pageAddr, GLOBAL_RMV_ID_OFF, rmvId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public byte getPartitionState(long buf) {
-        return PageUtils.getByte(buf, PARTITION_STATE_OFF);
+    public byte getPartitionState(long pageAddr) {
+        return PageUtils.getByte(pageAddr, PARTITION_STATE_OFF);
     }
 
     /**
-     * @param buf
-     * @param state
+     * @param pageAddr Page address
+     * @param state State.
      */
-    public void setPartitionState(long buf, byte state) {
-        PageUtils.putByte(buf, PARTITION_STATE_OFF, state);
+    public void setPartitionState(long pageAddr, byte state) {
+        PageUtils.putByte(pageAddr, PARTITION_STATE_OFF, state);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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 480c940..ccd8dee 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,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.database.tree.util;
 
 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;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -95,6 +96,7 @@ public abstract class PageHandler<X, R> {
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
      * @param lockLsnr Lock listener.
      * @param h Handler.
@@ -105,6 +107,7 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static <X, R> R writePage(
+        PageMemory pageMem,
         Page page,
         PageLockListener lockLsnr,
         PageHandler<X, R> h,
@@ -112,10 +115,11 @@ public abstract class PageHandler<X, R> {
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        return writePage(page, lockLsnr, h, null, null, arg, intArg, lockFailed);
+        return writePage(pageMem, page, lockLsnr, h, null, null, arg, intArg, lockFailed);
     }
 
     /**
+     * @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.
@@ -123,12 +127,13 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static void initPage(
+        PageMemory pageMem,
         Page page,
         PageLockListener lockLsnr,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
-        Boolean res = writePage(page, lockLsnr, NOOP, init, wal, null, 0, FALSE);
+        Boolean res = writePage(pageMem, page, lockLsnr, NOOP, init, wal, null, 0, FALSE);
 
         assert res == TRUE : res; // It must be newly allocated page, can't be recycled.
     }
@@ -200,6 +205,7 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static <X, R> R writePage(
+        PageMemory pageMem,
         Page page,
         PageLockListener lockLsnr,
         PageHandler<X, R> h,
@@ -220,7 +226,7 @@ public abstract class PageHandler<X, R> {
 
         try {
             if (init != null) // It is a new page and we have to initialize it.
-                doInitPage(page, pageAddr, init, wal);
+                doInitPage(pageMem, page, pageAddr, init, wal);
             else
                 init = PageIO.getPageIO(pageAddr);
 
@@ -239,6 +245,7 @@ public abstract class PageHandler<X, R> {
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
      * @param pageAddr Page address.
      * @param init Initial IO.
@@ -246,6 +253,7 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     private static void doInitPage(
+        PageMemory pageMem,
         Page page,
         long pageAddr,
         PageIO init,
@@ -255,7 +263,7 @@ public abstract class PageHandler<X, R> {
 
         long pageId = page.id();
 
-        init.initNewPage(pageAddr, pageId, page.size());
+        init.initNewPage(pageAddr, pageId, pageMem.pageSize());
 
         // Here we should never write full page, because it is known to be new.
         page.fullPageWalRecordPolicy(FALSE);


[03/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 94f22a0c08971e3bb8ff13c2cc8e32f748f1c3e4
Parents: 8fcac6f
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 11:06:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 11:19:27 2016 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapManagerImpl.java    |  99 ++++++-------
 .../cache/database/freelist/FreeListImpl.java   |   6 +-
 .../cache/database/tree/BPlusTree.java          | 147 +++++++++----------
 .../cache/database/tree/io/DataPageIO.java      | 129 ++++++++--------
 .../database/tree/reuse/ReuseListImpl.java      |   2 +-
 .../processors/query/h2/database/H2Tree.java    |   2 +-
 6 files changed, 189 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/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 02a068d..4b1d464 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
@@ -31,6 +31,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRowAdapter;
@@ -1085,7 +1086,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx, KeySearchRow row)
+        @Override protected int compare(BPlusIO<KeySearchRow> io, long buf, int idx, KeySearchRow row)
             throws IgniteCheckedException {
             int hash = ((RowLinkIO)io).getHash(buf, idx);
 
@@ -1100,7 +1101,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected DataRow getRow(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx)
+        @Override protected DataRow getRow(BPlusIO<KeySearchRow> io, long buf, int idx)
             throws IgniteCheckedException {
             int hash = ((RowLinkIO)io).getHash(buf, idx);
             long link = ((RowLinkIO)io).getLink(buf, idx);
@@ -1169,9 +1170,9 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
      * @param link Link.
      * @param hash Hash.
      */
-    private static void store0(ByteBuffer buf, int off, long link, int hash) {
-        buf.putLong(off, link);
-        buf.putInt(off + 8, hash);
+    private static void store0(long buf, int off, long link, int hash) {
+        PageUtils.putLong(buf, off, link);
+        PageUtils.putInt(buf, off + 8, hash);
     }
 
     /**
@@ -1183,14 +1184,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param idx Index.
          * @return Row link.
          */
-        public long getLink(ByteBuffer buf, int idx);
+        public long getLink(long buf, int idx);
 
         /**
          * @param buf Buffer.
          * @param idx Index.
          * @return Key hash code.
          */
-        public int getHash(ByteBuffer buf, int idx);
+        public int getHash(long buf, int idx);
     }
 
     /**
@@ -1210,14 +1211,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(long buf, int off, KeySearchRow row) {
             assert row.link() != 0;
 
             store0(buf, off, row.link(), row.hash);
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
+        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, long buf, int idx) {
             int hash = getHash(buf, idx);
             long link = getLink(buf, idx);
 
@@ -1225,7 +1226,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, long src,
             int srcIdx) {
             int hash = ((RowLinkIO)srcIo).getHash(src, srcIdx);
             long link = ((RowLinkIO)srcIo).getLink(src, srcIdx);
@@ -1234,15 +1235,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
+        @Override public long getLink(long buf, int idx) {
             assert idx < getCount(buf) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long buf, int idx) {
+            return PageUtils.getInt(buf, offset(idx) + 8);
         }
     }
 
@@ -1263,20 +1264,20 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(long buf, int off, KeySearchRow row) {
             assert row.link() != 0;
 
             store0(buf, off, row.link(), row.hash);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, long src,
             int srcIdx) {
             store0(dst, offset(dstIdx), getLink(src, srcIdx), getHash(src, srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
+        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, long buf, int idx) {
             int hash = getHash(buf, idx);
             long link = getLink(buf, idx);
 
@@ -1284,15 +1285,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
+        @Override public long getLink(long buf, int idx) {
             assert idx < getCount(buf) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long buf, int idx) {
+            return PageUtils.getInt(buf, offset(idx) + 8);
         }
     }
 
@@ -1387,7 +1388,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<PendingRow> io, ByteBuffer buf, int idx, PendingRow row)
+        @Override protected int compare(BPlusIO<PendingRow> io, long buf, int idx, PendingRow row)
             throws IgniteCheckedException {
             long expireTime = ((PendingRowIO)io).getExpireTime(buf, idx);
 
@@ -1405,7 +1406,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, ByteBuffer buf, int idx)
+        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, long buf, int idx)
             throws IgniteCheckedException {
             return io.getLookupRow(this, buf, idx);
         }
@@ -1420,14 +1421,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param idx Index.
          * @return Expire time.
          */
-        long getExpireTime(ByteBuffer buf, int idx);
+        long getExpireTime(long buf, int idx);
 
         /**
          * @param buf Buffer.
          * @param idx Index.
          * @return Link.
          */
-        long getLink(ByteBuffer buf, int idx);
+        long getLink(long buf, int idx);
     }
 
     /**
@@ -1447,43 +1448,43 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, PendingRow row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, PendingRow row) throws IgniteCheckedException {
             assert row.link != 0;
             assert row.expireTime != 0;
 
-            buf.putLong(off, row.expireTime);
-            buf.putLong(off + 8, row.link);
+            PageUtils.putLong(buf, off, row.expireTime);
+            PageUtils.putLong(buf, off + 8, row.link);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void store(long dst,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long src,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
             long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
             long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dst, dstOff, expireTime);
+            PageUtils.putLong(dst, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long buf, int idx)
             throws IgniteCheckedException {
             return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx) + 8);
         }
     }
 
@@ -1504,43 +1505,43 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, PendingRow row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, PendingRow row) throws IgniteCheckedException {
             assert row.link != 0;
             assert row.expireTime != 0;
 
-            buf.putLong(off, row.expireTime);
-            buf.putLong(off + 8, row.link);
+            PageUtils.putLong(buf, off, row.expireTime);
+            PageUtils.putLong(buf, off + 8, row.link);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void store(long dst,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long src,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
             long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
             long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dst, dstOff, expireTime);
+            PageUtils.putLong(dst, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long buf, int idx)
             throws IgniteCheckedException {
             return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx) + 8);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/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 af20d15..1d2524d 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
@@ -109,7 +109,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRow(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int rowSize
@@ -149,7 +149,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRowFragment(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int written,
@@ -179,7 +179,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
     /** */
     private final PageHandler<Void, Long> rmvRow = new PageHandler<Void, Long>() {
-        @Override public Long run(Page page, PageIO iox, ByteBuffer buf, Void arg, int itemId)
+        @Override public Long run(Page page, PageIO iox, long buf, Void arg, int itemId)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/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 6ea1bff..b8a1004 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
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.database.tree;
 
 import java.io.Externalizable;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -127,7 +126,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 return null;
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long buf = readLock(page); // No correctness guaranties.
 
                 try {
                     BPlusIO io = io(buf);
@@ -174,7 +173,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 return "<Zero>";
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long buf = readLock(page); // No correctness guaranties.
 
                 try {
                     BPlusIO<L> io = io(buf);
@@ -193,7 +192,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Get> askNeighbor = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Get g, int isBack) {
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Get g, int isBack) {
             assert !io.isLeaf(); // Inner page.
 
             boolean back = isBack == TRUE.ordinal();
@@ -220,7 +219,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Get> search = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Get g, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Get g, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
             if (io.getForward(buf) != g.fwdId)
@@ -296,7 +295,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Put> replace = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
             if (io.getForward(buf) != p.fwdId)
@@ -338,7 +337,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Put> insert = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             assert p.btmLvl == lvl : "we must always insert at the bottom level: " + p.btmLvl + " " + lvl;
 
@@ -378,7 +377,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> removeFromLeaf = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page leaf, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page leaf, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
@@ -450,7 +449,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockBackAndRemoveFromLeaf = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page back, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page back, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(buf) != r.pageId)
@@ -469,7 +468,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockBackAndTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page back, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page back, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(buf) != r.pageId)
@@ -487,7 +486,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockTailForward = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page page, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             r.addTail(page, buf, io, lvl, Tail.FORWARD);
 
@@ -497,7 +496,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
@@ -521,7 +520,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Void, Bool> cutRoot = new PageHandler<Void, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Void ignore, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long buf, Void ignore, int lvl)
             throws IgniteCheckedException {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
@@ -539,7 +538,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Long, Bool> addRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Long rootPageId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long buf, Long rootPageId, int lvl)
             throws IgniteCheckedException {
             assert rootPageId != null;
 
@@ -559,7 +558,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Long, Bool> initRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Long rootId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long buf, Long rootId, int lvl)
             throws IgniteCheckedException {
             assert rootId != null;
 
@@ -657,7 +656,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Root level.
      */
     private int getRootLevel(Page meta) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long buf = readLock(meta); // Meta can't be removed.
 
         try {
             return BPlusMetaIO.VERSIONS.forPage(buf).getRootLevel(buf);
@@ -673,7 +672,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Page ID.
      */
     private long getFirstPageId(Page meta, int lvl) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long buf = readLock(meta); // Meta can't be removed.
 
         try {
             BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
@@ -705,7 +704,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         try (Page first = page(firstPageId)) {
-            ByteBuffer buf = readLock(first); // We always merge pages backwards, the first page is never removed.
+            long buf = readLock(first); // We always merge pages backwards, the first page is never removed.
 
             try {
                 cursor.init(buf, io(buf), 0);
@@ -922,7 +921,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(buf);
@@ -983,7 +982,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private L getGreatestRowInSubTree(long pageId) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(buf);
@@ -1038,7 +1037,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         long leftmostChildId;
 
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(buf);
@@ -1068,7 +1067,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private void validateDownPages(Page meta, long pageId, long fwdId, final int lvl) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 long realPageId = BPlusIO.getPageId(buf);
@@ -1103,7 +1102,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                     if (fwdId != 0) {
                         // For the rightmost child ask neighbor.
                         try (Page fwd = page(fwdId)) {
-                            ByteBuffer fwdBuf = readLock(fwd); // No correctness guaranties.
+                            long fwdBuf = readLock(fwd); // No correctness guaranties.
 
                             try {
                                 if (io(fwdBuf) != io)
@@ -1135,7 +1134,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return String.
      * @throws IgniteCheckedException If failed.
      */
-    private String printPage(BPlusIO<L> io, ByteBuffer buf, boolean keys) throws IgniteCheckedException {
+    private String printPage(BPlusIO<L> io, long buf, boolean keys) throws IgniteCheckedException {
         StringBuilder b = new StringBuilder();
 
         b.append(formatPageId(PageIO.getPageId(buf)));
@@ -1170,7 +1169,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Keys as String.
      * @throws IgniteCheckedException If failed.
      */
-    private String printPageKeys(BPlusIO<L> io, ByteBuffer buf) throws IgniteCheckedException {
+    private String printPageKeys(BPlusIO<L> io, long buf) throws IgniteCheckedException {
         int cnt = io.getCount(buf);
 
         StringBuilder b = new StringBuilder();
@@ -1488,7 +1487,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
         while (pageId != 0) {
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long buf = readLock(page); // No correctness guaranties.
 
                 try {
                     if (io == null) {
@@ -1598,7 +1597,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         long pagesCnt = 0;
 
         try (Page meta = page(metaPageId)) {
-            ByteBuffer metaBuf = writeLock(meta); // No checks, we must be out of use.
+            long metaBuf = writeLock(meta); // No checks, we must be out of use.
 
             try {
                 for (long pageId : getFirstPageIds(metaBuf)) {
@@ -1606,7 +1605,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                     do {
                         try (Page page = page(pageId)) {
-                            ByteBuffer buf = writeLock(page); // No checks, we must be out of use.
+                            long buf = writeLock(page); // No checks, we must be out of use.
 
                             try {
                                 BPlusIO<L> io = io(buf);
@@ -1658,7 +1657,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param metaBuf Meta page buffer.
      * @return First page IDs.
      */
-    protected Iterable<Long> getFirstPageIds(ByteBuffer metaBuf) {
+    protected Iterable<Long> getFirstPageIds(long metaBuf) {
         List<Long> result = new ArrayList<>();
 
         BPlusMetaIO mio = BPlusMetaIO.VERSIONS.forPage(metaBuf);
@@ -1677,7 +1676,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Recycled 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 {
         // Rotate page ID to avoid concurrency issues with reused pages.
         pageId = PageIdUtils.rotatePageId(pageId);
 
@@ -1704,10 +1703,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     private boolean splitPage(
         BPlusIO io,
         Page page,
-        ByteBuffer buf,
+        long buf,
         long fwdId,
         Page fwd,
-        ByteBuffer fwdBuf,
+        long fwdBuf,
         int idx
     ) throws IgniteCheckedException {
         int cnt = io.getCount(buf);
@@ -1740,7 +1739,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param page Page.
      * @param buf Buffer.
      */
-    private void writeUnlockAndClose(Page page, ByteBuffer buf) {
+    private void writeUnlockAndClose(Page page, long buf) {
         try {
             writeUnlock(page, buf, true);
         }
@@ -1860,7 +1859,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param back Backward page.
      * @return Page ID.
      */
-    private long doAskNeighbor(BPlusIO<L> io, ByteBuffer buf, boolean back) {
+    private long doAskNeighbor(BPlusIO<L> io, long buf, boolean back) {
         long res;
 
         if (back) {
@@ -1938,9 +1937,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             int rootLvl;
             long rootId;
 
-            ByteBuffer buf = readLock(meta); // Meta can't be removed.
+            long buf = readLock(meta); // Meta can't be removed.
 
-            assert buf != null : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert buf != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
@@ -1975,7 +1974,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return {@code true} If we need to stop.
          * @throws IgniteCheckedException If failed.
          */
-        boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        boolean found(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -1989,7 +1988,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return {@code true} If we need to stop.
          * @throws IgniteCheckedException If failed.
          */
-        boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -2027,7 +2026,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             // Check if we are on an inner page and can't get row from it.
             if (lvl != 0 && !canGetRowFromInner)
                 return false;
@@ -2060,12 +2059,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             throw new IllegalStateException(); // Must never be called because we always have a shift.
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             if (lvl != 0)
                 return false;
 
@@ -2093,7 +2092,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         private Page tail;
 
         /** */
-        private ByteBuffer tailBuf;
+        private long tailBuf;
 
         /**
          * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level.
@@ -2117,7 +2116,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean found(BPlusIO<L> io, long buf, int idx, int lvl) {
             if (lvl == 0) // Leaf: need to stop.
                 return true;
 
@@ -2131,7 +2130,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) {
             assert btmLvl >= 0 : btmLvl;
             assert lvl >= btmLvl : lvl;
 
@@ -2142,8 +2141,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param tail Tail page.
          * @param tailBuf Tail buffer.
          */
-        private void tail(Page tail, ByteBuffer tailBuf) {
-            assert (tail == null) == (tailBuf == null);
+        private void tail(Page tail, long tailBuf) {
+            assert (tail == null) == (tailBuf == 0L);
 
             if (this.tail != null)
                 writeUnlockAndClose(this.tail, this.tailBuf);
@@ -2164,7 +2163,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             row = null;
             rightId = 0;
 
-            tail(null, null);
+            tail(null, 0L);
         }
 
         /**
@@ -2183,7 +2182,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insert(Page page, BPlusIO<L> io, ByteBuffer buf, int idx, int lvl)
+        private L insert(Page page, BPlusIO<L> io, long buf, int idx, int lvl)
             throws IgniteCheckedException {
             int maxCnt = io.getMaxCount(pageMem.pageSize(), buf);
             int cnt = io.getCount(buf);
@@ -2203,7 +2202,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param idx Index.
          * @throws IgniteCheckedException If failed.
          */
-        private void insertSimple(Page page, BPlusIO<L> io, ByteBuffer buf, int idx)
+        private void insertSimple(Page page, BPlusIO<L> io, long buf, int idx)
             throws IgniteCheckedException {
             io.insert(buf, idx, row, null, rightId);
 
@@ -2220,7 +2219,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insertWithSplit(Page page, BPlusIO<L> io, final ByteBuffer buf, int idx, int lvl)
+        private L insertWithSplit(Page page, BPlusIO<L> io, final long buf, int idx, int lvl)
             throws IgniteCheckedException {
             long fwdId = allocatePage(bag);
 
@@ -2228,9 +2227,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 // Need to check this before the actual split, because after the split we will have new forward page here.
                 boolean hadFwd = io.getForward(buf) != 0;
 
-                ByteBuffer fwdBuf = writeLock(fwd); // Initial write, no need to check for concurrent modification.
+                long fwdBuf = writeLock(fwd); // Initial write, no need to check for concurrent modification.
 
-                assert fwdBuf != null;
+                assert fwdBuf != 0L;
 
                 try {
                     // Never write full forward page, because it is known to be new.
@@ -2275,9 +2274,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                             if (io.isLeaf())
                                 io = latestInnerIO();
 
-                            ByteBuffer newRootBuf = writeLock(newRoot); // Initial write.
+                            long newRootBuf = writeLock(newRoot); // Initial write.
 
-                            assert newRootBuf != null;
+                            assert newRootBuf != 0L;
 
                             try {
                                 // Never write full new root page, because it is known to be new.
@@ -2405,7 +2404,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) {
             if (lvl == 0) {
                 assert tail == null;
 
@@ -2772,7 +2771,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, ByteBuffer buf, int cnt, int idx)
+        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, long buf, int cnt, int idx)
             throws IgniteCheckedException {
             assert idx >= 0 && idx < cnt: idx;
             assert io.isLeaf(): "inner";
@@ -2794,7 +2793,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void doRemove(Page page, BPlusIO<L> io, ByteBuffer buf, int cnt, int idx)
+        private void doRemove(Page page, BPlusIO<L> io, long buf, int cnt, int idx)
             throws IgniteCheckedException {
             assert cnt > 0 : cnt;
             assert idx >= 0 && idx < cnt : idx + " " + cnt;
@@ -2966,7 +2965,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param release Release write lock and release page.
          * @throws IgniteCheckedException If failed.
          */
-        private void freePage(Page page, ByteBuffer buf, boolean release)
+        private void freePage(Page page, long buf, boolean release)
             throws IgniteCheckedException {
             long pageId = page.id();
 
@@ -3165,7 +3164,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param type Type.
          * @return Added tail.
          */
-        private Tail<L> addTail(Page page, ByteBuffer buf, BPlusIO<L> io, int lvl, byte type) {
+        private Tail<L> addTail(Page page, long buf, BPlusIO<L> io, int lvl, byte type) {
             final Tail<L> t = new Tail<>(page, buf, io, type, lvl);
 
             if (tail == null)
@@ -3273,7 +3272,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         private final Page page;
 
         /** */
-        private final ByteBuffer buf;
+        private final long buf;
 
         /** */
         private final BPlusIO<L> io;
@@ -3300,7 +3299,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param type Type.
          * @param lvl Level.
          */
-        private Tail(Page page, ByteBuffer buf, BPlusIO<L> io, byte type, int lvl) {
+        private Tail(Page page, long buf, BPlusIO<L> io, byte type, int lvl) {
             assert type == BACK || type == EXACT || type == FORWARD : type;
             assert lvl >= 0 && lvl <= Byte.MAX_VALUE : lvl;
             assert page != null;
@@ -3353,7 +3352,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param shift Shift if equal.
      * @return Insertion point as in {@link Arrays#binarySearch(Object[], Object, Comparator)}.
      */
-    private int findInsertionPoint(BPlusIO<L> io, ByteBuffer buf, int low, int cnt, L row, int shift)
+    private int findInsertionPoint(BPlusIO<L> io, long buf, int low, int cnt, L row, int shift)
         throws IgniteCheckedException {
         assert row != null;
 
@@ -3383,8 +3382,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param buf Buffer.
      * @return IO.
      */
-    private BPlusIO<L> io(ByteBuffer buf) {
-        assert buf != null;
+    private BPlusIO<L> io(long buf) {
+        assert buf != 0;
 
         int type = PageIO.getType(buf);
         int ver = PageIO.getVersion(buf);
@@ -3429,7 +3428,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Comparison result as in {@link Comparator#compare(Object, Object)}.
      * @throws IgniteCheckedException If failed.
      */
-    protected abstract int compare(BPlusIO<L> io, ByteBuffer buf, int idx, L row) throws IgniteCheckedException;
+    protected abstract int compare(BPlusIO<L> io, long buf, int idx, L row) throws IgniteCheckedException;
 
     /**
      * Get the full detached row. Can be called on inner page only if {@link #canGetRowFromInner} is {@code true}.
@@ -3440,7 +3439,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Full detached data row.
      * @throws IgniteCheckedException If failed.
      */
-    protected abstract T getRow(BPlusIO<L> io, ByteBuffer buf, int idx) throws IgniteCheckedException;
+    protected abstract T getRow(BPlusIO<L> io, long buf, int idx) throws IgniteCheckedException;
 
     /**
      * Forward cursor.
@@ -3480,7 +3479,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param startIdx Start index.
          * @throws IgniteCheckedException If failed.
          */
-        private void init(ByteBuffer buf, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
+        private void init(long buf, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
             nextPageId = 0;
             row = -1;
 
@@ -3509,7 +3508,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Adjusted to lower bound start index.
          * @throws IgniteCheckedException If failed.
          */
-        private int findLowerBound(ByteBuffer buf, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
+        private int findLowerBound(long buf, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
             // Compare with the first row on the page.
             int cmp = compare(io, buf, 0, lowerBound);
 
@@ -3532,7 +3531,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Corrected number of rows with respect to upper bound.
          * @throws IgniteCheckedException If failed.
          */
-        private int findUpperBound(ByteBuffer buf, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
+        private int findUpperBound(long buf, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
             // Compare with the last row on the page.
             int cmp = compare(io, buf, cnt - 1, upperBound);
 
@@ -3558,7 +3557,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @throws IgniteCheckedException If failed.
          */
         @SuppressWarnings("unchecked")
-        private boolean fillFromBuffer(ByteBuffer buf, BPlusIO<L> io, int startIdx, int cnt)
+        private boolean fillFromBuffer(long buf, BPlusIO<L> io, int startIdx, int cnt)
             throws IgniteCheckedException {
             assert io.isLeaf();
             assert cnt != 0: cnt; // We can not see empty pages (empty tree handled in init).
@@ -3665,10 +3664,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 }
 
                 try (Page next = page(nextPageId)) {
-                    ByteBuffer buf = readLock(next); // Doing explicit null check.
+                    long buf = readLock(next); // Doing explicit null check.
 
                     // If concurrent merge occurred we have to reinitialize cursor from the last returned row.
-                    if (buf == null)
+                    if (buf == 0L)
                         break;
 
                     try {
@@ -3715,7 +3714,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     private abstract class GetPageHandler<G extends Get> extends PageHandler<G, Result> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override public final Result run(Page page, PageIO iox, ByteBuffer buf, G g, int lvl)
+        @Override public final Result run(Page page, PageIO iox, long buf, G g, int lvl)
             throws IgniteCheckedException {
             assert PageIO.getPageId(buf) == page.id();
 
@@ -3739,7 +3738,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        protected abstract Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, G g, int lvl)
+        protected abstract Result run0(Page page, long buf, BPlusIO<L> io, G g, int lvl)
             throws IgniteCheckedException;
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index e1bab5f..ca8ea39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -88,7 +88,7 @@ public class DataPageIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
+    @Override public void initNewPage(long buf, long pageId) {
         super.initNewPage(buf, pageId);
 
         setEmptyPage(buf);
@@ -98,7 +98,7 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      */
-    private void setEmptyPage(ByteBuffer buf) {
+    private void setEmptyPage(long buf) {
         setDirectCount(buf, 0);
         setIndirectCount(buf, 0);
         setFirstEntryOffset(buf, buf.capacity());
@@ -127,8 +127,8 @@ public class DataPageIO extends PageIO {
      * @param show What elements of data page entry to show in the result.
      * @return Data page entry size.
      */
-    private int getPageEntrySize(ByteBuffer buf, int dataOff, int show) {
-        int payloadLen = buf.getShort(dataOff) & 0xFFFF;
+    private int getPageEntrySize(long buf, int dataOff, int show) {
+        int payloadLen = PageUtils.getShort(buf, dataOff) & 0xFFFF;
 
         if ((payloadLen & FRAGMENTED_FLAG) != 0)
             payloadLen &= ~FRAGMENTED_FLAG; // We are fragmented and have a link.
@@ -164,28 +164,28 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param dataOff Entry data offset.
      */
-    private void setFirstEntryOffset(ByteBuffer buf, int dataOff) {
+    private void setFirstEntryOffset(long buf, int dataOff) {
         assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= buf.capacity(): dataOff;
 
-        buf.putShort(FIRST_ENTRY_OFF, (short)dataOff);
+        PageUtils.putShort(buf, FIRST_ENTRY_OFF, (short)dataOff);
     }
 
     /**
      * @param buf Buffer.
      * @return Entry data offset.
      */
-    private int getFirstEntryOffset(ByteBuffer buf) {
-        return buf.getShort(FIRST_ENTRY_OFF) & 0xFFFF;
+    private int getFirstEntryOffset(long buf) {
+        return PageUtils.getShort(buf, FIRST_ENTRY_OFF) & 0xFFFF;
     }
 
     /**
      * @param buf Buffer.
      * @param freeSpace Free space.
      */
-    private void setRealFreeSpace(ByteBuffer buf, int freeSpace) {
+    private void setRealFreeSpace(long buf, int freeSpace) {
         assert freeSpace == actualFreeSpace(buf): freeSpace + " != " + actualFreeSpace(buf);
 
-        buf.putShort(FREE_SPACE_OFF, (short)freeSpace);
+        PageUtils.putShort(buf, FREE_SPACE_OFF, (short)freeSpace);
     }
 
     /**
@@ -195,7 +195,7 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @return Free space.
      */
-    public int getFreeSpace(ByteBuffer buf) {
+    public int getFreeSpace(long buf) {
         if (getFreeItemSlots(buf) == 0)
             return 0;
 
@@ -218,23 +218,23 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * Equivalent for {@link #actualFreeSpace(ByteBuffer)} but reads saved value.
+     * Equivalent for {@link #actualFreeSpace(long)} but reads saved value.
      *
      * @param buf Buffer.
      * @return Free space.
      */
-    private int getRealFreeSpace(ByteBuffer buf) {
-        return buf.getShort(FREE_SPACE_OFF);
+    private int getRealFreeSpace(long buf) {
+        return PageUtils.getShort(buf, FREE_SPACE_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param cnt Direct count.
      */
-    private void setDirectCount(ByteBuffer buf, int cnt) {
+    private void setDirectCount(long buf, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(DIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(buf, DIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
@@ -249,10 +249,10 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param cnt Indirect count.
      */
-    private void setIndirectCount(ByteBuffer buf, int cnt) {
+    private void setIndirectCount(long buf, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(INDIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(buf, INDIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
@@ -275,15 +275,15 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @return Indirect count.
      */
-    private int getIndirectCount(ByteBuffer buf) {
-        return buf.get(INDIRECT_CNT_OFF) & 0xFF;
+    private int getIndirectCount(long buf) {
+        return PageUtils.getByte(buf, INDIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
      * @param buf Buffer.
      * @return Number of free entry slots.
      */
-    private int getFreeItemSlots(ByteBuffer buf) {
+    private int getFreeItemSlots(long buf) {
         return 0xFF - getDirectCount(buf);
     }
 
@@ -294,7 +294,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Found index of indirect item.
      */
-    private int findIndirectItemIndex(ByteBuffer buf, int itemId, int directCnt, int indirectCnt) {
+    private int findIndirectItemIndex(long buf, int itemId, int directCnt, int indirectCnt) {
         int low = directCnt;
         int high = directCnt + indirectCnt - 1;
 
@@ -318,7 +318,7 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @return String representation.
      */
-    public String printPageLayout(ByteBuffer buf) {
+    public String printPageLayout(long buf) {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
         int free = getRealFreeSpace(buf);
@@ -395,7 +395,7 @@ public class DataPageIO extends PageIO {
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @return Data entry offset in bytes.
      */
-    private int getDataOffset(ByteBuffer buf, int itemId) {
+    private int getDataOffset(long buf, int itemId) {
         assert checkIndex(itemId): itemId;
 
         int directCnt = getDirectCount(buf);
@@ -427,10 +427,10 @@ public class DataPageIO extends PageIO {
      * @param dataOff Points to the entry start.
      * @return Link to the next entry fragment or 0 if no fragments left or if entry is not fragmented.
      */
-    private long getNextFragmentLink(ByteBuffer buf, int dataOff) {
+    private long getNextFragmentLink(long buf, int dataOff) {
         assert isFragmented(buf, dataOff);
 
-        return buf.getLong(dataOff + PAYLOAD_LEN_SIZE);
+        return PageUtils.getLong(buf, dataOff + PAYLOAD_LEN_SIZE);
     }
 
     /**
@@ -438,8 +438,8 @@ public class DataPageIO extends PageIO {
      * @param dataOff Data offset.
      * @return {@code true} If the data row is fragmented across multiple pages.
      */
-    private boolean isFragmented(ByteBuffer buf, int dataOff) {
-        return (buf.getShort(dataOff) & FRAGMENTED_FLAG) != 0;
+    private boolean isFragmented(long buf, int dataOff) {
+        return (PageUtils.getShort(buf, dataOff) & FRAGMENTED_FLAG) != 0;
     }
 
     /**
@@ -449,7 +449,7 @@ public class DataPageIO extends PageIO {
      * @param itemId Item to position on.
      * @return Link to the next fragment or {@code 0} if it is the last fragment or the data row is not fragmented.
      */
-    public long setPositionAndLimitOnPayload(final ByteBuffer buf, final int itemId) {
+    public long setPositionAndLimitOnPayload(final long buf, final int itemId) {
         int dataOff = getDataOffset(buf, itemId);
 
         boolean fragmented = isFragmented(buf, dataOff);
@@ -468,8 +468,8 @@ public class DataPageIO extends PageIO {
      * @param idx Item index.
      * @return Item.
      */
-    private short getItem(ByteBuffer buf, int idx) {
-        return buf.getShort(itemOffset(idx));
+    private short getItem(long buf, int idx) {
+        return PageUtils.getShort(buf, itemOffset(idx));
     }
 
     /**
@@ -477,8 +477,8 @@ public class DataPageIO extends PageIO {
      * @param idx Item index.
      * @param item Item.
      */
-    private void setItem(ByteBuffer buf, int idx, short item) {
-        buf.putShort(itemOffset(idx), item);
+    private void setItem(long buf, int idx, short item) {
+        PageUtils.putShort(buf, itemOffset(idx), item);
     }
 
     /**
@@ -546,7 +546,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return {@code true} If the last direct item already had corresponding indirect item.
      */
-    private boolean moveLastItem(ByteBuffer buf, int freeDirectIdx, int directCnt, int indirectCnt) {
+    private boolean moveLastItem(long buf, int freeDirectIdx, int directCnt, int indirectCnt) {
         int lastIndirectId = findIndirectIndexForLastDirect(buf, directCnt, indirectCnt);
 
         int lastItemId = directCnt - 1;
@@ -577,7 +577,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Index of indirect item for the last direct item.
      */
-    private int findIndirectIndexForLastDirect(ByteBuffer buf, int directCnt, int indirectCnt) {
+    private int findIndirectIndexForLastDirect(long buf, int directCnt, int indirectCnt) {
         int lastDirectId = directCnt - 1;
 
         for (int i = directCnt, end = directCnt + indirectCnt; i < end; i++) {
@@ -596,7 +596,7 @@ public class DataPageIO extends PageIO {
      * @return Next link for fragmented entries or {@code 0} if none.
      * @throws IgniteCheckedException If failed.
      */
-    public long removeRow(ByteBuffer buf, int itemId) throws IgniteCheckedException {
+    public long removeRow(long buf, int itemId) throws IgniteCheckedException {
         assert checkIndex(itemId) : itemId;
 
         final int dataOff = getDataOffset(buf, itemId);
@@ -675,7 +675,7 @@ public class DataPageIO extends PageIO {
      * @param cnt Count.
      * @param step Step.
      */
-    private void moveItems(ByteBuffer buf, int idx, int cnt, int step) {
+    private void moveItems(long buf, int idx, int cnt, int step) {
         assert cnt >= 0: cnt;
 
         if (cnt != 0)
@@ -702,7 +702,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
+        long buf,
         CacheDataRow row,
         int rowSize
     ) throws IgniteCheckedException {
@@ -729,7 +729,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
+        long buf,
         byte[] payload
     ) throws IgniteCheckedException {
         assert payload.length <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
@@ -755,7 +755,7 @@ public class DataPageIO extends PageIO {
      * @return First entry offset after compaction.
      */
     private int compactIfNeed(
-        final ByteBuffer buf,
+        final long buf,
         final int entryFullSize,
         final int directCnt,
         final int indirectCnt,
@@ -780,7 +780,7 @@ public class DataPageIO extends PageIO {
      * @param dataOff Data offset.
      * @return Item ID.
      */
-    private int addItem(final ByteBuffer buf, final int fullEntrySize, final int directCnt,
+    private int addItem(final long buf, final int fullEntrySize, final int directCnt,
         final int indirectCnt, final int dataOff) {
         setFirstEntryOffset(buf, dataOff);
 
@@ -803,7 +803,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Offset in the buffer where the entry must be written.
      */
-    private int getDataOffsetForWrite(ByteBuffer buf, int fullEntrySize, int directCnt, int indirectCnt) {
+    private int getDataOffsetForWrite(long buf, int fullEntrySize, int directCnt, int indirectCnt) {
         int dataOff = getFirstEntryOffset(buf);
 
         // Compact if we do not have enough space for entry.
@@ -826,7 +826,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public int addRowFragment(
-        ByteBuffer buf,
+        long buf,
         CacheDataRow row,
         int written,
         int rowSize
@@ -843,7 +843,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void addRowFragment(
-        ByteBuffer buf,
+        long buf,
         byte[] payload,
         long lastLink
     ) throws IgniteCheckedException {
@@ -863,7 +863,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     private int addRowFragment(
-        ByteBuffer buf,
+        long buf,
         int written,
         int rowSize,
         long lastLink,
@@ -881,23 +881,16 @@ public class DataPageIO extends PageIO {
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
         int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
 
-        try {
-            buf.position(dataOff);
-
-            buf.putShort((short) (payloadSize | FRAGMENTED_FLAG));
-            buf.putLong(lastLink);
+        PageUtils.putShort(buf, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
+        PageUtils.putLong(buf, dataOff + 2, lastLink);
 
-            if (payload == null) {
-                int rowOff = rowSize - written - payloadSize;
+        if (payload == null) {
+            int rowOff = rowSize - written - payloadSize;
 
-                writeFragmentData(row, buf, rowOff, payloadSize);
-            }
-            else
-                buf.put(payload);
-        }
-        finally {
-            buf.position(0);
+            writeFragmentData(row, buf + dataOff + 10, rowOff, payloadSize);
         }
+        else
+            PageUtils.putBytes(buf, dataOff + 10, payload);
 
         int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
 
@@ -912,7 +905,7 @@ public class DataPageIO extends PageIO {
      * @param buf Page buffer.
      * @param itemId Item ID.
      */
-    private void setLink(CacheDataRow row, ByteBuffer buf, int itemId) {
+    private void setLink(CacheDataRow row, long buf, int itemId) {
         row.link(PageIdUtils.link(getPageId(buf), itemId));
     }
 
@@ -927,7 +920,7 @@ public class DataPageIO extends PageIO {
      */
     private void writeFragmentData(
         final CacheDataRow row,
-        final ByteBuffer buf,
+        final long buf,
         final int rowOff,
         final int payloadSize
     ) throws IgniteCheckedException {
@@ -953,7 +946,7 @@ public class DataPageIO extends PageIO {
      */
     private int writeFragment(
         final CacheDataRow row,
-        final ByteBuffer buf,
+        final long buf,
         final int rowOff,
         final int payloadSize,
         final EntryPart type,
@@ -1089,7 +1082,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Item ID (insertion index).
      */
-    private int insertItem(ByteBuffer buf, int dataOff, int directCnt, int indirectCnt) {
+    private int insertItem(long buf, int dataOff, int directCnt, int indirectCnt) {
         if (indirectCnt > 0) {
             // If the first indirect item is on correct place to become the last direct item, do the transition
             // and insert the new item into the free slot which was referenced by this first indirect item.
@@ -1124,7 +1117,7 @@ public class DataPageIO extends PageIO {
      * @param directCnt Direct items count.
      * @return New first entry offset.
      */
-    private int compactDataEntries(ByteBuffer buf, int directCnt) {
+    private int compactDataEntries(long buf, int directCnt) {
         assert checkCount(directCnt): directCnt;
 
         int[] offs = new int[directCnt];
@@ -1173,7 +1166,7 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer to scan.
      * @return Actual free space in the buffer.
      */
-    private int actualFreeSpace(ByteBuffer buf) {
+    private int actualFreeSpace(long buf) {
         int directCnt = getDirectCount(buf);
 
         int entriesSize = 0;
@@ -1195,7 +1188,7 @@ public class DataPageIO extends PageIO {
      * @param cnt Count.
      * @param step Step.
      */
-    private void moveBytes(ByteBuffer buf, int off, int cnt, int step) {
+    private void moveBytes(long buf, int off, int cnt, int step) {
         assert step != 0: step;
         assert off + step >= 0;
         assert off + step + cnt <= buf.capacity() : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
@@ -1212,7 +1205,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long buf,
         int dataOff,
         int payloadSize,
         CacheDataRow row
@@ -1245,7 +1238,7 @@ public class DataPageIO extends PageIO {
      * @param payload Payload
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long buf,
         int dataOff,
         byte[] payload
     ) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/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 d2813c9..ce8266a 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, null, 0);
+        put(bag, null, 0, 0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/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 0593c60..5f9d0fa 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
@@ -76,7 +76,7 @@ public abstract class H2Tree extends BPlusTree<SearchRow, GridH2Row> {
     }
 
     /** {@inheritDoc} */
-    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, ByteBuffer buf, int idx)
+    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, long buf, int idx)
         throws IgniteCheckedException {
         return (GridH2Row)io.getLookupRow(this, buf, idx);
     }


[18/25] ignite git commit: 11810

Posted by sb...@apache.org.
11810


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

Branch: refs/heads/ignite-gg-11810
Commit: a481704b9cd2235cbdf697e6ea300492a0f6bee5
Parents: 061a059
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 13:53:40 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 13:53:40 2016 +0300

----------------------------------------------------------------------
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 96 +++++---------------
 .../internal/pagemem/impl/PageNoStoreImpl.java  | 48 +---------
 .../cache/IgniteCacheOffheapManagerImpl.java    | 29 +-----
 .../cache/database/CacheDataRowAdapter.java     | 72 +++++++++++++++
 .../IgniteCacheDatabaseSharedManager.java       |  2 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  2 +-
 .../database/BPlusTreeReuseSelfTest.java        |  3 +
 .../processors/database/BPlusTreeSelfTest.java  |  2 +-
 .../database/FreeListImplSelfTest.java          |  2 +-
 .../database/MetadataStorageSelfTest.java       |  2 +-
 10 files changed, 113 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/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 409703e..830f781a 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
@@ -20,15 +20,14 @@ package org.apache.ignite.internal.pagemem.impl;
 import java.io.Closeable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
 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.DirectMemoryRegion;
 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;
@@ -37,7 +36,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import sun.misc.JavaNioAccess;
 import sun.misc.SharedSecrets;
@@ -90,10 +88,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     public static final int PAGE_ID_OFFSET = 8;
 
     /** Page pin counter offset. */
-    public static final int PIN_CNT_OFFSET = 16;
-
-    /** Page pin counter offset. */
-    public static final int LOCK_OFFSET = 24;
+    public static final int LOCK_OFFSET = 16;
 
     /**
      * Need a 8-byte pointer for linked list, 8 bytes for internal needs (flags),
@@ -134,21 +129,28 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     /** */
     private OffheapReadWriteLock rwLock;
 
+    /** */
+    private final boolean trackAcquiredPages;
+
     /**
+     * @param log Logger.
      * @param directMemoryProvider Memory allocator to use.
      * @param sharedCtx Cache shared context.
      * @param pageSize Page size.
+     * @param trackAcquiredPages If {@code true} tracks number of allocated pages (for tests purpose only).
      */
     public PageMemoryNoStoreImpl(
         IgniteLogger log,
         DirectMemoryProvider directMemoryProvider,
         GridCacheSharedContext<?, ?> sharedCtx,
-        int pageSize
+        int pageSize,
+        boolean trackAcquiredPages
     ) {
         assert log != null || sharedCtx != null;
 
         this.log = sharedCtx != null ? sharedCtx.logger(PageMemoryNoStoreImpl.class) : log;
         this.directMemoryProvider = directMemoryProvider;
+        this.trackAcquiredPages = trackAcquiredPages;
 
         sysPageSize = pageSize + PAGE_OVERHEAD;
 
@@ -253,9 +255,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
         writePageId(absPtr, pageId);
 
-        // Clear pin counter.
-        GridUnsafe.putLong(absPtr + PIN_CNT_OFFSET, 0);
-
         // TODO pass an argument to decide whether the page should be cleaned.
         GridUnsafe.setMemory(absPtr + PAGE_OVERHEAD, sysPageSize - PAGE_OVERHEAD, (byte)0);
 
@@ -275,23 +274,21 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     @Override public Page page(int cacheId, long pageId) throws IgniteCheckedException {
         Segment seg = segment(pageId);
 
-        return seg.acquirePage(cacheId, pageId, false);
+        return seg.acquirePage(cacheId, pageId);
     }
 
     /** {@inheritDoc} */
     @Override public Page page(int cacheId, long pageId, boolean restore) throws IgniteCheckedException {
-        Segment seg = segment(pageId);
-
-        return seg.acquirePage(cacheId, pageId, restore);
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void releasePage(Page p) {
-        PageNoStoreImpl page = (PageNoStoreImpl)p;
+        if (trackAcquiredPages) {
+            Segment seg = segment(p.id());
 
-        Segment seg = segments[page.segmentIndex()];
-
-        seg.releasePage(page);
+            seg.onPageRelease();
+        }
     }
 
     /** {@inheritDoc} */
@@ -440,7 +437,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
      * @param absPtr Absolute memory pointer to the page header.
      * @param pageId Page ID to write.
      */
-    void writePageId(long absPtr, long pageId) {
+    private void writePageId(long absPtr, long pageId) {
         GridUnsafe.putLong(absPtr + PAGE_ID_OFFSET, pageId);
     }
 
@@ -540,52 +537,23 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         }
 
         /**
+         * @param cacheId Cache ID.
          * @param pageId Page ID to pin.
          * @return Pinned page impl.
          */
         @SuppressWarnings("TypeMayBeWeakened")
-        private PageNoStoreImpl acquirePage(int cacheId, long pageId, boolean restore) {
+        private PageNoStoreImpl acquirePage(int cacheId, long pageId) {
             long absPtr = absolute(pageId);
 
-            long marker = GridUnsafe.getLong(absPtr);
-
-            if (marker != PAGE_MARKER)
-                throw new IllegalStateException("Page was not allocated [absPtr=" + U.hexLong(absPtr) +
-                    ", cacheId=" + cacheId + ", pageId=" + U.hexLong(pageId) +
-                    ", marker=" + U.hexLong(marker) + ']');
-
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
-
-                if (pinCnt < 0)
-                    throw new IllegalStateException("Page has been deallocated [absPtr=" + U.hexLong(absPtr) +
-                        ", cacheId=" + cacheId + ", pageId=" + U.hexLong(pageId) + ", pinCnt=" + pinCnt + ']');
+            if (trackAcquiredPages)
+                acquiredPages.incrementAndGet();
 
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, pinCnt, pinCnt + 1))
-                    break;
-            }
-
-            acquiredPages.incrementAndGet();
-
-            return new PageNoStoreImpl(PageMemoryNoStoreImpl.this, idx, absPtr, cacheId, pageId, restore);
+            return new PageNoStoreImpl(PageMemoryNoStoreImpl.this, absPtr, cacheId, pageId);
         }
 
         /**
-         * @param pinnedPage Page to unpin.
          */
-        private void releasePage(PageNoStoreImpl pinnedPage) {
-            long absPtr = pinnedPage.absolutePointer();
-
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
-
-                assert pinCnt > 0 : "Releasing a page that was not pinned [page=" + pinnedPage +
-                    ", pinCnt=" + pinCnt + ']';
-
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, pinCnt, pinCnt - 1))
-                    break;
-            }
-
+        private void onPageRelease() {
             acquiredPages.decrementAndGet();
         }
 
@@ -598,9 +566,9 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
             pageIdx &= idxMask;
 
-            long offset = pageIdx * sysPageSize;
+            long off = pageIdx * sysPageSize;
 
-            return pagesBase + offset;
+            return pagesBase + off;
         }
 
         /**
@@ -626,20 +594,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
             long absPtr = absolute(relPtr);
 
-            // Prepare page to free.
-            // First, swap pin counter down to -1.
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
-
-                assert pinCnt >= 0 : "pinCnt=" + pinCnt + ", relPtr=" + U.hexLong(relPtr);
-
-                if (pinCnt > 0)
-                    throw new IllegalStateException("Releasing a page being in use: " + U.hexLong(relPtr));
-
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, 0, -1))
-                    break;
-            }
-
             // Second, write clean relative pointer instead of page ID.
             writePageId(absPtr, relPtr);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/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
index 5748acd..5908434 100644
--- 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
@@ -33,9 +33,6 @@ import org.jetbrains.annotations.Nullable;
  */
 public class PageNoStoreImpl implements Page {
     /** */
-    private int segIdx;
-
-    /** */
     private long absPtr;
 
     /** */
@@ -47,23 +44,17 @@ public class PageNoStoreImpl implements Page {
     /** */
     private PageMemoryNoStoreImpl pageMem;
 
-    /** Page for memory restore */
-    private final boolean noTagCheck;
-
     /**
-     * @param segIdx Segment index.
      * @param absPtr Absolute pointer.
      */
-    public PageNoStoreImpl(
-        PageMemoryNoStoreImpl pageMem, int segIdx, long absPtr, int cacheId, long pageId, boolean noTagCheck
+    PageNoStoreImpl(
+        PageMemoryNoStoreImpl pageMem, long absPtr, int cacheId, long pageId
     ) {
         this.pageMem = pageMem;
-        this.segIdx = segIdx;
         this.absPtr = absPtr;
 
         this.cacheId = cacheId;
         this.pageId = pageId;
-        this.noTagCheck = noTagCheck;
     }
 
     /**
@@ -103,20 +94,18 @@ public class PageNoStoreImpl implements Page {
 
     /** {@inheritDoc} */
     @Override public long getForWritePointer() {
-        int tag = noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS : PageIdUtils.tag(pageId);
+        int tag = PageIdUtils.tag(pageId);
         boolean locked = pageMem.writeLockPage(absPtr, tag);
 
-        if (!locked && !noTagCheck)
+        if (!locked)
             return 0L;
 
-        assert locked;
-
         return pointer();
     }
 
     /** {@inheritDoc} */
     @Override public long tryGetForWritePointer() {
-        int tag = noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+        int tag = PageIdUtils.tag(pageId);
 
         if (pageMem.tryWriteLockPage(absPtr, tag))
             return pointer();
@@ -151,38 +140,11 @@ public class PageNoStoreImpl implements Page {
         pageMem.releasePage(this);
     }
 
-    /**
-     * @return Segment index.
-     */
-    int segmentIndex() {
-        return segIdx;
-    }
-
-    /**
-     * @return Absolute pointer to the system page start.
-     */
-    long absolutePointer() {
-        return absPtr;
-    }
-
-    /**
-     * @param buf Byte buffer.
-     * @return The given buffer back.
-     */
-    private ByteBuffer reset(ByteBuffer buf) {
-        buf.order(PageMemory.NATIVE_BYTE_ORDER);
-
-        buf.rewind();
-
-        return buf;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         SB sb = new SB("PageNoStoreImpl [absPtr=0x");
 
         sb.appendHex(absPtr);
-        sb.a(", segIdx=").a(segIdx);
         sb.a(", cacheId=").a(cacheId);
         sb.a(", pageId=0x").appendHex(pageId);
         sb.a("]");

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/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 4b1d464..5fc612c 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
@@ -83,7 +83,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     protected final ConcurrentMap<Integer, CacheDataStore> partDataStores = new ConcurrentHashMap<>();
 
     /** */
-    protected final CacheDataStore removedStore = new CacheDataStoreImpl(-1, null, null, null);
+    protected final CacheDataStore rmvStore = new CacheDataStoreImpl(-1, null, null, null);
 
     /** */
     protected PendingEntriesTree pendingEntries;
@@ -1095,9 +1095,9 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             if (cmp != 0)
                 return cmp;
 
-            KeySearchRow row0 = io.getLookupRow(this, buf, idx);
+            long link = ((RowLinkIO)io).getLink(buf, idx);
 
-            return compareKeys(row0.key(), row.key());
+            return row.compareKey(cctx, link);
         }
 
         /** {@inheritDoc} */
@@ -1108,29 +1108,6 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
             return rowStore.dataRow(hash, link);
         }
-
-        /**
-         * @param key1 First key.
-         * @param key2 Second key.
-         * @return Compare result.
-         * @throws IgniteCheckedException If failed.
-         */
-        private int compareKeys(CacheObject key1, CacheObject key2) throws IgniteCheckedException {
-            byte[] bytes1 = key1.valueBytes(cctx.cacheObjectContext());
-            byte[] bytes2 = key2.valueBytes(cctx.cacheObjectContext());
-
-            int len = Math.min(bytes1.length, bytes2.length);
-
-            for (int i = 0; i < len; i++) {
-                byte b1 = bytes1[i];
-                byte b2 = bytes2[i];
-
-                if (b1 != b2)
-                    return b1 > b2 ? 1 : -1;
-            }
-
-            return Integer.compare(bytes1.length, bytes2.length);
-        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/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 d940d62..f559fd4 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
@@ -73,6 +73,78 @@ public class CacheDataRowAdapter implements CacheDataRow {
     }
 
     /**
+     * Compare key with key of record, given by link.
+     *
+     * @param cctx Context.
+     * @param link Link to second record.
+     * @return compare result.
+     * @throws IgniteCheckedException if fails.
+     */
+    public int compareKey(GridCacheContext cctx, long link) throws IgniteCheckedException {
+        byte[] bytes = key().valueBytes(cctx.cacheObjectContext());
+
+        PageMemory pageMem = cctx.shared().database().pageMemory();
+
+        try (Page page = page(pageId(link), cctx)) {
+            long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
+
+            assert pageAddr != 0L : link;
+
+            try {
+                DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+
+                DataPagePayload data = io.readPayload(pageAddr,
+                    itemId(link),
+                    pageMem.pageSize());
+
+                link = data.nextLink();
+
+                if (link == 0) {
+                    long addr = pageAddr + data.offset();
+
+                    int len = PageUtils.getInt(addr, 0);
+
+                    int size = Math.min(bytes.length, len);
+
+                    addr += 5; // Skip length and type byte.
+
+                    for (int i = 0; i < size; i++) {
+                        byte b1 = PageUtils.getByte(addr, i);
+                        byte b2 = bytes[i];
+
+                        if (b1 != b2)
+                            return b1 > b2 ? 1 : -1;
+                    }
+
+                    return Integer.compare(len, bytes.length);
+                }
+            }
+            finally {
+                page.releaseRead();
+            }
+        }
+
+        // TODO GG-11768.
+        CacheDataRowAdapter other = new CacheDataRowAdapter(link);
+        other.initFromLink(cctx, true);
+
+        byte[] bytes1 = other.key().valueBytes(cctx.cacheObjectContext());
+        byte[] bytes2 = key.valueBytes(cctx.cacheObjectContext());
+
+        int len = Math.min(bytes1.length, bytes2.length);
+
+        for (int i = 0; i < len; i++) {
+            byte b1 = bytes1[i];
+            byte b2 = bytes2[i];
+
+            if (b1 != b2)
+                return b1 > b2 ? 1 : -1;
+        }
+
+        return Integer.compare(bytes1.length, bytes2.length);
+    }
+
+    /**
      * Read row from data pages.
      *
      * @param cctx Cache context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
index 91f1f63..98e2d68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
@@ -216,7 +216,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 true,
                 sizes);
 
-        return new PageMemoryNoStoreImpl(log, memProvider, cctx, dbCfg.getPageSize());
+        return new PageMemoryNoStoreImpl(log, memProvider, cctx, dbCfg.getPageSize(), false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/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 483b228..f03c5b4 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
@@ -273,7 +273,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
         DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), memDir, true,
             sizes);
 
-        return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE);
+        return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/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 4996eef..e1d0da6 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
@@ -46,6 +46,9 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         assertTrue(TestReuseList.checkNoLocks());
     }
 
+    /**
+     *
+     */
     private static class TestReuseList extends ReuseListImpl {
         /** */
         private static ThreadLocal<Set<Long>> readLocks = new ThreadLocal<Set<Long>>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/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 b524b45..0358f5a 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
@@ -1408,7 +1408,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE, true);
 
         pageMem.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index ad5e78f..66987e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -316,7 +316,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize, true);
 
         pageMem.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a481704b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
index 252ea82..7eeda4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
@@ -160,6 +160,6 @@ public class MetadataStorageSelfTest extends GridCommonAbstractTest {
 
         DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath, clean, sizes);
 
-        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE);
+        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE, true);
     }
 }


[04/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 6f1ed0daf83e363f259a3cfd3433f1bbc497aca6
Parents: 94f22a0
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 23 13:28:01 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 23 13:28:01 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/pagemem/Page.java    |   4 +
 .../ignite/internal/pagemem/PageUtils.java      |  12 +
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  10 +
 .../processors/cache/CacheObjectAdapter.java    |  10 +-
 .../processors/cache/IncompleteObject.java      |  13 ++
 .../cache/database/CacheDataRowAdapter.java     |  64 +++++-
 .../cache/database/DataStructure.java           |   7 +
 .../cache/database/MetadataStorage.java         | 115 ++++------
 .../cache/database/freelist/FreeListImpl.java   |  21 +-
 .../cache/database/freelist/PagesList.java      |   2 +-
 .../database/freelist/io/PagesListMetaIO.java   |   4 +-
 .../database/freelist/io/PagesListNodeIO.java   |   4 +-
 .../cache/database/tree/BPlusTree.java          |  18 +-
 .../cache/database/tree/io/BPlusIO.java         |  10 +-
 .../cache/database/tree/io/BPlusInnerIO.java    |   5 +-
 .../cache/database/tree/io/CacheVersionIO.java  |  48 ++++
 .../cache/database/tree/io/DataPageIO.java      | 226 +++++++++++--------
 .../cache/database/tree/io/DataPagePayload.java |  50 ++++
 .../cache/database/tree/io/PageIO.java          |   2 +-
 .../cache/database/tree/io/PageMetaIO.java      |   4 +-
 .../database/tree/io/PagePartitionMetaIO.java   |   4 +-
 21 files changed, 413 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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
index 89848cc..5f9d424 100644
--- 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
@@ -95,6 +95,10 @@ public interface Page extends AutoCloseable {
      */
     public Boolean fullPageWalRecordPolicy();
 
+    public int size();
+
+    public ByteBuffer pageBuffer();
+
     /**
      * Release page.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index 69719e1..d4ca73d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -32,6 +32,18 @@ public class PageUtils {
         return unsafe.getByte(buf, off);
     }
 
+    public static byte[] getBytes(long buf, int off, int len) {
+        byte[] bytes = new byte[len];
+
+        unsafe.copyMemory(null, buf + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
+
+        return bytes;
+    }
+
+    public static void getBytes(long src, int srcOff, byte[] dst, int dstOff, int len) {
+        unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
+    }
+
     public static short getShort(long buf, int off) {
         return unsafe.getShort(buf, off);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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
index 0ad206b..bbe2993 100644
--- 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
@@ -71,6 +71,11 @@ public class PageNoStoreImpl implements Page {
         buf = pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
     }
 
+    /** {@inheritDoc} */
+    @Override public ByteBuffer pageBuffer() {
+        return pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
+    }
+
     private long pointer() {
         return absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD;
     }
@@ -175,6 +180,11 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
+    @Override public int size() {
+        return pageMem.pageSize();
+    }
+
+    /** {@inheritDoc} */
     @Override public void close() {
         pageMem.releasePage(this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index a34e98d..1394fc2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@ -164,8 +164,14 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
      * @return {@code True} if data were successfully written.
      * @throws IgniteCheckedException If failed.
      */
-    public static boolean putValue(byte cacheObjType, final ByteBuffer buf, int off, int len,
-        byte[] valBytes, final int start) throws IgniteCheckedException {
+    public static boolean putValue(byte cacheObjType,
+        final ByteBuffer buf,
+        int off,
+        int len,
+        byte[] valBytes,
+        final int start)
+        throws IgniteCheckedException
+    {
         int dataLen = valBytes.length;
 
         if (buf.remaining() < len)

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
index 666fc27..ce0e306 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
  * Incomplete object.
@@ -86,4 +87,16 @@ public class IncompleteObject<T> {
 
         off += len;
     }
+
+    public int readData(long buf, int remaining) {
+        assert data != null;
+
+        final int len = Math.min(data.length - off, remaining);
+
+        PageUtils.getBytes(buf, 0, data, off, len);
+
+        off += len;
+
+        return len;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 b5babc4..59e2fc4 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
@@ -21,6 +21,7 @@ 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.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -29,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.IncompleteObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -68,7 +70,6 @@ public class CacheDataRowAdapter implements CacheDataRow {
         // Link can be 0 here.
         this.link = link;
     }
-
     /**
      * Read row from data pages.
      *
@@ -89,19 +90,21 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
         do {
             try (Page page = page(pageId(nextLink), cctx)) {
-                ByteBuffer buf = page.getForRead(); // Non-empty data page must not be recycled.
+                long buf = page.getForReadPointer(); // Non-empty data page must not be recycled.
 
-                assert buf != null: nextLink;
+                assert buf != 0L : nextLink;
 
                 try {
                     DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
 
-                    nextLink = io.setPositionAndLimitOnPayload(buf, itemId(nextLink));
+                    DataPagePayload data = io.readPayload(buf, itemId(nextLink), page.size());
+
+                    nextLink = data.nextLink();
 
                     if (first) {
                         if (nextLink == 0) {
                             // Fast path for a single page row.
-                            readFullRow(coctx, buf, keyOnly);
+                            readFullRow(coctx, data.offset(), keyOnly);
 
                             return;
                         }
@@ -109,7 +112,12 @@ public class CacheDataRowAdapter implements CacheDataRow {
                         first = false;
                     }
 
-                    incomplete = readFragment(coctx, buf, keyOnly, incomplete);
+                    ByteBuffer buf0 = page.pageBuffer();
+
+                    buf0.position(data.offset());
+                    buf0.limit(data.offset() + data.payloadSize());
+
+                    incomplete = readFragment(coctx, buf0, keyOnly, incomplete);
 
                     if (keyOnly && key != null)
                         return;
@@ -121,7 +129,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         }
         while(nextLink != 0);
 
-        assert isReady(): "ready";
+        assert isReady() : "ready";
     }
 
     /**
@@ -179,8 +187,23 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param keyOnly {@code true} If need to read only key object.
      * @throws IgniteCheckedException If failed.
      */
-    private void readFullRow(CacheObjectContext coctx, ByteBuffer buf, boolean keyOnly) throws IgniteCheckedException {
-        key = coctx.processor().toKeyCacheObject(coctx, buf);
+    private void readFullRow(CacheObjectContext coctx, long buf, boolean keyOnly) throws IgniteCheckedException {
+        int off = 0;
+
+        int len = PageUtils.getInt(buf, off);
+        off += 4;
+
+        if (len == 0)
+            key = null;
+        else {
+            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            off += len;
+
+            byte type = PageUtils.getByte(buf, off);
+            off++;
+
+            key = coctx.processor().toKeyCacheObject(coctx, type, bytes);
+        }
 
         if (keyOnly) {
             assert key != null: "key";
@@ -188,9 +211,26 @@ public class CacheDataRowAdapter implements CacheDataRow {
             return;
         }
 
-        val = coctx.processor().toCacheObject(coctx, buf);
-        ver = CacheVersionIO.read(buf, false);
-        expireTime = buf.getLong();
+        len = PageUtils.getInt(buf, off);
+        off += 4;
+
+        if (len == 0)
+            val = null;
+        else {
+            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            off += len;
+
+            byte type = PageUtils.getByte(buf, off);
+            off++;
+
+            val = coctx.processor().toCacheObject(coctx, type, bytes);
+        }
+
+        ver = CacheVersionIO.read(buf + off, false);
+
+        off += CacheVersionIO.size(ver, false);
+
+        expireTime = PageUtils.getLong(buf, off);
 
         assert isReady(): "ready";
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 26df029..ce5422f 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
@@ -171,6 +171,13 @@ public abstract class DataStructure implements PageLockListener {
         PageHandler.readUnlock(page, buf, this);
     }
 
+    /**
+     * @return Page size.
+     */
+    protected final int pageSize() {
+        return pageMem.pageSize();
+    }
+
     /** {@inheritDoc} */
     @Override public void onBeforeWriteLock(Page page) {
         // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
index 26151ac..b19d03d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.database;
 
-import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 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;
 
 /**
@@ -190,19 +191,19 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(final BPlusIO<IndexItem> io, final ByteBuffer buf, final int idx,
+        @Override protected int compare(final BPlusIO<IndexItem> io, final long buf, final int idx,
             final IndexItem row) throws IgniteCheckedException {
             final int off = ((IndexIO)io).getOffset(idx);
 
             int shift = 0;
 
             // Compare index names.
-            final byte len = buf.get(off + shift);
+            final byte len = PageUtils.getByte(off, shift);
 
             shift += BYTE_LEN;
 
             for (int i = 0; i < len && i < row.idxName.length; i++) {
-                final int cmp = Byte.compare(buf.get(off + i + shift), row.idxName[i]);
+                final int cmp = Byte.compare(PageUtils.getByte(off, i + shift), row.idxName[i]);
 
                 if (cmp != 0)
                     return cmp;
@@ -212,7 +213,7 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final ByteBuffer buf,
+        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final long buf,
             final int idx) throws IgniteCheckedException {
             return readRow(buf, ((IndexIO)io).getOffset(idx));
         }
@@ -251,27 +252,18 @@ public class MetadataStorage implements MetaStore {
      * @param row Row to store.
      */
     private static void storeRow(
-        final ByteBuffer buf,
+        final long buf,
         final int off,
         final IndexItem row
     ) {
-        int origPos = buf.position();
+        // Index name length.
+        PageUtils.putByte(buf, off, (byte)row.idxName.length);
 
-        try {
-            buf.position(off);
-
-            // Index name length.
-            buf.put((byte)row.idxName.length);
-
-            // Index name.
-            buf.put(row.idxName);
+        // Index name.
+        PageUtils.putBytes(buf, off + 1, row.idxName);
 
-            // Page ID.
-            buf.putLong(row.pageId);
-        }
-        finally {
-            buf.position(origPos);
-        }
+        // Page ID.
+        PageUtils.putLong(buf, off + 1 + row.idxName.length, row.pageId);
     }
 
     /**
@@ -283,39 +275,20 @@ public class MetadataStorage implements MetaStore {
      * @param srcOff Src buf offset.
      */
     private static void storeRow(
-        final ByteBuffer dst,
+        final long dst,
         final int dstOff,
-        final ByteBuffer src,
+        final long src,
         final int srcOff
     ) {
-        int srcOrigPos = src.position();
-        int dstOrigPos = dst.position();
-
-        try {
-            src.position(srcOff);
-            dst.position(dstOff);
-
-            // Index name length.
-            final byte len = src.get();
-
-            dst.put(len);
-
-            int lim = src.limit();
+        // Index name length.
+        final byte len = PageUtils.getByte(src, srcOff);
 
-            src.limit(src.position() + len);
+        PageUtils.putByte(dst, dstOff, len);
 
-            // Index name.
-            dst.put(src);
+        PageHandler.copyMemory(src, srcOff + 1, dst, dstOff + 1, len);
 
-            src.limit(lim);
-
-            // Page ID.
-            dst.putLong(src.getLong());
-        }
-        finally {
-            src.position(srcOrigPos);
-            dst.position(dstOrigPos);
-        }
+        // Page ID.
+        PageUtils.putLong(dst, dstOff + 1 + len, PageUtils.getLong(src, srcOff + 1 + len));
     }
 
     /**
@@ -325,28 +298,17 @@ public class MetadataStorage implements MetaStore {
      * @param off Offset in buf.
      * @return Read row.
      */
-    private static IndexItem readRow(final ByteBuffer buf, final int off) {
-        int origOff = buf.position();
-
-        try {
-            buf.position(off);
+    private static IndexItem readRow(final long buf, final int off) {
+        // Index name length.
+        final int len = PageUtils.getByte(buf, 0) & 0xFF;
 
-            // Index name length.
-            final int len = buf.get() & 0xFF;
+        // Index name.
+        final byte[] idxName = PageUtils.getBytes(buf, 1, len);
 
-            // Index name.
-            final byte[] idxName = new byte[len];
+        // Page ID.
+        final long pageId = PageUtils.getLong(buf, off + 1 + len);
 
-            buf.get(idxName);
-
-            // Page ID.
-            final long pageId = buf.getLong();
-
-            return new IndexItem(idxName, pageId);
-        }
-        finally {
-            buf.position(origOff);
-        }
+        return new IndexItem(idxName, pageId);
     }
 
     /**
@@ -378,19 +340,19 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, IndexItem row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
             storeRow(buf, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void store(final long dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
+            final long src,
             final int srcIdx) throws IgniteCheckedException {
             storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final long buf,
             final int idx) throws IgniteCheckedException {
             return readRow(buf, offset(idx));
         }
@@ -419,19 +381,22 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, IndexItem row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
             storeRow(buf, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void store(final long dst,
+            final int dstIdx,
+            final BPlusIO<IndexItem> srcIo,
+            final long src,
             final int srcIdx) throws IgniteCheckedException {
             storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree,
+            final long buf,
             final int idx) throws IgniteCheckedException {
             return readRow(buf, offset(idx));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 1d2524d..23f50f1 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
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist;
 
-import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 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.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertFragmentRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertRecord;
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageRemoveRecord;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 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;
@@ -114,19 +115,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 CacheDataRow row,
                 int rowSize
             ) throws IgniteCheckedException {
-                // TODO: context parameter.
-                io.addRow(buf, row, rowSize);
+                io.addRow(buf, row, rowSize, pageSize());
 
                 if (isWalDeltaRecordNeeded(wal, page)) {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[rowSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
 
-                    assert buf.remaining() == rowSize;
+                    assert data.payloadSize() == rowSize;
 
-                    buf.get(payload);
-                    buf.position(0);
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, rowSize);
 
                     wal.log(new DataPageInsertRecord(
                         cacheId,
@@ -166,9 +165,9 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[payloadSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
-                    buf.get(payload);
-                    buf.position(0);
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
+
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, payloadSize);
 
                     wal.log(new DataPageInsertFragmentRecord(cacheId, page.id(), payload, lastLink));
                 }
@@ -187,7 +186,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
             assert oldFreeSpace >= 0: oldFreeSpace;
 
-            long nextLink = io.removeRow(buf, itemId);
+            long nextLink = io.removeRow(buf, itemId, pageSize());
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new DataPageRemoveRecord(cacheId, page.id(), itemId));

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 8311b65..f925daa 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
@@ -884,7 +884,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(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 6ac8cde..67e2667 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
@@ -53,8 +53,8 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setCount(buf, 0);
         setNextMetaPageId(buf, 0L);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 c92e4a9..30f47d6 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
@@ -52,8 +52,8 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setEmpty(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 b8a1004..a92f39a 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
@@ -408,7 +408,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             // 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(pageMem.pageSize(), buf)))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(pageSize(), buf)))) {
                 // 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);
@@ -527,7 +527,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getRootLevel(buf); // Can drop only root.
 
-            io.cutRoot(pageMem.pageSize(), buf);
+            io.cutRoot(pageSize(), buf);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
@@ -547,7 +547,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getLevelsCount(buf);
 
-            io.addRoot(pageMem.pageSize(), buf, rootPageId);
+            io.addRoot(pageSize(), buf, rootPageId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
@@ -565,7 +565,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            io.initRoot(pageMem.pageSize(), buf, rootId);
+            io.initRoot(pageSize(), buf, rootId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageInitRootRecord(cacheId, meta.id(), rootId));
@@ -1721,7 +1721,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         // Update forward page.
-        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt);
+        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt, pageSize());
 
         // TODO GG-11640 log a correct forward page record.
         fwd.fullPageWalRecordPolicy(Boolean.TRUE);
@@ -2184,7 +2184,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          */
         private L insert(Page page, BPlusIO<L> io, long buf, int idx, int lvl)
             throws IgniteCheckedException {
-            int maxCnt = io.getMaxCount(pageMem.pageSize(), buf);
+            int maxCnt = io.getMaxCount(pageSize(), buf);
             int cnt = io.getCount(buf);
 
             if (cnt == maxCnt) // Need to split page.
@@ -2284,7 +2284,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                                 long pageId = PageIO.getPageId(buf);
 
-                                inner(io).initNewRoot(newRootBuf, newRootId, pageId, moveUpRow, null, fwdId);
+                                inner(io).initNewRoot(newRootBuf, newRootId, pageId, moveUpRow, null, fwdId, pageSize());
 
                                 if (needWalDeltaRecord(newRoot))
                                     wal.log(new NewRootInitRecord<>(cacheId, newRoot.id(), newRootId,
@@ -2624,7 +2624,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                         // Exit: we are done.
                     }
                     else if (tail.sibling != null &&
-                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(pageMem.pageSize(), tail.buf)) {
+                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(pageSize(), tail.buf)) {
                         // Release everything lower than tail, we've already merged this path.
                         doReleaseTail(tail.down);
                         tail.down = null;
@@ -2939,7 +2939,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             boolean emptyBranch = needMergeEmptyBranch == TRUE || needMergeEmptyBranch == READY;
 
-            if (!left.io.merge(pageMem.pageSize(), prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch))
+            if (!left.io.merge(pageSize(), prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch))
                 return false;
 
             // Invalidate indexes after successful merge.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
index 6c739a5..7ebd2f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
@@ -72,8 +72,8 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setCount(buf, 0);
         setForward(buf, 0);
@@ -263,6 +263,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param fwdBuf Forward buffer.
      * @param mid Bisection index.
      * @param cnt Initial elements count in the page being split.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void splitForwardPage(
@@ -270,9 +271,10 @@ public abstract class BPlusIO<L> extends PageIO {
         long fwdId,
         long fwdBuf,
         int mid,
-        int cnt
+        int cnt,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(fwdBuf, fwdId);
+        initNewPage(fwdBuf, fwdId, pageSize);
 
         cnt -= mid;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
index d0ead51..ebf7522 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
@@ -157,9 +157,10 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
         long leftChildId,
         L row,
         byte[] rowBytes,
-        long rightChildId
+        long rightChildId,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(newRootBuf, newRootId);
+        initNewPage(newRootBuf, newRootId, pageSize);
 
         setCount(newRootBuf, 1);
         setLeft(newRootBuf, 0, leftChildId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
index ed75d12..db36ce2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
+import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -102,6 +103,53 @@ public class CacheVersionIO {
      * @return Size of serialized version.
      * @throws IgniteCheckedException If failed.
      */
+    public static int readSize(ByteBuffer buf, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(buf.get(buf.position()), allowNull);
+
+        switch (protoVer) {
+            case NULL_PROTO_VER:
+                return NULL_SIZE;
+
+            case 1:
+                return SIZE_V1;
+
+            default:
+                throw new IllegalStateException();
+        }
+    }
+
+    /**
+     * Reads GridCacheVersion instance from the given buffer. Moves buffer's position by the number of used
+     * bytes.
+     *
+     * @param buf Byte buffer.
+     * @param allowNull Is {@code null} version allowed.
+     * @return Version.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static GridCacheVersion read(ByteBuffer buf, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(buf.get(), allowNull);
+
+        if (protoVer == NULL_PROTO_VER)
+            return null;
+
+        int topVer = buf.getInt();
+        int nodeOrderDrId = buf.getInt();
+        long globalTime = buf.getLong();
+        long order = buf.getLong();
+
+        return new GridCacheVersion(topVer, nodeOrderDrId, globalTime, order);
+    }
+
+    /**
+     * Gets needed buffer size to read the whole version instance.
+     * Does not change buffer position.
+     *
+     * @param buf Buffer.
+     * @param allowNull Is {@code null} version allowed.
+     * @return Size of serialized version.
+     * @throws IgniteCheckedException If failed.
+     */
     public static int readSize(long buf, boolean allowNull) throws IgniteCheckedException {
         byte protoVer = checkProtocolVersion(PageUtils.getByte(buf, 0), allowNull);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index ca8ea39..31fb19b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -88,21 +89,22 @@ public class DataPageIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
-        setEmptyPage(buf);
+        setEmptyPage(buf, pageSize);
         setFreeListPageId(buf, 0L);
     }
 
     /**
      * @param buf Buffer.
+     * @param pageSize Page size.
      */
-    private void setEmptyPage(long buf) {
+    private void setEmptyPage(long buf, int pageSize) {
         setDirectCount(buf, 0);
         setIndirectCount(buf, 0);
-        setFirstEntryOffset(buf, buf.capacity());
-        setRealFreeSpace(buf, buf.capacity() - ITEMS_OFF);
+        setFirstEntryOffset(buf, pageSize, pageSize);
+        setRealFreeSpace(buf, pageSize - ITEMS_OFF, pageSize);
     }
 
     /**
@@ -163,9 +165,10 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param dataOff Entry data offset.
+     * @param pageSize Page size.
      */
-    private void setFirstEntryOffset(long buf, int dataOff) {
-        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= buf.capacity(): dataOff;
+    private void setFirstEntryOffset(long buf, int dataOff, int pageSize) {
+        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= pageSize : dataOff;
 
         PageUtils.putShort(buf, FIRST_ENTRY_OFF, (short)dataOff);
     }
@@ -182,8 +185,8 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param freeSpace Free space.
      */
-    private void setRealFreeSpace(long buf, int freeSpace) {
-        assert freeSpace == actualFreeSpace(buf): freeSpace + " != " + actualFreeSpace(buf);
+    private void setRealFreeSpace(long buf, int freeSpace, int pageSize) {
+        assert freeSpace == actualFreeSpace(buf, pageSize): freeSpace + " != " + actualFreeSpace(buf, pageSize);
 
         PageUtils.putShort(buf, FREE_SPACE_OFF, (short)freeSpace);
     }
@@ -218,7 +221,7 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * Equivalent for {@link #actualFreeSpace(long)} but reads saved value.
+     * Equivalent for {@link #actualFreeSpace(long, int)} but reads saved value.
      *
      * @param buf Buffer.
      * @return Free space.
@@ -316,9 +319,10 @@ public class DataPageIO extends PageIO {
 
     /**
      * @param buf Buffer.
+     * @param pageSize Page size.
      * @return String representation.
      */
-    public String printPageLayout(long buf) {
+    public String printPageLayout(long buf, int pageSize) {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
         int free = getRealFreeSpace(buf);
@@ -337,7 +341,7 @@ public class DataPageIO extends PageIO {
 
             short item = getItem(buf, i);
 
-            if (item < ITEMS_OFF || item >= buf.capacity())
+            if (item < ITEMS_OFF || item >= pageSize)
                 valid = false;
 
             entriesSize += getPageEntrySize(buf, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
@@ -375,7 +379,7 @@ public class DataPageIO extends PageIO {
 
         b.a("][free=").a(free);
 
-        int actualFree = buf.capacity() - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
+        int actualFree = pageSize - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
 
         if (free != actualFree) {
             b.a(", actualFree=").a(actualFree);
@@ -395,19 +399,19 @@ public class DataPageIO extends PageIO {
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @return Data entry offset in bytes.
      */
-    private int getDataOffset(long buf, int itemId) {
+    private int getDataOffset(long buf, int itemId, int pageSize) {
         assert checkIndex(itemId): itemId;
 
         int directCnt = getDirectCount(buf);
 
-        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(buf);
+        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(buf, pageSize);
 
         if (itemId >= directCnt) { // Need to do indirect lookup.
             int indirectCnt = getIndirectCount(buf);
 
             // Must have indirect items here.
             assert indirectCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", indirectCnt=" + indirectCnt +
-                ", page=" + printPageLayout(buf);
+                ", page=" + printPageLayout(buf, pageSize);
 
             int indirectItemIdx = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
 
@@ -449,18 +453,16 @@ public class DataPageIO extends PageIO {
      * @param itemId Item to position on.
      * @return Link to the next fragment or {@code 0} if it is the last fragment or the data row is not fragmented.
      */
-    public long setPositionAndLimitOnPayload(final long buf, final int itemId) {
-        int dataOff = getDataOffset(buf, itemId);
+    public DataPagePayload readPayload(final long buf, final int itemId, final int pageSize) {
+        int dataOff = getDataOffset(buf, itemId, pageSize);
 
         boolean fragmented = isFragmented(buf, dataOff);
         long nextLink = fragmented ? getNextFragmentLink(buf, dataOff) : 0;
         int payloadSize = getPageEntrySize(buf, dataOff, 0);
 
-        buf.position(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0));
-
-        buf.limit(buf.position() + payloadSize);
-
-        return nextLink;
+        return new DataPagePayload(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0),
+            payloadSize,
+            nextLink);
     }
 
     /**
@@ -593,13 +595,14 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
+     * @param pageSize Page size.
      * @return Next link for fragmented entries or {@code 0} if none.
      * @throws IgniteCheckedException If failed.
      */
-    public long removeRow(long buf, int itemId) throws IgniteCheckedException {
+    public long removeRow(long buf, int itemId, int pageSize) throws IgniteCheckedException {
         assert checkIndex(itemId) : itemId;
 
-        final int dataOff = getDataOffset(buf, itemId);
+        final int dataOff = getDataOffset(buf, itemId, pageSize);
         final long nextLink = isFragmented(buf, dataOff) ? getNextFragmentLink(buf, dataOff) : 0;
 
         // Record original counts to calculate delta in free space in the end of remove.
@@ -615,7 +618,7 @@ public class DataPageIO extends PageIO {
             assert (indirectCnt == 0 && itemId == 0) ||
                 (indirectCnt == 1 && itemId == itemId(getItem(buf, 1))) : itemId;
 
-            setEmptyPage(buf);
+            setEmptyPage(buf, pageSize);
         }
         else {
             // Get the entry size before the actual remove.
@@ -642,15 +645,15 @@ public class DataPageIO extends PageIO {
 
             if (indirectId == 0) {// For the last direct item with no indirect item.
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectCnt, -1);
+                    moveItems(buf, directCnt, indirectCnt, -1, pageSize);
                 else
                     curIndirectCnt++;
             }
             else {
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectId - directCnt, -1);
+                    moveItems(buf, directCnt, indirectId - directCnt, -1, pageSize);
 
-                moveItems(buf, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1);
+                moveItems(buf, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1, pageSize);
 
                 if (dropLast)
                     curIndirectCnt--;
@@ -662,8 +665,9 @@ public class DataPageIO extends PageIO {
             assert getIndirectCount(buf) <= getDirectCount(buf);
 
             // Increase free space.
-            setRealFreeSpace(buf, getRealFreeSpace(buf) + rmvEntrySize +
-                ITEM_SIZE * (directCnt - getDirectCount(buf) + indirectCnt - getIndirectCount(buf)));
+            setRealFreeSpace(buf,
+                getRealFreeSpace(buf) + rmvEntrySize + ITEM_SIZE * (directCnt - getDirectCount(buf) + indirectCnt - getIndirectCount(buf)),
+                pageSize);
         }
 
         return nextLink;
@@ -674,12 +678,13 @@ public class DataPageIO extends PageIO {
      * @param idx Index.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveItems(long buf, int idx, int cnt, int step) {
+    private void moveItems(long buf, int idx, int cnt, int step, int pageSize) {
         assert cnt >= 0: cnt;
 
         if (cnt != 0)
-            moveBytes(buf, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE);
+            moveBytes(buf, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE, pageSize);
     }
 
     /**
@@ -699,12 +704,14 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param row Cache data row.
      * @param rowSize Row size.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
         long buf,
         CacheDataRow row,
-        int rowSize
+        int rowSize,
+        int pageSize
     ) throws IgniteCheckedException {
         assert rowSize <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
 
@@ -713,11 +720,11 @@ public class DataPageIO extends PageIO {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         writeRowData(buf, dataOff, rowSize, row);
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         setLink(row, buf, itemId);
     }
@@ -726,11 +733,14 @@ public class DataPageIO extends PageIO {
      * Adds row to this data page and sets respective link to the given row object.
      *
      * @param buf Buffer.
+     * @param payload Payload.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
         long buf,
-        byte[] payload
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
         assert payload.length <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
 
@@ -739,11 +749,11 @@ public class DataPageIO extends PageIO {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         writeRowData(buf, dataOff, payload);
 
-        addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
     }
 
     /**
@@ -752,6 +762,7 @@ public class DataPageIO extends PageIO {
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff First entry offset.
+     * @param pageSize Page size.
      * @return First entry offset after compaction.
      */
     private int compactIfNeed(
@@ -759,10 +770,11 @@ public class DataPageIO extends PageIO {
         final int entryFullSize,
         final int directCnt,
         final int indirectCnt,
-        int dataOff
+        int dataOff,
+        int pageSize
     ) {
         if (!isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt)) {
-            dataOff = compactDataEntries(buf, directCnt);
+            dataOff = compactDataEntries(buf, directCnt, pageSize);
 
             assert isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt);
         }
@@ -778,20 +790,27 @@ public class DataPageIO extends PageIO {
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff Data offset.
+     * @param pageSize Page size.
      * @return Item ID.
      */
-    private int addItem(final long buf, final int fullEntrySize, final int directCnt,
-        final int indirectCnt, final int dataOff) {
-        setFirstEntryOffset(buf, dataOff);
+    private int addItem(final long buf,
+        final int fullEntrySize,
+        final int directCnt,
+        final int indirectCnt,
+        final int dataOff,
+        final int pageSize)
+    {
+        setFirstEntryOffset(buf, dataOff, pageSize);
 
-        int itemId = insertItem(buf, dataOff, directCnt, indirectCnt);
+        int itemId = insertItem(buf, dataOff, directCnt, indirectCnt, pageSize);
 
         assert checkIndex(itemId): itemId;
         assert getIndirectCount(buf) <= getDirectCount(buf);
 
         // Update free space. If number of indirect items changed, then we were able to reuse an item slot.
-        setRealFreeSpace(buf, getRealFreeSpace(buf) - fullEntrySize +
-            (getIndirectCount(buf) != indirectCnt ? ITEM_SIZE : 0));
+        setRealFreeSpace(buf,
+            getRealFreeSpace(buf) - fullEntrySize + (getIndirectCount(buf) != indirectCnt ? ITEM_SIZE : 0),
+            pageSize);
 
         return itemId;
     }
@@ -801,13 +820,14 @@ public class DataPageIO extends PageIO {
      * @param fullEntrySize Full entry size.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Offset in the buffer where the entry must be written.
      */
-    private int getDataOffsetForWrite(long buf, int fullEntrySize, int directCnt, int indirectCnt) {
+    private int getDataOffsetForWrite(long buf, int fullEntrySize, int directCnt, int indirectCnt, int pageSize) {
         int dataOff = getFirstEntryOffset(buf);
 
         // Compact if we do not have enough space for entry.
-        dataOff = compactIfNeed(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        dataOff = compactIfNeed(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         // We will write data right before the first entry.
         dataOff -= fullEntrySize - ITEM_SIZE;
@@ -826,7 +846,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public int addRowFragment(
-        long buf,
+        ByteBuffer buf,
         CacheDataRow row,
         int written,
         int rowSize
@@ -863,12 +883,13 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     private int addRowFragment(
-        long buf,
+        ByteBuffer buf,
         int written,
         int rowSize,
         long lastLink,
         CacheDataRow row,
-        byte[] payload
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
         assert payload == null ^ row == null;
 
@@ -881,18 +902,25 @@ public class DataPageIO extends PageIO {
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
         int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
 
-        PageUtils.putShort(buf, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
-        PageUtils.putLong(buf, dataOff + 2, lastLink);
+        try {
+            buf.position(dataOff);
+
+            buf.putShort((short) (payloadSize | FRAGMENTED_FLAG));
+            buf.putLong(lastLink);
 
-        if (payload == null) {
-            int rowOff = rowSize - written - payloadSize;
+            if (payload == null) {
+                int rowOff = rowSize - written - payloadSize;
 
-            writeFragmentData(row, buf + dataOff + 10, rowOff, payloadSize);
+                writeFragmentData(row, buf, rowOff, payloadSize);
+            }
+            else
+                buf.put(payload);
+        }
+        finally {
+            buf.position(0);
         }
-        else
-            PageUtils.putBytes(buf, dataOff + 10, payload);
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         if (row != null)
             setLink(row, buf, itemId);
@@ -920,7 +948,7 @@ public class DataPageIO extends PageIO {
      */
     private void writeFragmentData(
         final CacheDataRow row,
-        final long buf,
+        final ByteBuffer buf,
         final int rowOff,
         final int payloadSize
     ) throws IgniteCheckedException {
@@ -946,7 +974,7 @@ public class DataPageIO extends PageIO {
      */
     private int writeFragment(
         final CacheDataRow row,
-        final long buf,
+        final ByteBuffer buf,
         final int rowOff,
         final int payloadSize,
         final EntryPart type,
@@ -1080,9 +1108,10 @@ public class DataPageIO extends PageIO {
      * @param dataOff Data offset.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Item ID (insertion index).
      */
-    private int insertItem(long buf, int dataOff, int directCnt, int indirectCnt) {
+    private int insertItem(long buf, int dataOff, int directCnt, int indirectCnt, int pageSize) {
         if (indirectCnt > 0) {
             // If the first indirect item is on correct place to become the last direct item, do the transition
             // and insert the new item into the free slot which was referenced by this first indirect item.
@@ -1102,7 +1131,7 @@ public class DataPageIO extends PageIO {
         }
 
         // Move all the indirect items forward to make a free slot and insert new item at the end of direct items.
-        moveItems(buf, directCnt, indirectCnt, +1);
+        moveItems(buf, directCnt, indirectCnt, +1, pageSize);
 
         setItem(buf, directCnt, directItemFromOffset(dataOff));
 
@@ -1115,9 +1144,10 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param directCnt Direct items count.
+     * @param pageSize Page size.
      * @return New first entry offset.
      */
-    private int compactDataEntries(long buf, int directCnt) {
+    private int compactDataEntries(long buf, int directCnt, int pageSize) {
         assert checkCount(directCnt): directCnt;
 
         int[] offs = new int[directCnt];
@@ -1131,7 +1161,7 @@ public class DataPageIO extends PageIO {
         Arrays.sort(offs);
 
         // Move right all of the entries if possible to make the page as compact as possible to its tail.
-        int prevOff = buf.capacity();
+        int prevOff = pageSize;
 
         for (int i = directCnt - 1; i >= 0; i--) {
             int off = offs[i] >>> 8;
@@ -1145,7 +1175,7 @@ public class DataPageIO extends PageIO {
             if (delta != 0) { // Move right.
                 assert delta > 0: delta;
 
-                moveBytes(buf, off, entrySize, delta);
+                moveBytes(buf, off, entrySize, delta, pageSize);
 
                 int itemId = offs[i] & 0xFF;
 
@@ -1164,9 +1194,10 @@ public class DataPageIO extends PageIO {
      * Full-scan free space calculation procedure.
      *
      * @param buf Buffer to scan.
+     * @param pageSize Page size.
      * @return Actual free space in the buffer.
      */
-    private int actualFreeSpace(long buf) {
+    private int actualFreeSpace(long buf, int pageSize) {
         int directCnt = getDirectCount(buf);
 
         int entriesSize = 0;
@@ -1179,7 +1210,7 @@ public class DataPageIO extends PageIO {
             entriesSize += entrySize;
         }
 
-        return buf.capacity() - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(buf)) * ITEM_SIZE;
+        return pageSize - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(buf)) * ITEM_SIZE;
     }
 
     /**
@@ -1187,12 +1218,13 @@ public class DataPageIO extends PageIO {
      * @param off Offset.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveBytes(long buf, int off, int cnt, int step) {
+    private void moveBytes(long buf, int off, int cnt, int step, int pageSize) {
         assert step != 0: step;
         assert off + step >= 0;
-        assert off + step + cnt <= buf.capacity() : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
-            ", cap=" + buf.capacity() + ']';
+        assert off + step + cnt <= pageSize : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
+            ", cap=" + pageSize + ']';
 
         PageHandler.copyMemory(buf, buf, off, off + step, cnt);
     }
@@ -1210,26 +1242,36 @@ public class DataPageIO extends PageIO {
         int payloadSize,
         CacheDataRow row
     ) throws IgniteCheckedException {
-        try {
-            buf.position(dataOff);
+        PageUtils.putShort(buf, dataOff, (short)payloadSize);
+        dataOff += 2;
 
-            buf.putShort((short)payloadSize);
+        byte[] bytes = row.key().valueBytes(null);
 
-            boolean ok = row.key().putValue(buf);
+        PageUtils.putInt(buf, dataOff, bytes.length);
+        dataOff += 4;
 
-            assert ok;
+        PageUtils.putBytes(buf, dataOff, bytes);
+        dataOff += bytes.length;
 
-            ok = row.value().putValue(buf);
+        PageUtils.putByte(buf, dataOff, row.key().cacheObjectType());
+        dataOff++;
 
-            assert ok;
+        bytes = row.value().valueBytes(null);
 
-            CacheVersionIO.write(buf, row.version(), false);
+        PageUtils.putInt(buf, dataOff, bytes.length);
+        dataOff += 4;
 
-            buf.putLong(row.expireTime());
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putBytes(buf, dataOff, bytes);
+        dataOff += bytes.length;
+
+        PageUtils.putByte(buf, dataOff, row.value().cacheObjectType());
+        dataOff++;
+
+        CacheVersionIO.write(buf + dataOff, row.version(), false);
+
+        dataOff += CacheVersionIO.size(row.version(), false);
+
+        PageUtils.putLong(buf, dataOff, row.expireTime());
     }
 
     /**
@@ -1242,15 +1284,9 @@ public class DataPageIO extends PageIO {
         int dataOff,
         byte[] payload
     ) {
-        try {
-            buf.position(dataOff);
+        PageUtils.putShort(buf, dataOff, (short)payload.length);
+        dataOff += 2;
 
-            buf.putShort((short)payload.length);
-
-            buf.put(payload);
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putBytes(buf, dataOff, payload);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
new file mode 100644
index 0000000..203429e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.database.tree.io;
+
+/**
+ *
+ */
+public class DataPagePayload {
+    /** */
+    private final int off;
+
+    /** */
+    private final int payloadSize;
+
+    /** */
+    private final long nextLink;
+
+    DataPagePayload(int off, int payloadSize, long nextLink) {
+        this.off = off;
+        this.payloadSize = payloadSize;
+        this.nextLink = nextLink;
+    }
+
+    public int offset() {
+        return off;
+    }
+
+    public int payloadSize() {
+        return payloadSize;
+    }
+
+    public long nextLink() {
+        return nextLink;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/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 12646ae..3aee268 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
@@ -275,7 +275,7 @@ public abstract class PageIO {
      * @param buf Buffer.
      * @param pageId Page ID.
      */
-    public void initNewPage(long buf, long pageId) {
+    public void initNewPage(long buf, long pageId, int pageSize) {
         setType(buf, getType());
         setVersion(buf, getVersion());
         setPageId(buf, pageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index fb8762e..7ef4db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@ -68,8 +68,8 @@ public class PageMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setTreeRoot(buf, 0);
         setReuseListRoot(buf, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1ed0da/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
index 72893fb..531ba1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
@@ -42,8 +42,8 @@ public class PagePartitionMetaIO extends PageMetaIO {
     );
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setSize(buf, 0);
         setUpdateCounter(buf, 0);


[07/25] ignite git commit: Merge remote-tracking branch 'origin/ignite-11810' into ignite-11810

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-11810' into ignite-11810

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java


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

Branch: refs/heads/ignite-gg-11810
Commit: 3bb4407951a9565c449383800ca1607d017c092f
Parents: 664ceac 6f1ed0d
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 23 14:20:54 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 23 14:20:54 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/pagemem/PageUtils.java   | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb44079/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index b6faeb1,d4ca73d..7ba452b
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@@ -29,18 -29,18 +29,30 @@@ public class PageUtils 
      private static final Unsafe unsafe = GridUnsafe.UNSAFE;
  
      public static byte getByte(long buf, int off) {
 -        return unsafe.getByte(buf, off);
 +        return unsafe.getByte(buf + off);
 +    }
 +
 +    public static byte[] getBytes(long buf, int off, int len) {
 +        byte[] bytes = new byte[len];
 +
 +        unsafe.copyMemory(null, buf + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
 +
 +        return bytes;
 +    }
 +
 +    public static void getBytes(long src, int srcOff, byte[] dst, int dstOff, int len) {
++        unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
+     }
+ 
+     public static byte[] getBytes(long buf, int off, int len) {
+         byte[] bytes = new byte[len];
+ 
+         unsafe.copyMemory(null, buf + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
+ 
+         return bytes;
+     }
+ 
+     public static void getBytes(long src, int srcOff, byte[] dst, int dstOff, int len) {
          unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
      }
  


[14/25] ignite git commit: test

Posted by sb...@apache.org.
test


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

Branch: refs/heads/ignite-gg-11810
Commit: c41f751bedc0e567b54a2e39b286427444d78ca8
Parents: e3420e9
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 28 16:15:19 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 28 18:54:43 2016 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryEnumObjectImpl.java   |   7 +
 .../internal/binary/BinaryObjectImpl.java       |   5 +
 .../binary/BinaryObjectOffheapImpl.java         |   5 +
 .../apache/ignite/internal/pagemem/Page.java    |   5 -
 .../ignite/internal/pagemem/PageUtils.java      |  86 +++--
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  30 +-
 .../internal/processors/cache/CacheObject.java  |   2 +
 .../processors/cache/CacheObjectAdapter.java    |  23 ++
 .../cache/CacheObjectByteArrayImpl.java         |   5 +
 .../cache/database/CacheDataRowAdapter.java     |   1 +
 .../cache/database/DataStructure.java           |  12 +-
 .../cache/database/MetadataStorage.java         |  34 +-
 .../cache/database/freelist/FreeListImpl.java   |  28 +-
 .../cache/database/freelist/PagesList.java      | 146 ++++-----
 .../cache/database/tree/BPlusTree.java          | 316 ++++++++++---------
 .../cache/database/tree/io/BPlusIO.java         | 136 ++++----
 .../cache/database/tree/io/BPlusInnerIO.java    |  44 +--
 .../cache/database/tree/io/BPlusLeafIO.java     |   8 +-
 .../cache/database/tree/io/DataPageIO.java      |  48 +--
 .../cache/database/tree/util/PageHandler.java   |  93 +++---
 .../internal/GridAffinityNoCacheSelfTest.java   |   5 +
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  16 +-
 .../IgniteIncompleteCacheObjectSelfTest.java    |   5 +
 .../database/FreeListImplSelfTest.java          | 135 ++++----
 .../query/h2/database/H2TreeIndex.java          |   3 +-
 .../query/h2/database/io/H2InnerIO.java         |  16 +-
 .../query/h2/database/io/H2LeafIO.java          |  16 +-
 .../query/h2/database/io/H2RowLinkIO.java       |   4 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   5 +
 29 files changed, 616 insertions(+), 623 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index 7f3f5e4..ef45ecd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -272,6 +272,13 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        assert valBytes != null : "Value bytes must be initialized before object is stored";
+
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), valBytes, 0);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         return CacheObjectAdapter.putValue(cacheObjectType(), buf, off, len, valBytes, 0);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index abb84cf..31dd399 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -165,6 +165,11 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), arr, start);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         return CacheObjectAdapter.putValue(cacheObjectType(), buf, off, len, arr, start);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 0ac57fd..e2e1f04 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -140,6 +140,11 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        throw new UnsupportedOperationException("TODO implement");
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
         throws IgniteCheckedException {
         throw new UnsupportedOperationException("TODO implement");

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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
index 5f9d424..eef61c8 100644
--- 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
@@ -66,11 +66,6 @@ public interface Page extends AutoCloseable {
     /**
      * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
      */
-    @Nullable public ByteBuffer tryGetForWrite();
-
-    /**
-     * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
-     */
     public long tryGetForWritePointer();
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index 6f46cd5..c74fd63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -28,89 +28,109 @@ public class PageUtils {
     /** */
     private static final Unsafe unsafe = GridUnsafe.UNSAFE;
 
-    public static byte getByte(long buf, int off) {
-        assert buf > 0 : buf;
+    /**
+     * @param addr Start address. 
+     * @param off Offset.
+     * @return Byte value from given address.
+     */
+    public static byte getByte(long addr, int off) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getByte(buf + off);
+        return unsafe.getByte(addr + off);
     }
 
-    public static byte[] getBytes(long buf, int off, int len) {
-        assert buf > 0 : buf;
+    /**
+     * @param addr Start address.
+     * @param off Offset.
+     * @param len Bytes length.
+     * @return Bytes from given address.
+     */
+    public static byte[] getBytes(long addr, int off, int len) {
+        assert addr > 0 : addr;
         assert off >= 0;
         assert len >= 0;
 
         byte[] bytes = new byte[len];
 
-        unsafe.copyMemory(null, buf + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
+        unsafe.copyMemory(null, addr + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
 
         return bytes;
     }
 
-    public static void getBytes(long src, int srcOff, byte[] dst, int dstOff, int len) {
-        assert src > 0;
+    public static void getBytes(long srcAddr, int srcOff, byte[] dst, int dstOff, int len) {
+        assert srcAddr > 0;
         assert srcOff > 0;
         assert dst != null;
         assert dstOff >= 0;
         assert len >= 0;
 
-        unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
+        unsafe.copyMemory(null, srcAddr + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
     }
 
-    public static short getShort(long buf, int off) {
-        assert buf > 0 : buf;
+    public static short getShort(long addr, int off) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getShort(buf + off);
+        return unsafe.getShort(addr + off);
     }
 
-    public static int getInt(long buf, int off) {
-        assert buf > 0 : buf;
+    public static int getInt(long addr, int off) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getInt(buf + off);
+        return unsafe.getInt(addr + off);
     }
 
-    public static long getLong(long buf, int off) {
-        assert buf > 0 : buf;
+    public static long getLong(long addr, int off) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getLong(buf + off);
+        return unsafe.getLong(addr + off);
     }
 
-    public static void putBytes(long buf, int off, byte[] bytes) {
-        assert buf > 0 : buf;
+    public static void putBytes(long addr, int off, byte[] bytes) {
+        assert addr > 0 : addr;
         assert off >= 0;
         assert bytes != null;
 
-        unsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, buf + off, bytes.length);
+        unsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, addr + off, bytes.length);
     }
 
-    public static void putByte(long buf, int off, byte v) {
-        assert buf > 0 : buf;
+    public static void putBytes(long addr, int off, byte[] bytes, int bytesOff) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert bytes != null;
+        assert bytesOff >= 0 && bytesOff < bytes.length : bytesOff;
+
+        unsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF + bytesOff, null, addr + off, bytes.length - bytesOff);
+    }
+
+    public static void putByte(long addr, int off, byte v) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putByte(buf + off, v);
+        unsafe.putByte(addr + off, v);
     }
 
-    public static void putShort(long buf, int off, short v) {
-        assert buf > 0 : buf;
+    public static void putShort(long addr, int off, short v) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putShort(buf + off, v);
+        unsafe.putShort(addr + off, v);
     }
 
-    public static void putInt(long buf, int off, int v) {
-        assert buf > 0 : buf;
+    public static void putInt(long addr, int off, int v) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putInt(buf + off, v);
+        unsafe.putInt(addr + off, v);
     }
 
-    public static void putLong(long buf, int off, long v) {
-        assert buf > 0 : buf;
+    public static void putLong(long addr, int off, long v) {
+        assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putLong(buf + off, v);
+        unsafe.putLong(addr + off, v);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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
index fd59b52..2fa1caf 100644
--- 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
@@ -47,9 +47,6 @@ public class PageNoStoreImpl implements Page {
     /** */
     private PageMemoryNoStoreImpl pageMem;
 
-    /** */
-    private final ByteBuffer buf;
-
     /** Page for memory restore */
     private final boolean noTagCheck;
 
@@ -67,8 +64,6 @@ public class PageNoStoreImpl implements Page {
         this.cacheId = cacheId;
         this.pageId = pageId;
         this.noTagCheck = noTagCheck;
-
-        buf = pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */
@@ -92,10 +87,7 @@ public class PageNoStoreImpl implements Page {
 
     /** {@inheritDoc} */
     @Override public ByteBuffer getForRead() {
-        if (pageMem.readLockPage(absPtr, PageIdUtils.tag(pageId)))
-            return reset(buf.asReadOnlyBuffer());
-
-        return null;
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
@@ -113,15 +105,7 @@ public class PageNoStoreImpl implements Page {
 
     /** {@inheritDoc} */
     @Override public ByteBuffer getForWrite() {
-        int tag =  noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
-        boolean locked = pageMem.writeLockPage(absPtr, tag);
-
-        if (!locked && !noTagCheck)
-            return null;
-
-        assert locked;
-
-        return reset(buf);
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
@@ -148,16 +132,6 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer tryGetForWrite() {
-        int tag = noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
-
-        if (pageMem.tryWriteLockPage(absPtr, tag))
-            return reset(buf);
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public void releaseWrite(boolean markDirty) {
         long updatedPageId = PageIO.getPageId(pointer());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
index 92b72ce..b5604cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
@@ -67,6 +67,8 @@ public interface CacheObject extends Message {
      */
     public boolean putValue(ByteBuffer buf) throws IgniteCheckedException;
 
+    public int putValue(long addr) throws IgniteCheckedException;
+
     /**
      * @param buf Buffer to write value to.
      * @param off Offset in source binary data.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index 1394fc2..43cb016 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@ -24,6 +24,7 @@ import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -77,6 +78,28 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        assert valBytes != null : "Value bytes must be initialized before object is stored";
+
+        return putValue(addr, cacheObjectType(), valBytes, 0);
+    }
+
+    public static int putValue(long addr, byte type, byte[] valBytes, int valOff) {
+        int off = 0;
+
+        PageUtils.putInt(addr, off, valBytes.length);
+        off += 4;
+
+        PageUtils.putByte(addr, off, type);
+        off++;
+
+        PageUtils.putBytes(addr, off, valBytes, valOff);
+        off += valBytes.length - valOff;
+
+        return off;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         assert valBytes != null : "Value bytes must be initialized before object is stored";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
index b3a4117..eee6fcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
@@ -82,6 +82,11 @@ public class CacheObjectByteArrayImpl implements CacheObject, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), val, 0);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         assert val != null : "Value is not initialized";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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 38b19a1..c989eee 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
@@ -70,6 +70,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         // Link can be 0 here.
         this.link = link;
     }
+
     /**
      * Read row from data pages.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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 ce5422f..7997e6f 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
@@ -131,7 +131,7 @@ public abstract class DataStructure implements PageLockListener {
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
     protected final long tryWriteLock(Page page) {
         return PageHandler.writeLock(page, this, true);
@@ -140,7 +140,7 @@ public abstract class DataStructure implements PageLockListener {
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
     protected final long writeLock(Page page) {
         return PageHandler.writeLock(page, this, false);
@@ -148,16 +148,16 @@ public abstract class DataStructure implements PageLockListener {
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dirty Dirty page.
      */
-    protected final void writeUnlock(Page page, long buf, boolean dirty) {
-        PageHandler.writeUnlock(page, buf, this, dirty);
+    protected final void writeUnlock(Page page, long pageAddr, boolean dirty) {
+        PageHandler.writeUnlock(page, pageAddr, this, dirty);
     }
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
     protected final long readLock(Page page) {
         return PageHandler.readLock(page, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
index b19d03d..d1d5ba9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
@@ -198,12 +198,12 @@ public class MetadataStorage implements MetaStore {
             int shift = 0;
 
             // Compare index names.
-            final byte len = PageUtils.getByte(off, shift);
+            final byte len = PageUtils.getByte(buf, off + shift);
 
             shift += BYTE_LEN;
 
             for (int i = 0; i < len && i < row.idxName.length; i++) {
-                final int cmp = Byte.compare(PageUtils.getByte(off, i + shift), row.idxName[i]);
+                final int cmp = Byte.compare(PageUtils.getByte(buf, off + i + shift), row.idxName[i]);
 
                 if (cmp != 0)
                     return cmp;
@@ -253,17 +253,19 @@ public class MetadataStorage implements MetaStore {
      */
     private static void storeRow(
         final long buf,
-        final int off,
+        int off,
         final IndexItem row
     ) {
         // Index name length.
         PageUtils.putByte(buf, off, (byte)row.idxName.length);
+        off++;
 
         // Index name.
-        PageUtils.putBytes(buf, off + 1, row.idxName);
+        PageUtils.putBytes(buf, off, row.idxName);
+        off += row.idxName.length;
 
         // Page ID.
-        PageUtils.putLong(buf, off + 1 + row.idxName.length, row.pageId);
+        PageUtils.putLong(buf, off, row.pageId);
     }
 
     /**
@@ -276,19 +278,23 @@ public class MetadataStorage implements MetaStore {
      */
     private static void storeRow(
         final long dst,
-        final int dstOff,
+        int dstOff,
         final long src,
-        final int srcOff
+        int srcOff
     ) {
         // Index name length.
         final byte len = PageUtils.getByte(src, srcOff);
+        srcOff++;
 
         PageUtils.putByte(dst, dstOff, len);
+        dstOff++;
 
-        PageHandler.copyMemory(src, srcOff + 1, dst, dstOff + 1, len);
+        PageHandler.copyMemory(src, srcOff, dst, dstOff, len);
+        srcOff += len;
+        dstOff += len;
 
         // Page ID.
-        PageUtils.putLong(dst, dstOff + 1 + len, PageUtils.getLong(src, srcOff + 1 + len));
+        PageUtils.putLong(dst, dstOff, PageUtils.getLong(src, srcOff));
     }
 
     /**
@@ -298,15 +304,17 @@ public class MetadataStorage implements MetaStore {
      * @param off Offset in buf.
      * @return Read row.
      */
-    private static IndexItem readRow(final long buf, final int off) {
+    private static IndexItem readRow(final long buf, int off) {
         // Index name length.
-        final int len = PageUtils.getByte(buf, 0) & 0xFF;
+        final int len = PageUtils.getByte(buf, off) & 0xFF;
+        off++;
 
         // Index name.
-        final byte[] idxName = PageUtils.getBytes(buf, 1, len);
+        final byte[] idxName = PageUtils.getBytes(buf, off, len);
+        off += len;
 
         // Page ID.
-        final long pageId = PageUtils.getLong(buf, off + 1 + len);
+        final long pageId = PageUtils.getLong(buf, off);
 
         return new IndexItem(idxName, pageId);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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 c40656e..5b3e781 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
@@ -73,26 +73,26 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     /** */
     private final PageHandler<CacheDataRow, Integer> writeRow =
         new PageHandler<CacheDataRow, Integer>() {
-            @Override public Integer run(Page page, PageIO iox, long buf, CacheDataRow row, int written)
+            @Override public Integer run(Page page, PageIO iox, long pageAddr, CacheDataRow row, int written)
                 throws IgniteCheckedException {
                 DataPageIO io = (DataPageIO)iox;
 
                 int rowSize = getRowSize(row);
-                int oldFreeSpace = io.getFreeSpace(buf);
+                int oldFreeSpace = io.getFreeSpace(pageAddr);
 
                 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, buf, io, row, rowSize):
-                    addRowFragment(page, buf, io, row, written, rowSize);
+                written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(page, pageAddr, io, row, rowSize):
+                    addRowFragment(page, pageAddr, io, row, written, rowSize);
 
                 // Reread free space after update.
-                int newFreeSpace = io.getFreeSpace(buf);
+                int newFreeSpace = io.getFreeSpace(pageAddr);
 
                 if (newFreeSpace > MIN_PAGE_FREE_SPACE) {
                     int bucket = bucket(newFreeSpace, false);
 
-                    put(null, page, buf, bucket);
+                    put(null, page, pageAddr, bucket);
                 }
 
                 // Avoid boxing with garbage generation for usual case.
@@ -159,7 +159,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                 int payloadSize = io.addRowFragment(buf, row, written, rowSize, pageSize());
 
-                assert payloadSize > 0: payloadSize;
+                assert payloadSize > 0 : payloadSize;
 
                 if (isWalDeltaRecordNeeded(wal, page)) {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
@@ -178,15 +178,15 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
     /** */
     private final PageHandler<Void, Long> rmvRow = new PageHandler<Void, Long>() {
-        @Override public Long run(Page page, PageIO iox, long buf, Void arg, int itemId)
+        @Override public Long run(Page page, PageIO iox, long pageAddr, Void arg, int itemId)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 
-            int oldFreeSpace = io.getFreeSpace(buf);
+            int oldFreeSpace = io.getFreeSpace(pageAddr);
 
             assert oldFreeSpace >= 0: oldFreeSpace;
 
-            long nextLink = io.removeRow(buf, itemId, pageSize());
+            long nextLink = io.removeRow(pageAddr, itemId, pageSize());
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new DataPageRemoveRecord(cacheId, page.id(), itemId));
@@ -199,7 +199,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 //                    put(null, page, buf, REUSE_BUCKET);
 //            }
 
-            int newFreeSpace = io.getFreeSpace(buf);
+            int newFreeSpace = io.getFreeSpace(pageAddr);
 
             if (newFreeSpace > MIN_PAGE_FREE_SPACE) {
                 int newBucket = bucket(newFreeSpace, false);
@@ -209,12 +209,12 @@ 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, buf, io, oldBucket))
-                            put(null, page, buf, newBucket);
+                        if (removeDataPage(page, pageAddr, io, oldBucket))
+                            put(null, page, pageAddr, newBucket);
                     }
                 }
                 else
-                    put(null, page, buf, newBucket);
+                    put(null, page, pageAddr, newBucket);
             }
 
             // For common case boxed 0L will be cached inside of Long, so no garbage will be produced.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c41f751b/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 91f2ba5..9e7f9ff 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
@@ -85,17 +85,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, long 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));
@@ -149,16 +149,16 @@ public abstract class PagesList extends DataStructure {
 
                 while (nextPageId != 0) {
                     try (Page page = page(nextPageId)) {
-                        long buf = readLock(page); // No concurrent recycling on init.
+                        long pageAddr = readLock(page); // No concurrent recycling on init.
 
-                        assert buf != 0L;
+                        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) + ']';
@@ -166,7 +166,7 @@ public abstract class PagesList extends DataStructure {
                             nextPageId = next0;
                         }
                         finally {
-                            readUnlock(page, buf);
+                            readUnlock(page, pageAddr);
                         }
                     }
                 }
@@ -198,7 +198,7 @@ public abstract class PagesList extends DataStructure {
         assert metaPageId != 0;
 
         Page curPage = null;
-        long curBuf = 0L;
+        long curPageAddr = 0L;
         PagesListMetaIO curIo = null;
 
         long nextPageId = metaPageId;
@@ -211,39 +211,39 @@ public abstract class PagesList extends DataStructure {
                     int tailIdx = 0;
 
                     while (tailIdx < tails.length) {
-                        int written = curPage != null ? curIo.addTails(pageMem.pageSize(), 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, pageSize());
+                                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;
@@ -252,22 +252,22 @@ public abstract class PagesList extends DataStructure {
             }
         }
         finally {
-            releaseAndClose(curPage, curBuf);
+            releaseAndClose(curPage, curPageAddr);
         }
 
         while (nextPageId != 0L) {
             try (Page page = page(nextPageId)) {
-                long 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);
 
-                    nextPageId = io.getNextMetaPageId(buf);
+                    nextPageId = io.getNextMetaPageId(pageAddr);
                 }
                 finally {
-                    writeUnlock(page, buf, true);
+                    writeUnlock(page, pageAddr, true);
                 }
             }
         }
@@ -462,19 +462,19 @@ public abstract class PagesList extends DataStructure {
                 long pageId = tail.tailId;
 
                 try (Page page = page(pageId)) {
-                    long 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);
                     }
                 }
             }
@@ -627,17 +627,17 @@ public abstract class PagesList extends DataStructure {
             long nextId = allocatePage(null);
 
             try (Page next = page(nextId)) {
-                long nextBuf = writeLock(next); // Newly allocated page.
+                long nextPageAddr = writeLock(next); // Newly allocated page.
 
-                assert nextBuf != 0L;
+                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, pageSize());
+                    int idx = io.addPage(nextPageAddr, dataPageId, pageSize());
 
                     // Here we should never write full page, because it is known to be new.
                     next.fullPageWalRecordPolicy(FALSE);
@@ -663,7 +663,7 @@ public abstract class PagesList extends DataStructure {
                     updateTail(bucket, pageId, nextId);
                 }
                 finally {
-                    writeUnlock(next, nextBuf, true);
+                    writeUnlock(next, nextPageAddr, true);
                 }
             }
         }
@@ -704,9 +704,9 @@ public abstract class PagesList extends DataStructure {
 
                 if (idx == -1) { // Attempt to add page failed: the node page is full.
                     try (Page next = page(nextId)) {
-                        long 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 != 0L;
+                        assert nextPageAddr != 0L;
 
                         if (locked == null) {
                             lockedBufs = new ArrayList<>(2);
@@ -714,9 +714,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));
@@ -737,7 +737,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;
                     }
@@ -780,15 +780,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.
      */
     private long writeLockPage(Page page, int bucket, int lockAttempt)
         throws IgniteCheckedException {
-        long buf = tryWriteLock(page);
+        long pageAddr = tryWriteLock(page);
 
-        if (buf != 0L)
-            return buf;
+        if (pageAddr != 0L)
+            return pageAddr;
 
         if (lockAttempt == TRY_LOCK_ATTEMPTS) {
             Stripe[] stripes = getBucket(bucket);
@@ -943,17 +943,17 @@ public abstract class PagesList extends DataStructure {
 
             long recycleId = 0L;
 
-            long buf = writeLock(page); // Explicit check.
+            long pageAddr = writeLock(page); // Explicit check.
 
-            if (buf == 0L)
+            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;
@@ -967,20 +967,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).
@@ -1040,25 +1040,25 @@ public abstract class PagesList extends DataStructure {
             try (Page next = nextId == 0L ? null : page(nextId)) {
                 boolean write = false;
 
-                long nextBuf = next == null ? 0L : writeLock(next); // Explicit check.
-                long buf = writeLock(page); // Explicit check.
+                long nextPageAddr = next == null ? 0L : writeLock(next); // Explicit check.
+                long pageAddr = writeLock(page); // Explicit check.
 
-                if (buf == 0L) {
-                    if (nextBuf != 0L) // 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 == 0L)) {
-                        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;
 
@@ -1066,13 +1066,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 != 0L)
-                        writeUnlock(next, nextBuf, write);
+                    if (nextPageAddr != 0L)
+                        writeUnlock(next, nextPageAddr, write);
 
-                    writeUnlock(page, buf, write);
+                    writeUnlock(page, pageAddr, write);
                 }
             }
         }
@@ -1142,19 +1142,19 @@ public abstract class PagesList extends DataStructure {
         long nextBuf)
         throws IgniteCheckedException {
         try (Page prev = page(prevId)) {
-            long prevBuf = writeLock(prev); // No check, we keep a reference.
+            long prevPageAddr = writeLock(prev); // No check, we keep a reference.
 
-            assert prevBuf != 0L;
+            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));
@@ -1165,7 +1165,7 @@ public abstract class PagesList extends DataStructure {
                     wal.log(new PagesListSetPreviousRecord(cacheId, nextId, prevId));
             }
             finally {
-                writeUnlock(prev, prevBuf, true);
+                writeUnlock(prev, prevPageAddr, true);
             }
         }
     }


[08/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 0a556cbb7c8c911adfc42216d440f184160cbad5
Parents: 3bb4407
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 23 15:06:37 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 23 15:06:37 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/pagemem/PageUtils.java   | 12 ------------
 1 file changed, 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0a556cbb/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index 7ba452b..b6faeb1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -44,18 +44,6 @@ public class PageUtils {
         unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
     }
 
-    public static byte[] getBytes(long buf, int off, int len) {
-        byte[] bytes = new byte[len];
-
-        unsafe.copyMemory(null, buf + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
-
-        return bytes;
-    }
-
-    public static void getBytes(long src, int srcOff, byte[] dst, int dstOff, int len) {
-        unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
-    }
-
     public static short getShort(long buf, int off) {
         return unsafe.getShort(buf + off);
     }


[06/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 664ceac5ddf2aaeb6c4848592948cf430dc5c75e
Parents: 94f22a0
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 23 13:28:01 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 23 14:20:05 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/pagemem/Page.java    |   4 +
 .../ignite/internal/pagemem/PageUtils.java      |  28 ++-
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  12 +-
 .../delta/DataPageInsertFragmentRecord.java     |   4 +-
 .../wal/record/delta/DataPageInsertRecord.java  |   4 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |   4 +-
 .../delta/DataPageSetFreeListPageRecord.java    |   2 +-
 .../wal/record/delta/FixCountRecord.java        |   2 +-
 .../record/delta/FixLeftmostChildRecord.java    |   2 +-
 .../pagemem/wal/record/delta/FixRemoveId.java   |   2 +-
 .../wal/record/delta/InitNewPageRecord.java     |   4 +-
 .../wal/record/delta/InnerReplaceRecord.java    |   2 +-
 .../pagemem/wal/record/delta/InsertRecord.java  |   2 +-
 .../pagemem/wal/record/delta/MergeRecord.java   |   2 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |   3 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |   2 +-
 .../wal/record/delta/MetaPageInitRecord.java    |   5 +-
 .../record/delta/MetaPageInitRootRecord.java    |   4 +-
 .../MetaPageUpdateCandidateAllocatedIndex.java  |   4 +-
 .../delta/MetaPageUpdateLastAllocatedIndex.java |   2 +-
 .../MetaPageUpdateLastSuccessfulBackupId.java   |   3 +-
 ...etaPageUpdateLastSuccessfulFullBackupId.java |   3 +-
 .../delta/MetaPageUpdateNextBackupId.java       |   3 +-
 .../MetaPageUpdatePartitionDataRecord.java      |   3 +-
 .../wal/record/delta/NewRootInitRecord.java     |   5 +-
 .../wal/record/delta/PageDeltaRecord.java       |   2 +-
 .../record/delta/PagesListAddPageRecord.java    |   5 +-
 .../delta/PagesListInitNewPageRecord.java       |   7 +-
 .../record/delta/PagesListRemovePageRecord.java |   3 +-
 .../record/delta/PagesListSetNextRecord.java    |   3 +-
 .../delta/PagesListSetPreviousRecord.java       |   3 +-
 .../pagemem/wal/record/delta/RecycleRecord.java |   3 +-
 .../pagemem/wal/record/delta/RemoveRecord.java  |   3 +-
 .../pagemem/wal/record/delta/ReplaceRecord.java |   2 +-
 .../record/delta/SplitExistingPageRecord.java   |   3 +-
 .../record/delta/SplitForwardPageRecord.java    |   2 +-
 .../record/delta/TrackingPageDeltaRecord.java   |   4 +-
 .../processors/cache/CacheObjectAdapter.java    |  10 +-
 .../processors/cache/IncompleteObject.java      |  13 ++
 .../cache/database/CacheDataRowAdapter.java     |  64 +++++-
 .../cache/database/DataStructure.java           |   7 +
 .../cache/database/MetadataStorage.java         | 115 ++++------
 .../cache/database/freelist/FreeListImpl.java   |  23 +-
 .../cache/database/freelist/PagesList.java      |  12 +-
 .../database/freelist/io/PagesListMetaIO.java   |   4 +-
 .../database/freelist/io/PagesListNodeIO.java   |   7 +-
 .../cache/database/tree/BPlusTree.java          |  18 +-
 .../cache/database/tree/io/BPlusIO.java         |  18 +-
 .../cache/database/tree/io/BPlusInnerIO.java    |   7 +-
 .../cache/database/tree/io/BPlusLeafIO.java     |   2 +-
 .../cache/database/tree/io/CacheVersionIO.java  |  71 ++++++
 .../cache/database/tree/io/DataPageIO.java      | 227 +++++++++++--------
 .../cache/database/tree/io/DataPagePayload.java |  50 ++++
 .../cache/database/tree/io/PageIO.java          |   2 +-
 .../cache/database/tree/io/PageMetaIO.java      |   4 +-
 .../database/tree/io/PagePartitionMetaIO.java   |   4 +-
 .../cache/database/tree/io/TrackingPageIO.java  |   2 +-
 .../cache/database/tree/util/PageHandler.java   |   2 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  11 +-
 .../database/tree/io/TrackingPageIOTest.java    |  47 ++--
 .../database/BPlusTreeReuseSelfTest.java        |  11 +-
 .../processors/database/BPlusTreeSelfTest.java  |  50 ++--
 62 files changed, 573 insertions(+), 359 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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
index 89848cc..5f9d424 100644
--- 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
@@ -95,6 +95,10 @@ public interface Page extends AutoCloseable {
      */
     public Boolean fullPageWalRecordPolicy();
 
+    public int size();
+
+    public ByteBuffer pageBuffer();
+
     /**
      * Release page.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index 69719e1..b6faeb1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -29,19 +29,31 @@ public class PageUtils {
     private static final Unsafe unsafe = GridUnsafe.UNSAFE;
 
     public static byte getByte(long buf, int off) {
-        return unsafe.getByte(buf, off);
+        return unsafe.getByte(buf + off);
+    }
+
+    public static byte[] getBytes(long buf, int off, int len) {
+        byte[] bytes = new byte[len];
+
+        unsafe.copyMemory(null, buf + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
+
+        return bytes;
+    }
+
+    public static void getBytes(long src, int srcOff, byte[] dst, int dstOff, int len) {
+        unsafe.copyMemory(null, src + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
     }
 
     public static short getShort(long buf, int off) {
-        return unsafe.getShort(buf, off);
+        return unsafe.getShort(buf + off);
     }
 
     public static int getInt(long buf, int off) {
-        return unsafe.getInt(buf, off);
+        return unsafe.getInt(buf + off);
     }
 
     public static long getLong(long buf, int off) {
-        return unsafe.getLong(buf, off);
+        return unsafe.getLong(buf + off);
     }
 
     public static void putBytes(long buf, int off, byte[] bytes) {
@@ -49,18 +61,18 @@ public class PageUtils {
     }
 
     public static void putByte(long buf, int off, byte v) {
-        unsafe.putByte(buf, off, v);
+        unsafe.putByte(buf + off, v);
     }
 
     public static void putShort(long buf, int off, short v) {
-        unsafe.putShort(buf, off, v);
+        unsafe.putShort(buf + off, v);
     }
 
     public static void putInt(long buf, int off, int v) {
-        unsafe.putInt(buf, off, v);
+        unsafe.putInt(buf + off, v);
     }
 
     public static void putLong(long buf, int off, long v) {
-        unsafe.putLong(buf, off, v);
+        unsafe.putLong(buf + off, v);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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
index 0ad206b..fd59b52 100644
--- 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
@@ -71,6 +71,11 @@ public class PageNoStoreImpl implements Page {
         buf = pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
     }
 
+    /** {@inheritDoc} */
+    @Override public ByteBuffer pageBuffer() {
+        return pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
+    }
+
     private long pointer() {
         return absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD;
     }
@@ -121,7 +126,7 @@ public class PageNoStoreImpl implements Page {
 
     /** {@inheritDoc} */
     @Override public long getForWritePointer() {
-        int tag =  noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+        int tag = noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS : PageIdUtils.tag(pageId);
         boolean locked = pageMem.writeLockPage(absPtr, tag);
 
         if (!locked && !noTagCheck)
@@ -175,6 +180,11 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
+    @Override public int size() {
+        return pageMem.pageSize();
+    }
+
+    /** {@inheritDoc} */
     @Override public void close() {
         pageMem.releasePage(this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
index 49b2626..96bf4ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
@@ -50,10 +50,10 @@ public class DataPageInsertFragmentRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
 
-        io.addRowFragment(buf, payload, lastLink);
+        io.addRowFragment(buf, payload, lastLink, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
index ceb06c7..66b07bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
@@ -51,12 +51,12 @@ public class DataPageInsertRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         assert payload != null;
 
         DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
 
-        io.addRow(buf, payload);
+        io.addRow(buf, payload, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
index 511094f..3a6023e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
@@ -48,11 +48,11 @@ public class DataPageRemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(long buf, int pageSize)
         throws IgniteCheckedException {
         DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
 
-        io.removeRow(buf, itemId);
+        io.removeRow(buf, itemId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
index 6463989..09d1442 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
@@ -49,7 +49,7 @@ public class DataPageSetFreeListPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
 
         io.setFreeListPageId(buf, freeListPage);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
index 5089c76..e95fdf1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
@@ -40,7 +40,7 @@ public class FixCountRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusIO<?> io = PageIO.getBPlusIO(buf);
 
         io.setCount(buf, cnt);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
index fe09dd1..c8a68d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
@@ -41,7 +41,7 @@ public class FixLeftmostChildRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusInnerIO<?> io = PageIO.getBPlusIO(buf);
 
         io.setLeft(buf, 0, rightId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
index 5147854..b9febc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
@@ -41,7 +41,7 @@ public class FixRemoveId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(long buf, int pageSize)
         throws IgniteCheckedException {
         BPlusIO<?> io = PageIO.getBPlusIO(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
index 5ffc193..ac4dda0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
@@ -54,10 +54,10 @@ public class InitNewPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PageIO io = PageIO.getPageIO(ioType, ioVer);
 
-        io.initNewPage(buf, newPageId);
+        io.initNewPage(buf, newPageId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
index ccc3449..a9140ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
@@ -57,7 +57,7 @@ public class InnerReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer dstBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(long dstBuf, int pageSize) throws IgniteCheckedException {
         throw new IgniteCheckedException("Inner replace record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
index f439bf3..948f829 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
@@ -72,7 +72,7 @@ public class InsertRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         io.insert(buf, idx, row, rowBytes, rightId);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
index d8113fe..c3b5934 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
@@ -63,7 +63,7 @@ public class MergeRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer leftBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(long leftBuf, int pageSize) throws IgniteCheckedException {
         throw new IgniteCheckedException("Merge record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
index a84933c..19d8140 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
@@ -40,7 +39,7 @@ public class MetaPageAddRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
         io.addRoot(0, buf, rootId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
index 1145736..a2957e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
@@ -34,7 +34,7 @@ public class MetaPageCutRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
         io.cutRoot(0, buf);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
index df671f3..fecf80d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
@@ -75,12 +74,12 @@ public class MetaPageInitRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PageMetaIO io = ioType == PageIO.T_META ?
             PageMetaIO.VERSIONS.forPage(buf) :
             PagePartitionMetaIO.VERSIONS.forPage(buf);
 
-        io.initNewPage(buf, newPageId);
+        io.initNewPage(buf, newPageId, pageSize);
 
         io.setTreeRoot(buf, treeRoot);
         io.setReuseListRoot(buf, reuseListRoot);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
index 96bc949..e67e974 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
@@ -40,8 +39,7 @@ public class MetaPageInitRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
-        throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
         io.initRoot(0, buf, rootId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
index 32286dc..e9e1792 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
@@ -21,8 +21,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
-import java.nio.ByteBuffer;
-
 /**
  *
  */
@@ -40,7 +38,7 @@ public class MetaPageUpdateCandidateAllocatedIndex extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         assert PageIO.getType(buf) == PageIO.T_META || PageIO.getType(buf) == PageIO.T_PART_META;
 
         PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(buf));

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
index 836a4eb..727ed23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
@@ -39,7 +39,7 @@ public class MetaPageUpdateLastAllocatedIndex extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         assert PageIO.getType(buf) == PageIO.T_META || PageIO.getType(buf) == PageIO.T_PART_META;
 
         PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(buf));

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
index ada247e..f9b9e8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
@@ -38,7 +37,7 @@ public class MetaPageUpdateLastSuccessfulBackupId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
 
         io.setLastSuccessfulBackupId(buf, lastSuccessfulBackupId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
index bea90c3..c63788d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
@@ -38,7 +37,7 @@ public class MetaPageUpdateLastSuccessfulFullBackupId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
 
         io.setLastSuccessfulFullBackupId(buf, lastSuccessfulFullBackupId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
index f33afc0..6cdf974 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
@@ -38,7 +37,7 @@ public class MetaPageUpdateNextBackupId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
 
         io.setNextBackupId(buf, nextBackupId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
index 494c0ad..f959761 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PagePartitionMetaIO;
 
@@ -79,7 +78,7 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(buf);
 
         io.setUpdateCounter(buf, updateCntr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
index e0284f7..756ad93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 
@@ -74,8 +73,8 @@ public class NewRootInitRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        io.initNewRoot(buf, newRootId, leftChildId, row, rowBytes, rightChildId);
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
+        io.initNewRoot(buf, newRootId, leftChildId, row, rowBytes, rightChildId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
index 9acdd08..80668a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
@@ -65,7 +65,7 @@ public abstract class PageDeltaRecord extends WALRecord {
      * @param buf Page buffer.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void applyDelta(ByteBuffer buf) throws IgniteCheckedException;
+    public abstract void applyDelta(long buf, int pageSize) throws IgniteCheckedException;
 
     /** {@inheritDoc} */
     @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
index 7ea16b8..208de66 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -51,10 +50,10 @@ public class PagesListAddPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
 
-        int cnt = io.addPage(buf, dataPageId);
+        int cnt = io.addPage(buf, dataPageId, pageSize);
 
         assert cnt >= 0 : cnt;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
index 922f212..c3ac7ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -73,14 +72,14 @@ public class PagesListInitNewPageRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PagesListNodeIO io = PageIO.getPageIO(PageIO.T_PAGE_LIST_NODE, ioVer);
 
-        io.initNewPage(buf, pageId());
+        io.initNewPage(buf, pageId(), pageSize);
         io.setPreviousId(buf, prevPageId);
 
         if (addDataPageId != 0L) {
-            int cnt = io.addPage(buf, addDataPageId);
+            int cnt = io.addPage(buf, addDataPageId, pageSize);
 
             assert cnt == 0 : cnt;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
index 128e396..95c0d47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -51,7 +50,7 @@ public class PagesListRemovePageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
 
         boolean rmvd = io.removePage(buf, rmvdPageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
index 07ce6d2..801d679 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 
@@ -47,7 +46,7 @@ public class PagesListSetNextRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
 
         io.setNextId(buf, nextPageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
index bb3a877..bfac436 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 
@@ -47,7 +46,7 @@ public class PagesListSetPreviousRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
 
         io.setPreviousId(buf, prevPageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
index 470f441..c40f014 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
@@ -40,7 +39,7 @@ public class RecycleRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         PageIO.setPageId(buf, newPageId);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
index 453525c..c9c8ea3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -47,7 +46,7 @@ public class RemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusIO<?> io = PageIO.getBPlusIO(buf);
 
         if (io.getCount(buf) != cnt)

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
index 8a9e4e3..4d65121 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
@@ -55,7 +55,7 @@ public class ReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(long buf, int pageSize)
         throws IgniteCheckedException {
         if (io.getCount(buf) < idx)
             throw new DeltaApplicationException("Index is greater than count: " + idx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
index 8ec5f8f..f9dfd37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -50,7 +49,7 @@ public class SplitExistingPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusIO<?> io = PageIO.getBPlusIO(buf);
 
         io.splitExistingPage(buf, mid, fwdId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
index b4487fa..f67e8d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
@@ -79,7 +79,7 @@ public class SplitForwardPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer fwdBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(long fwdBuf, int pageSize) throws IgniteCheckedException {
         throw new IgniteCheckedException("Split forward page record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
index 03953df..e95afbb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
@@ -70,8 +70,8 @@ public class TrackingPageDeltaRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        TrackingPageIO.VERSIONS.forPage(buf).markChanged(buf, pageIdToMark, nextBackupId, lastSuccessfulBackupId, buf.capacity());
+    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
+        TrackingPageIO.VERSIONS.forPage(buf).markChanged(buf, pageIdToMark, nextBackupId, lastSuccessfulBackupId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index a34e98d..1394fc2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@ -164,8 +164,14 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
      * @return {@code True} if data were successfully written.
      * @throws IgniteCheckedException If failed.
      */
-    public static boolean putValue(byte cacheObjType, final ByteBuffer buf, int off, int len,
-        byte[] valBytes, final int start) throws IgniteCheckedException {
+    public static boolean putValue(byte cacheObjType,
+        final ByteBuffer buf,
+        int off,
+        int len,
+        byte[] valBytes,
+        final int start)
+        throws IgniteCheckedException
+    {
         int dataLen = valBytes.length;
 
         if (buf.remaining() < len)

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
index 666fc27..ce0e306 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IncompleteObject.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
  * Incomplete object.
@@ -86,4 +87,16 @@ public class IncompleteObject<T> {
 
         off += len;
     }
+
+    public int readData(long buf, int remaining) {
+        assert data != null;
+
+        final int len = Math.min(data.length - off, remaining);
+
+        PageUtils.getBytes(buf, 0, data, off, len);
+
+        off += len;
+
+        return len;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 b5babc4..59e2fc4 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
@@ -21,6 +21,7 @@ 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.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -29,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.IncompleteObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -68,7 +70,6 @@ public class CacheDataRowAdapter implements CacheDataRow {
         // Link can be 0 here.
         this.link = link;
     }
-
     /**
      * Read row from data pages.
      *
@@ -89,19 +90,21 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
         do {
             try (Page page = page(pageId(nextLink), cctx)) {
-                ByteBuffer buf = page.getForRead(); // Non-empty data page must not be recycled.
+                long buf = page.getForReadPointer(); // Non-empty data page must not be recycled.
 
-                assert buf != null: nextLink;
+                assert buf != 0L : nextLink;
 
                 try {
                     DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
 
-                    nextLink = io.setPositionAndLimitOnPayload(buf, itemId(nextLink));
+                    DataPagePayload data = io.readPayload(buf, itemId(nextLink), page.size());
+
+                    nextLink = data.nextLink();
 
                     if (first) {
                         if (nextLink == 0) {
                             // Fast path for a single page row.
-                            readFullRow(coctx, buf, keyOnly);
+                            readFullRow(coctx, data.offset(), keyOnly);
 
                             return;
                         }
@@ -109,7 +112,12 @@ public class CacheDataRowAdapter implements CacheDataRow {
                         first = false;
                     }
 
-                    incomplete = readFragment(coctx, buf, keyOnly, incomplete);
+                    ByteBuffer buf0 = page.pageBuffer();
+
+                    buf0.position(data.offset());
+                    buf0.limit(data.offset() + data.payloadSize());
+
+                    incomplete = readFragment(coctx, buf0, keyOnly, incomplete);
 
                     if (keyOnly && key != null)
                         return;
@@ -121,7 +129,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         }
         while(nextLink != 0);
 
-        assert isReady(): "ready";
+        assert isReady() : "ready";
     }
 
     /**
@@ -179,8 +187,23 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param keyOnly {@code true} If need to read only key object.
      * @throws IgniteCheckedException If failed.
      */
-    private void readFullRow(CacheObjectContext coctx, ByteBuffer buf, boolean keyOnly) throws IgniteCheckedException {
-        key = coctx.processor().toKeyCacheObject(coctx, buf);
+    private void readFullRow(CacheObjectContext coctx, long buf, boolean keyOnly) throws IgniteCheckedException {
+        int off = 0;
+
+        int len = PageUtils.getInt(buf, off);
+        off += 4;
+
+        if (len == 0)
+            key = null;
+        else {
+            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            off += len;
+
+            byte type = PageUtils.getByte(buf, off);
+            off++;
+
+            key = coctx.processor().toKeyCacheObject(coctx, type, bytes);
+        }
 
         if (keyOnly) {
             assert key != null: "key";
@@ -188,9 +211,26 @@ public class CacheDataRowAdapter implements CacheDataRow {
             return;
         }
 
-        val = coctx.processor().toCacheObject(coctx, buf);
-        ver = CacheVersionIO.read(buf, false);
-        expireTime = buf.getLong();
+        len = PageUtils.getInt(buf, off);
+        off += 4;
+
+        if (len == 0)
+            val = null;
+        else {
+            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            off += len;
+
+            byte type = PageUtils.getByte(buf, off);
+            off++;
+
+            val = coctx.processor().toCacheObject(coctx, type, bytes);
+        }
+
+        ver = CacheVersionIO.read(buf + off, false);
+
+        off += CacheVersionIO.size(ver, false);
+
+        expireTime = PageUtils.getLong(buf, off);
 
         assert isReady(): "ready";
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 26df029..ce5422f 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
@@ -171,6 +171,13 @@ public abstract class DataStructure implements PageLockListener {
         PageHandler.readUnlock(page, buf, this);
     }
 
+    /**
+     * @return Page size.
+     */
+    protected final int pageSize() {
+        return pageMem.pageSize();
+    }
+
     /** {@inheritDoc} */
     @Override public void onBeforeWriteLock(Page page) {
         // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
index 26151ac..b19d03d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.database;
 
-import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 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;
 
 /**
@@ -190,19 +191,19 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(final BPlusIO<IndexItem> io, final ByteBuffer buf, final int idx,
+        @Override protected int compare(final BPlusIO<IndexItem> io, final long buf, final int idx,
             final IndexItem row) throws IgniteCheckedException {
             final int off = ((IndexIO)io).getOffset(idx);
 
             int shift = 0;
 
             // Compare index names.
-            final byte len = buf.get(off + shift);
+            final byte len = PageUtils.getByte(off, shift);
 
             shift += BYTE_LEN;
 
             for (int i = 0; i < len && i < row.idxName.length; i++) {
-                final int cmp = Byte.compare(buf.get(off + i + shift), row.idxName[i]);
+                final int cmp = Byte.compare(PageUtils.getByte(off, i + shift), row.idxName[i]);
 
                 if (cmp != 0)
                     return cmp;
@@ -212,7 +213,7 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final ByteBuffer buf,
+        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final long buf,
             final int idx) throws IgniteCheckedException {
             return readRow(buf, ((IndexIO)io).getOffset(idx));
         }
@@ -251,27 +252,18 @@ public class MetadataStorage implements MetaStore {
      * @param row Row to store.
      */
     private static void storeRow(
-        final ByteBuffer buf,
+        final long buf,
         final int off,
         final IndexItem row
     ) {
-        int origPos = buf.position();
+        // Index name length.
+        PageUtils.putByte(buf, off, (byte)row.idxName.length);
 
-        try {
-            buf.position(off);
-
-            // Index name length.
-            buf.put((byte)row.idxName.length);
-
-            // Index name.
-            buf.put(row.idxName);
+        // Index name.
+        PageUtils.putBytes(buf, off + 1, row.idxName);
 
-            // Page ID.
-            buf.putLong(row.pageId);
-        }
-        finally {
-            buf.position(origPos);
-        }
+        // Page ID.
+        PageUtils.putLong(buf, off + 1 + row.idxName.length, row.pageId);
     }
 
     /**
@@ -283,39 +275,20 @@ public class MetadataStorage implements MetaStore {
      * @param srcOff Src buf offset.
      */
     private static void storeRow(
-        final ByteBuffer dst,
+        final long dst,
         final int dstOff,
-        final ByteBuffer src,
+        final long src,
         final int srcOff
     ) {
-        int srcOrigPos = src.position();
-        int dstOrigPos = dst.position();
-
-        try {
-            src.position(srcOff);
-            dst.position(dstOff);
-
-            // Index name length.
-            final byte len = src.get();
-
-            dst.put(len);
-
-            int lim = src.limit();
+        // Index name length.
+        final byte len = PageUtils.getByte(src, srcOff);
 
-            src.limit(src.position() + len);
+        PageUtils.putByte(dst, dstOff, len);
 
-            // Index name.
-            dst.put(src);
+        PageHandler.copyMemory(src, srcOff + 1, dst, dstOff + 1, len);
 
-            src.limit(lim);
-
-            // Page ID.
-            dst.putLong(src.getLong());
-        }
-        finally {
-            src.position(srcOrigPos);
-            dst.position(dstOrigPos);
-        }
+        // Page ID.
+        PageUtils.putLong(dst, dstOff + 1 + len, PageUtils.getLong(src, srcOff + 1 + len));
     }
 
     /**
@@ -325,28 +298,17 @@ public class MetadataStorage implements MetaStore {
      * @param off Offset in buf.
      * @return Read row.
      */
-    private static IndexItem readRow(final ByteBuffer buf, final int off) {
-        int origOff = buf.position();
-
-        try {
-            buf.position(off);
+    private static IndexItem readRow(final long buf, final int off) {
+        // Index name length.
+        final int len = PageUtils.getByte(buf, 0) & 0xFF;
 
-            // Index name length.
-            final int len = buf.get() & 0xFF;
+        // Index name.
+        final byte[] idxName = PageUtils.getBytes(buf, 1, len);
 
-            // Index name.
-            final byte[] idxName = new byte[len];
+        // Page ID.
+        final long pageId = PageUtils.getLong(buf, off + 1 + len);
 
-            buf.get(idxName);
-
-            // Page ID.
-            final long pageId = buf.getLong();
-
-            return new IndexItem(idxName, pageId);
-        }
-        finally {
-            buf.position(origOff);
-        }
+        return new IndexItem(idxName, pageId);
     }
 
     /**
@@ -378,19 +340,19 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, IndexItem row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
             storeRow(buf, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void store(final long dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
+            final long src,
             final int srcIdx) throws IgniteCheckedException {
             storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final long buf,
             final int idx) throws IgniteCheckedException {
             return readRow(buf, offset(idx));
         }
@@ -419,19 +381,22 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, IndexItem row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
             storeRow(buf, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void store(final long dst,
+            final int dstIdx,
+            final BPlusIO<IndexItem> srcIo,
+            final long src,
             final int srcIdx) throws IgniteCheckedException {
             storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree,
+            final long buf,
             final int idx) throws IgniteCheckedException {
             return readRow(buf, offset(idx));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 1d2524d..c40656e 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
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist;
 
-import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 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.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertFragmentRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertRecord;
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageRemoveRecord;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 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;
@@ -114,19 +115,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 CacheDataRow row,
                 int rowSize
             ) throws IgniteCheckedException {
-                // TODO: context parameter.
-                io.addRow(buf, row, rowSize);
+                io.addRow(buf, row, rowSize, pageSize());
 
                 if (isWalDeltaRecordNeeded(wal, page)) {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[rowSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
 
-                    assert buf.remaining() == rowSize;
+                    assert data.payloadSize() == rowSize;
 
-                    buf.get(payload);
-                    buf.position(0);
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, rowSize);
 
                     wal.log(new DataPageInsertRecord(
                         cacheId,
@@ -158,7 +157,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 // Read last link before the fragment write, because it will be updated there.
                 long lastLink = row.link();
 
-                int payloadSize = io.addRowFragment(buf, row, written, rowSize);
+                int payloadSize = io.addRowFragment(buf, row, written, rowSize, pageSize());
 
                 assert payloadSize > 0: payloadSize;
 
@@ -166,9 +165,9 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[payloadSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
-                    buf.get(payload);
-                    buf.position(0);
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
+
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, payloadSize);
 
                     wal.log(new DataPageInsertFragmentRecord(cacheId, page.id(), payload, lastLink));
                 }
@@ -187,7 +186,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
             assert oldFreeSpace >= 0: oldFreeSpace;
 
-            long nextLink = io.removeRow(buf, itemId);
+            long nextLink = io.removeRow(buf, itemId, pageSize());
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new DataPageRemoveRecord(cacheId, page.id(), itemId));

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 8311b65..91f2ba5 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
@@ -229,7 +229,7 @@ public abstract class PagesList extends DataStructure {
 
                                 curIo = PagesListMetaIO.VERSIONS.latest();
 
-                                curIo.initNewPage(curBuf, nextPageId);
+                                curIo.initNewPage(curBuf, nextPageId, pageSize());
                             }
                             else {
                                 releaseAndClose(curPage, curBuf);
@@ -321,7 +321,7 @@ public abstract class PagesList extends DataStructure {
     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);
@@ -558,7 +558,7 @@ public abstract class PagesList extends DataStructure {
 
         long dataPageId = dataPage.id();
 
-        int idx = io.addPage(pageMem.pageSize(), buf, dataPageId);
+        int idx = io.addPage(buf, dataPageId, pageSize());
 
         if (idx == -1)
             handlePageFull(pageId, page, buf, io, dataPage, dataPageBuf, bucket);
@@ -637,7 +637,7 @@ public abstract class PagesList extends DataStructure {
                     if (isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PagesListSetNextRecord(cacheId, pageId, nextId));
 
-                    int idx = io.addPage(pageMem.pageSize(), nextBuf, dataPageId);
+                    int idx = io.addPage(nextBuf, dataPageId, pageSize());
 
                     // Here we should never write full page, because it is known to be new.
                     next.fullPageWalRecordPolicy(FALSE);
@@ -700,7 +700,7 @@ public abstract class PagesList extends DataStructure {
 
         try {
             while ((nextId = bag.pollFreePage()) != 0L) {
-                int idx = io.addPage(pageMem.pageSize(), 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)) {
@@ -884,7 +884,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(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 6ac8cde..67e2667 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
@@ -53,8 +53,8 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setCount(buf, 0);
         setNextMetaPageId(buf, 0L);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 c92e4a9..ad48bc1 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
@@ -52,8 +52,8 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long buf, long pageId, int pageSize) {
+        super.initNewPage(buf, pageId, pageSize);
 
         setEmpty(buf);
 
@@ -163,9 +163,10 @@ public class PagesListNodeIO extends PageIO {
      *
      * @param buf Page buffer.
      * @param pageId Page ID.
+     * @param pageSize Page size.
      * @return Total number of items in this page.
      */
-    public int addPage(int pageSize, long buf, long pageId) {
+    public int addPage(long buf, long pageId, int pageSize) {
         int cnt = getCount(buf);
 
         if (cnt == getCapacity(pageSize))

http://git-wip-us.apache.org/repos/asf/ignite/blob/664ceac5/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 b8a1004..6eb7fe1 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
@@ -408,7 +408,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             // 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(pageMem.pageSize(), buf)))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(buf, 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);
@@ -527,7 +527,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getRootLevel(buf); // Can drop only root.
 
-            io.cutRoot(pageMem.pageSize(), buf);
+            io.cutRoot(pageSize(), buf);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
@@ -547,7 +547,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getLevelsCount(buf);
 
-            io.addRoot(pageMem.pageSize(), buf, rootPageId);
+            io.addRoot(pageSize(), buf, rootPageId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
@@ -565,7 +565,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            io.initRoot(pageMem.pageSize(), buf, rootId);
+            io.initRoot(pageSize(), buf, rootId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageInitRootRecord(cacheId, meta.id(), rootId));
@@ -1721,7 +1721,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         // Update forward page.
-        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt);
+        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt, pageSize());
 
         // TODO GG-11640 log a correct forward page record.
         fwd.fullPageWalRecordPolicy(Boolean.TRUE);
@@ -2184,7 +2184,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          */
         private L insert(Page page, BPlusIO<L> io, long buf, int idx, int lvl)
             throws IgniteCheckedException {
-            int maxCnt = io.getMaxCount(pageMem.pageSize(), buf);
+            int maxCnt = io.getMaxCount(buf, pageSize());
             int cnt = io.getCount(buf);
 
             if (cnt == maxCnt) // Need to split page.
@@ -2284,7 +2284,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                                 long pageId = PageIO.getPageId(buf);
 
-                                inner(io).initNewRoot(newRootBuf, newRootId, pageId, moveUpRow, null, fwdId);
+                                inner(io).initNewRoot(newRootBuf, newRootId, pageId, moveUpRow, null, fwdId, pageSize());
 
                                 if (needWalDeltaRecord(newRoot))
                                     wal.log(new NewRootInitRecord<>(cacheId, newRoot.id(), newRootId,
@@ -2624,7 +2624,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                         // Exit: we are done.
                     }
                     else if (tail.sibling != null &&
-                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(pageMem.pageSize(), tail.buf)) {
+                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(tail.buf, pageSize())) {
                         // Release everything lower than tail, we've already merged this path.
                         doReleaseTail(tail.down);
                         tail.down = null;
@@ -2939,7 +2939,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             boolean emptyBranch = needMergeEmptyBranch == TRUE || needMergeEmptyBranch == READY;
 
-            if (!left.io.merge(pageMem.pageSize(), prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch))
+            if (!left.io.merge(prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch, pageSize()))
                 return false;
 
             // Invalidate indexes after successful merge.


[25/25] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-3477' into ignite-gg-11810

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-3477' into ignite-gg-11810

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
#	modules/yardstick/config/ignite-base-config.xml


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

Branch: refs/heads/ignite-gg-11810
Commit: b81c30acb3f9be9ec78c287cae4be548fefbdb4f
Parents: 75a9a07 3443c4e
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jan 11 16:20:50 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jan 11 16:20:50 2017 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     |   2 +-
 .../internal/managers/GridManagerAdapter.java   |   3 +-
 .../ignite/internal/pagemem/FullPageId.java     |  29 +++-
 .../cache/DynamicCacheDescriptor.java           |  17 +++
 .../processors/cache/GridCacheAdapter.java      |   7 +-
 .../cache/GridCacheConcurrentMapImpl.java       |   7 +-
 .../processors/cache/GridCacheMapEntry.java     |  60 ++++++---
 .../GridCachePartitionExchangeManager.java      | 131 +++++++++---------
 .../processors/cache/GridCacheProcessor.java    |  45 +++++--
 .../cache/GridCacheSharedContext.java           |  23 +++-
 .../cache/GridCacheSharedTtlCleanupManager.java |   5 +-
 .../processors/cache/GridCacheTtlManager.java   |  13 +-
 .../GridChangeGlobalStateMessageResponse.java   |   2 +-
 .../cache/IgniteCacheOffheapManager.java        |   8 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  44 ++++--
 .../IgniteCacheDatabaseSharedManager.java       |  16 ++-
 .../cache/database/tree/BPlusTree.java          |   4 +-
 .../GridDistributedTxRemoteAdapter.java         |  14 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  57 ++++++--
 .../dht/GridDhtPartitionTopologyImpl.java       | 135 ++++++++++---------
 .../distributed/dht/GridDhtTxFinishFuture.java  |   7 +-
 .../dht/GridPartitionedGetFuture.java           |   4 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  60 ++++-----
 .../dht/preloader/GridDhtForceKeysFuture.java   |  10 +-
 .../GridDhtPartitionsExchangeFuture.java        |  81 ++++++-----
 .../dht/preloader/GridDhtPreloader.java         |  29 ++--
 .../extras/GridCacheAttributesEntryExtras.java  |   4 +-
 .../GridCacheAttributesMvccEntryExtras.java     |   2 +-
 ...dCacheAttributesMvccObsoleteEntryExtras.java |   2 +-
 ...cheAttributesMvccObsoleteTtlEntryExtras.java |   4 +-
 .../GridCacheAttributesMvccTtlEntryExtras.java  |   4 +-
 .../GridCacheAttributesObsoleteEntryExtras.java |   2 +-
 ...idCacheAttributesObsoleteTtlEntryExtras.java |   4 +-
 .../GridCacheAttributesTtlEntryExtras.java      |   4 +-
 .../GridCacheMvccObsoleteEntryExtras.java       |   2 +-
 .../GridCacheMvccObsoleteTtlEntryExtras.java    |   4 +-
 .../extras/GridCacheMvccTtlEntryExtras.java     |   3 +-
 .../extras/GridCacheObsoleteTtlEntryExtras.java |   3 +-
 .../cache/extras/GridCacheTtlEntryExtras.java   |   2 +-
 .../processors/cache/local/GridLocalCache.java  |   7 +-
 .../continuous/CacheContinuousQueryHandler.java |  20 +--
 .../cluster/GridClusterStateProcessor.java      |   9 +-
 .../util/future/IgniteRemoteMapTask.java        |  17 +++
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  16 ++-
 .../GridCommunicationSendMessageSelfTest.java   |   7 +-
 .../cache/CacheClientStoreSelfTest.java         |   6 +-
 .../cache/CacheConfigurationLeakTest.java       |  19 ++-
 .../cache/CacheEntryProcessorCopySelfTest.java  |   8 --
 ...idAbstractCacheInterceptorRebalanceTest.java |   2 +
 .../cache/GridCacheAbstractMetricsSelfTest.java |  28 ++--
 .../cache/GridCacheAbstractTxReadTest.java      |   2 -
 .../cache/GridCacheBasicStoreAbstractTest.java  |   7 +-
 .../GridCacheOffHeapTieredAbstractSelfTest.java |   4 +
 .../cache/GridCacheP2PUndeploySelfTest.java     |   5 -
 .../GridCacheValueBytesPreloadingSelfTest.java  |  17 +--
 ...idCacheValueConsistencyAbstractSelfTest.java |  30 -----
 .../IgniteCacheConfigVariationsFullApiTest.java |  35 +++--
 .../IgniteCacheInterceptorSelfTestSuite.java    |  10 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |   6 +
 .../IgniteCacheReadThroughEvictionSelfTest.java |   5 +-
 .../GridDataStreamerImplSelfTest.java           |   8 +-
 .../database/tree/io/TrackingPageIOTest.java    |  17 +++
 ...CacheLoadingConcurrentGridStartSelfTest.java |  22 ++-
 ...tractDistributedByteArrayValuesSelfTest.java |   3 +
 ...tractPartitionedByteArrayValuesSelfTest.java |   1 +
 ...GridCachePreloadRestartAbstractSelfTest.java |  12 +-
 .../IgniteTxCachePrimarySyncTest.java           |   7 +
 ...teSynchronizationModesMultithreadedTest.java |   5 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   3 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |  13 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |  16 +--
 .../near/GridCacheNearReadersSelfTest.java      |   3 +-
 .../near/NoneRebalanceModeSelfTest.java         |   4 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |  25 ++--
 ...maryWriteOrderWithStoreExpiryPolicyTest.java |   2 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |  21 ++-
 ...cheLocalBasicStoreMultithreadedSelfTest.java |   1 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |   4 +-
 .../CacheKeepBinaryIterationTest.java           |   3 +
 .../GridCacheContinuousQueryConcurrentTest.java |  47 ++++---
 ...teCacheContinuousQueryNoUnsubscribeTest.java |   2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |  12 +-
 .../testsuites/IgniteCacheTestSuite2.java       |  16 ++-
 .../testsuites/IgniteCacheTestSuite3.java       |  13 +-
 .../testsuites/IgniteCacheTestSuite5.java       |   4 +-
 ...niteCacheLockPartitionOnAffinityRunTest.java |  36 +++--
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |  10 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |   3 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   5 +-
 modules/yardstick/config/ignite-base-config.xml |  13 ++
 .../yardstick/cache/IgniteGetAllBenchmark.java  |  42 ++++++
 .../yardstick/cache/IgniteGetBenchmark.java     |  38 +++++-
 .../cache/jdbc/JdbcAbstractBenchmark.java       |  17 +++
 .../yardstick/cache/jdbc/JdbcPutBenchmark.java  |  17 +++
 .../cache/jdbc/JdbcPutGetBenchmark.java         |  17 +++
 .../jdbc/JdbcPutIndexedValue8Benchmark.java     |  17 +++
 .../cache/jdbc/JdbcSqlQueryBenchmark.java       |  17 +++
 .../cache/jdbc/JdbcSqlQueryJoinBenchmark.java   |  17 +++
 100 files changed, 1074 insertions(+), 558 deletions(-)
----------------------------------------------------------------------


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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/b81c30ac/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 56bebbc,28fcfd8..c136799
--- 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
@@@ -3569,10 -3558,10 +3569,10 @@@ public abstract class BPlusTree<L, T ex
           * @throws IgniteCheckedException If failed.
           */
          @SuppressWarnings("unchecked")
 -        private boolean fillFromBuffer(ByteBuffer buf, BPlusIO<L> io, int startIdx, int cnt)
 +        private boolean fillFromBuffer(long buf, BPlusIO<L> io, int startIdx, int cnt)
              throws IgniteCheckedException {
-             assert io.isLeaf();
-             assert cnt != 0: cnt; // We can not see empty pages (empty tree handled in init).
+             assert io.isLeaf() : io;
+             assert cnt != 0 : cnt; // We can not see empty pages (empty tree handled in init).
              assert startIdx >= 0 : startIdx;
              assert cnt >= startIdx;
  

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/b81c30ac/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
----------------------------------------------------------------------


[02/25] ignite git commit: ignite-gg-11810

Posted by sb...@apache.org.
ignite-gg-11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 8fcac6f29c0fdad1b6af4acad232a5b24a71bbb7
Parents: 53e2cd2
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 21 17:34:38 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 21 18:25:19 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/pagemem/Page.java    |  15 +++
 .../ignite/internal/pagemem/PageMemory.java     |   2 +-
 .../ignite/internal/pagemem/PageUtils.java      |  66 ++++++++++++
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  40 +++++++-
 .../wal/record/delta/MetaPageAddRootRecord.java |   2 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |   2 +-
 .../record/delta/MetaPageInitRootRecord.java    |   2 +-
 .../cache/database/DataStructure.java           |  18 ++--
 .../cache/database/freelist/FreeListImpl.java   |   4 +-
 .../cache/database/freelist/PagesList.java      | 100 +++++++++----------
 .../database/freelist/io/PagesListMetaIO.java   |  41 ++++----
 .../database/freelist/io/PagesListNodeIO.java   |  56 +++++------
 .../cache/database/tree/BPlusTree.java          |  14 +--
 .../cache/database/tree/io/BPlusIO.java         |  73 ++++++--------
 .../cache/database/tree/io/BPlusInnerIO.java    |  32 +++---
 .../cache/database/tree/io/BPlusLeafIO.java     |   6 +-
 .../cache/database/tree/io/BPlusMetaIO.java     |  38 +++----
 .../cache/database/tree/io/CacheVersionIO.java  |  32 +++---
 .../cache/database/tree/io/DataPageIO.java      |  16 +--
 .../cache/database/tree/io/IOVersions.java      |   4 +-
 .../cache/database/tree/io/PageIO.java          |  50 +++++-----
 .../cache/database/tree/io/PageMetaIO.java      |  61 ++++++-----
 .../database/tree/io/PagePartitionMetaIO.java   |  39 ++++----
 .../cache/database/tree/io/TrackingPageIO.java  |  49 ++++-----
 .../cache/database/tree/util/PageHandler.java   |  79 +++++++--------
 .../database/tree/util/PageLockListener.java    |   8 +-
 .../apache/ignite/internal/util/GridUnsafe.java |   2 +-
 27 files changed, 480 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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
index e08fad6..89848cc 100644
--- 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
@@ -44,6 +44,11 @@ public interface Page extends AutoCloseable {
     public ByteBuffer getForRead();
 
     /**
+     * @return Pointer for modifying the page.
+     */
+    public long getForReadPointer();
+
+    /**
      * Releases reserved page. Released page can be evicted from RAM after flushing modifications to disk.
      */
     public void releaseRead();
@@ -54,11 +59,21 @@ public interface Page extends AutoCloseable {
     public ByteBuffer getForWrite();
 
     /**
+     * @return ByteBuffer for modifying the page.
+     */
+    public long getForWritePointer();
+
+    /**
      * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
      */
     @Nullable public ByteBuffer tryGetForWrite();
 
     /**
+     * @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);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 53b37f6..27c090e 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
@@ -25,7 +25,7 @@ import org.apache.ignite.lifecycle.LifecycleAware;
  */
 public interface PageMemory extends LifecycleAware, PageIdAllocator {
     /** */
-    ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+    public ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
 
     /**
      * Gets the page associated with the given page ID. Each page obtained with this method must be released by

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
new file mode 100644
index 0000000..69719e1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -0,0 +1,66 @@
+/*
+ * 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.internal.util.GridUnsafe;
+import sun.misc.Unsafe;
+
+/**
+ *
+ */
+@SuppressWarnings("deprecation")
+public class PageUtils {
+    /** */
+    private static final Unsafe unsafe = GridUnsafe.UNSAFE;
+
+    public static byte getByte(long buf, int off) {
+        return unsafe.getByte(buf, off);
+    }
+
+    public static short getShort(long buf, int off) {
+        return unsafe.getShort(buf, off);
+    }
+
+    public static int getInt(long buf, int off) {
+        return unsafe.getInt(buf, off);
+    }
+
+    public static long getLong(long buf, int off) {
+        return unsafe.getLong(buf, off);
+    }
+
+    public static void putBytes(long buf, int off, byte[] bytes) {
+        unsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, buf + off, bytes.length);
+    }
+
+    public static void putByte(long buf, int off, byte v) {
+        unsafe.putByte(buf, off, v);
+    }
+
+    public static void putShort(long buf, int off, short v) {
+        unsafe.putShort(buf, off, v);
+    }
+
+    public static void putInt(long buf, int off, int v) {
+        unsafe.putInt(buf, off, v);
+    }
+
+    public static void putLong(long buf, int off, long v) {
+        unsafe.putLong(buf, off, v);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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
index 404c0b2..0ad206b 100644
--- 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
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 
 import java.nio.ByteBuffer;
+import org.jetbrains.annotations.Nullable;
 
 /**
  *
@@ -70,6 +71,10 @@ public class PageNoStoreImpl implements Page {
         buf = pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
     }
 
+    private long pointer() {
+        return absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD;
+    }
+
     /** {@inheritDoc} */
     @Override public long id() {
         return pageId;
@@ -89,6 +94,14 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
+    @Override public long getForReadPointer() {
+        if (pageMem.readLockPage(absPtr, PageIdUtils.tag(pageId)))
+            return pointer();
+
+        return 0L;
+    }
+
+    /** {@inheritDoc} */
     @Override public void releaseRead() {
         pageMem.readUnlockPage(absPtr);
     }
@@ -107,8 +120,31 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer tryGetForWrite() {
+    @Override public long getForWritePointer() {
         int tag =  noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+        boolean locked = pageMem.writeLockPage(absPtr, tag);
+
+        if (!locked && !noTagCheck)
+            return 0L;
+
+        assert locked;
+
+        return pointer();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long tryGetForWritePointer() {
+        int tag = noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+
+        if (pageMem.tryWriteLockPage(absPtr, tag))
+            return pointer();
+
+        return 0L;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer tryGetForWrite() {
+        int tag = noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
 
         if (pageMem.tryWriteLockPage(absPtr, tag))
             return reset(buf);
@@ -118,7 +154,7 @@ public class PageNoStoreImpl implements Page {
 
     /** {@inheritDoc} */
     @Override public void releaseWrite(boolean markDirty) {
-        long updatedPageId = PageIO.getPageId(buf);
+        long updatedPageId = PageIO.getPageId(pointer());
 
         pageMem.writeUnlockPage(absPtr, PageIdUtils.tag(updatedPageId));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
index 75aaca4..a84933c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
@@ -43,7 +43,7 @@ public class MetaPageAddRootRecord extends PageDeltaRecord {
     @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
-        io.addRoot(buf, rootId);
+        io.addRoot(0, buf, rootId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
index ba1d04d..1145736 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
@@ -37,7 +37,7 @@ public class MetaPageCutRootRecord extends PageDeltaRecord {
     @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
-        io.cutRoot(buf);
+        io.cutRoot(0, buf);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
index a9d2677..96bc949 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
@@ -44,7 +44,7 @@ public class MetaPageInitRootRecord extends PageDeltaRecord {
         throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
-        io.initRoot(buf, rootId);
+        io.initRoot(0, buf, rootId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 5fd64b0..26df029 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
@@ -133,7 +133,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param page Page.
      * @return Buffer.
      */
-    protected final ByteBuffer tryWriteLock(Page page) {
+    protected final long tryWriteLock(Page page) {
         return PageHandler.writeLock(page, this, true);
     }
 
@@ -142,7 +142,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param page Page.
      * @return Buffer.
      */
-    protected final ByteBuffer writeLock(Page page) {
+    protected final long writeLock(Page page) {
         return PageHandler.writeLock(page, this, false);
     }
 
@@ -151,7 +151,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param buf Buffer.
      * @param dirty Dirty page.
      */
-    protected final void writeUnlock(Page page, ByteBuffer buf, boolean dirty) {
+    protected final void writeUnlock(Page page, long buf, boolean dirty) {
         PageHandler.writeUnlock(page, buf, this, dirty);
     }
 
@@ -159,7 +159,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param page Page.
      * @return Buffer.
      */
-    protected final ByteBuffer readLock(Page page) {
+    protected final long readLock(Page page) {
         return PageHandler.readLock(page, this);
     }
 
@@ -167,7 +167,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param page Page.
      * @param buf Buffer.
      */
-    protected final void readUnlock(Page page, ByteBuffer buf) {
+    protected final void readUnlock(Page page, long buf) {
         PageHandler.readUnlock(page, buf, this);
     }
 
@@ -177,12 +177,12 @@ public abstract class DataStructure implements PageLockListener {
     }
 
     /** {@inheritDoc} */
-    @Override public void onWriteLock(Page page, ByteBuffer buf) {
+    @Override public void onWriteLock(Page page, long buf) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
+    @Override public void onWriteUnlock(Page page, long buf) {
         // No-op.
     }
 
@@ -192,12 +192,12 @@ public abstract class DataStructure implements PageLockListener {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReadLock(Page page, ByteBuffer buf) {
+    @Override public void onReadLock(Page page, long buf) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onReadUnlock(Page page, ByteBuffer buf) {
+    @Override public void onReadUnlock(Page page, long buf) {
         // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 6a29027..af20d15 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
@@ -72,7 +72,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     /** */
     private final PageHandler<CacheDataRow, Integer> writeRow =
         new PageHandler<CacheDataRow, Integer>() {
-            @Override public Integer run(Page page, PageIO iox, ByteBuffer buf, CacheDataRow row, int written)
+            @Override public Integer run(Page page, PageIO iox, long buf, CacheDataRow row, int written)
                 throws IgniteCheckedException {
                 DataPageIO io = (DataPageIO)iox;
 
@@ -380,7 +380,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, null, REUSE_BUCKET);
+        put(bag, null, 0L, REUSE_BUCKET);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 cc85412..8311b65 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
@@ -85,7 +85,7 @@ 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 buf, Void ignore, int bucket)
             throws IgniteCheckedException {
             assert getPageId(buf) == page.id();
 
@@ -149,9 +149,9 @@ public abstract class PagesList extends DataStructure {
 
                 while (nextPageId != 0) {
                     try (Page page = page(nextPageId)) {
-                        ByteBuffer buf = readLock(page); // No concurrent recycling on init.
+                        long buf = readLock(page); // No concurrent recycling on init.
 
-                        assert buf != null;
+                        assert buf != 0L;
 
                         try {
                             PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
@@ -198,7 +198,7 @@ public abstract class PagesList extends DataStructure {
         assert metaPageId != 0;
 
         Page curPage = null;
-        ByteBuffer curBuf = null;
+        long curBuf = 0L;
         PagesListMetaIO curIo = null;
 
         long nextPageId = metaPageId;
@@ -211,7 +211,7 @@ 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(), curBuf, bucket, tails, tailIdx) : 0;
 
                         if (written == 0) {
                             if (nextPageId == 0L) {
@@ -257,7 +257,7 @@ public abstract class PagesList extends DataStructure {
 
         while (nextPageId != 0L) {
             try (Page page = page(nextPageId)) {
-                ByteBuffer buf = writeLock(page);
+                long buf = writeLock(page);
 
                 try {
                     PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
@@ -277,7 +277,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.
@@ -318,7 +318,7 @@ 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);
@@ -462,7 +462,7 @@ public abstract class PagesList extends DataStructure {
                 long pageId = tail.tailId;
 
                 try (Page page = page(pageId)) {
-                    ByteBuffer buf = readLock(page); // No correctness guaranties.
+                    long buf = readLock(page); // No correctness guaranties.
 
                     try {
                         PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
@@ -489,9 +489,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);
@@ -499,9 +499,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;
@@ -547,10 +547,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)
@@ -558,7 +558,7 @@ public abstract class PagesList extends DataStructure {
 
         long dataPageId = dataPage.id();
 
-        int idx = io.addPage(buf, dataPageId);
+        int idx = io.addPage(pageMem.pageSize(), buf, dataPageId);
 
         if (idx == -1)
             handlePageFull(pageId, page, buf, io, dataPage, dataPageBuf, bucket);
@@ -589,10 +589,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();
@@ -627,9 +627,9 @@ public abstract class PagesList extends DataStructure {
             long nextId = allocatePage(null);
 
             try (Page next = page(nextId)) {
-                ByteBuffer nextBuf = writeLock(next); // Newly allocated page.
+                long nextBuf = writeLock(next); // Newly allocated page.
 
-                assert nextBuf != null;
+                assert nextBuf != 0L;
 
                 try {
                     setupNextPage(io, pageId, buf, nextId, nextBuf);
@@ -637,7 +637,7 @@ public abstract class PagesList extends DataStructure {
                     if (isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PagesListSetNextRecord(cacheId, pageId, nextId));
 
-                    int idx = io.addPage(nextBuf, dataPageId);
+                    int idx = io.addPage(pageMem.pageSize(), nextBuf, dataPageId);
 
                     // Here we should never write full page, because it is known to be new.
                     next.fullPageWalRecordPolicy(FALSE);
@@ -683,7 +683,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
@@ -692,21 +692,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(pageMem.pageSize(), prevBuf, nextId);
 
                 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 nextBuf = writeLock(next); // Page from reuse bag can't be concurrently recycled.
 
-                        assert nextBuf != null;
+                        assert nextBuf != 0L;
 
                         if (locked == null) {
                             lockedBufs = new ArrayList<>(2);
@@ -783,11 +783,11 @@ public abstract class PagesList extends DataStructure {
      * @return Buffer if page is locket 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 buf = tryWriteLock(page);
 
-        if (buf != null)
+        if (buf != 0L)
             return buf;
 
         if (lockAttempt == TRY_LOCK_ATTEMPTS) {
@@ -796,11 +796,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.
     }
 
     /**
@@ -819,9 +819,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);
@@ -929,7 +929,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();
 
@@ -943,9 +943,9 @@ public abstract class PagesList extends DataStructure {
 
             long recycleId = 0L;
 
-            ByteBuffer buf = writeLock(page); // Explicit check.
+            long buf = writeLock(page); // Explicit check.
 
-            if (buf == null)
+            if (buf == 0L)
                 return false;
 
             boolean rmvd = false;
@@ -1003,7 +1003,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.
@@ -1040,11 +1040,11 @@ 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 nextBuf = next == null ? 0L : writeLock(next); // Explicit check.
+                long buf = writeLock(page); // Explicit check.
 
-                if (buf == null) {
-                    if (nextBuf != null) // Unlock next page if needed.
+                if (buf == 0L) {
+                    if (nextBuf != 0L) // Unlock next page if needed.
                         writeUnlock(next, nextBuf, false);
 
                     return 0L; // Someone has merged or taken our empty page concurrently. Nothing to do here.
@@ -1057,7 +1057,7 @@ 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(buf) == nextId && (nextId == 0L) == (nextBuf == null)) {
+                    if (io.getNextId(buf) == nextId && (nextId == 0L) == (nextBuf == 0L)) {
                         long recycleId = doMerge(pageId, page, buf, io, next, nextId, nextBuf, bucket);
 
                         write = true;
@@ -1069,7 +1069,7 @@ public abstract class PagesList extends DataStructure {
                     nextId = io.getNextId(buf);
                 }
                 finally {
-                    if (nextBuf != null)
+                    if (nextBuf != 0L)
                         writeUnlock(next, nextBuf, write);
 
                     writeUnlock(page, buf, write);
@@ -1093,11 +1093,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);
@@ -1139,12 +1139,12 @@ 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 prevBuf = writeLock(prev); // No check, we keep a reference.
 
-            assert prevBuf != null;
+            assert prevBuf != 0L;
 
             try {
                 PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevBuf);
@@ -1177,7 +1177,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/8fcac6f2/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..6ac8cde 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,7 +53,7 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
+    @Override public void initNewPage(long buf, long pageId) {
         super.initNewPage(buf, pageId);
 
         setCount(buf, 0);
@@ -64,55 +64,56 @@ public class PagesListMetaIO extends PageIO {
      * @param buf Buffer.
      * @return Stored items count.
      */
-    private int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    private int getCount(long buf) {
+        return PageUtils.getShort(buf, CNT_OFF);
     }
 
     /**
      * @param buf Buffer,
      * @param cnt Stored items count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long buf, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(buf, CNT_OFF, (short)cnt);
     }
 
     /**
      * @param buf Buffer.
      * @return Next meta page ID.
      */
-    public long getNextMetaPageId(ByteBuffer buf) {
-        return buf.getLong(NEXT_META_PAGE_OFF);
+    public long getNextMetaPageId(long buf) {
+        return PageUtils.getLong(buf, NEXT_META_PAGE_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param metaPageId Next meta page ID.
      */
-    public void setNextMetaPageId(ByteBuffer buf, long metaPageId) {
-        buf.putLong(NEXT_META_PAGE_OFF, metaPageId);
+    public void setNextMetaPageId(long buf, long metaPageId) {
+        PageUtils.putLong(buf, NEXT_META_PAGE_OFF, metaPageId);
     }
 
     /**
      * @param buf Buffer.
      */
-    public void resetCount(ByteBuffer buf) {
+    public void resetCount(long buf) {
         setCount(buf, 0);
     }
 
     /**
+     * @param pageSize Page size.
      * @param buf Buffer.
      * @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 buf, int bucket, PagesList.Stripe[] tails, int tailsOff) {
         assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
         int cnt = getCount(buf);
-        int cap = getCapacity(buf);
+        int cap = getCapacity(pageSize, buf);
 
         if (cnt == cap)
             return 0;
@@ -122,8 +123,8 @@ 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(buf, off, (short)bucket);
+            PageUtils.putLong(buf, off + 2, tails[tailsOff].tailId);
 
             tailsOff++;
 
@@ -139,7 +140,7 @@ public class PagesListMetaIO extends PageIO {
      * @param buf Buffer.
      * @param res Results map.
      */
-    public void getBucketsData(ByteBuffer buf, Map<Integer, GridLongList> res) {
+    public void getBucketsData(long buf, Map<Integer, GridLongList> res) {
         int cnt = getCount(buf);
 
         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(buf, off);
             assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
-            long tailId = buf.getLong(off + 2);
+            long tailId = PageUtils.getLong(buf, off + 2);
             assert tailId != 0;
 
             GridLongList list = res.get(bucket);
@@ -171,8 +172,8 @@ public class PagesListMetaIO extends PageIO {
      * @param buf Buffer.
      * @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 buf) {
+        return (pageSize - ITEMS_OFF) / ITEM_SIZE;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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..c92e4a9 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,7 +52,7 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
+    @Override public void initNewPage(long buf, long pageId) {
         super.initNewPage(buf, pageId);
 
         setEmpty(buf);
@@ -64,7 +64,7 @@ public class PagesListNodeIO extends PageIO {
     /**
      * @param buf Buffer.
      */
-    private void setEmpty(ByteBuffer buf) {
+    private void setEmpty(long buf) {
         setCount(buf, 0);
     }
 
@@ -72,32 +72,32 @@ public class PagesListNodeIO extends PageIO {
      * @param buf Buffer.
      * @return Next page ID.
      */
-    public long getNextId(ByteBuffer buf) {
-        return buf.getLong(NEXT_PAGE_ID_OFF);
+    public long getNextId(long buf) {
+        return PageUtils.getLong(buf, NEXT_PAGE_ID_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param nextId Next page ID.
      */
-    public void setNextId(ByteBuffer buf, long nextId) {
-        buf.putLong(NEXT_PAGE_ID_OFF, nextId);
+    public void setNextId(long buf, long nextId) {
+        PageUtils.putLong(buf, NEXT_PAGE_ID_OFF, nextId);
     }
 
     /**
      * @param buf Buffer.
      * @return Previous page ID.
      */
-    public long getPreviousId(ByteBuffer buf) {
-        return buf.getLong(PREV_PAGE_ID_OFF);
+    public long getPreviousId(long buf) {
+        return PageUtils.getLong(buf, PREV_PAGE_ID_OFF);
     }
 
     /**
      * @param buf Page buffer.
      * @param prevId Previous  page ID.
      */
-    public void setPreviousId(ByteBuffer buf, long prevId) {
-        buf.putLong(PREV_PAGE_ID_OFF, prevId);
+    public void setPreviousId(long buf, long prevId) {
+        PageUtils.putLong(buf, PREV_PAGE_ID_OFF, prevId);
     }
 
     /**
@@ -106,8 +106,8 @@ public class PagesListNodeIO extends PageIO {
      * @param buf Page buffer to get count from.
      * @return Total number of entries.
      */
-    public int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    public int getCount(long buf) {
+        return PageUtils.getShort(buf, CNT_OFF);
     }
 
     /**
@@ -116,20 +116,20 @@ public class PagesListNodeIO extends PageIO {
      * @param buf Page buffer to write to.
      * @param cnt Count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long buf, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(buf, 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
     }
 
     /**
@@ -145,8 +145,8 @@ public class PagesListNodeIO extends PageIO {
      * @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 buf, int idx) {
+        return PageUtils.getLong(buf, offset(idx));
     }
 
     /**
@@ -154,8 +154,8 @@ public class PagesListNodeIO extends PageIO {
      * @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 buf, int idx, long pageId) {
+        PageUtils.putLong(buf, offset(idx), pageId);
     }
 
     /**
@@ -165,10 +165,10 @@ public class PagesListNodeIO extends PageIO {
      * @param pageId Page ID.
      * @return Total number of items in this page.
      */
-    public int addPage(ByteBuffer buf, long pageId) {
+    public int addPage(int pageSize, long buf, long pageId) {
         int cnt = getCount(buf);
 
-        if (cnt == getCapacity(buf))
+        if (cnt == getCapacity(pageSize))
             return -1;
 
         setAt(buf, cnt, pageId);
@@ -183,7 +183,7 @@ public class PagesListNodeIO extends PageIO {
      * @param buf Page buffer.
      * @return Removed page ID.
      */
-    public long takeAnyPage(ByteBuffer buf) {
+    public long takeAnyPage(long buf) {
         int cnt = getCount(buf);
 
         if (cnt == 0)
@@ -201,7 +201,7 @@ public class PagesListNodeIO extends PageIO {
      * @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 buf, long dataPageId) {
         assert dataPageId != 0;
 
         int cnt = getCount(buf);
@@ -224,7 +224,7 @@ public class PagesListNodeIO extends PageIO {
      * @param buf Page buffer.
      * @return {@code true} if there are no items in this page.
      */
-    public boolean isEmpty(ByteBuffer buf) {
+    public boolean isEmpty(long buf) {
         return getCount(buf) == 0;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 0e41307..6ea1bff 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
@@ -409,7 +409,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             // 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(buf)))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(pageMem.pageSize(), buf)))) {
                 // 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);
@@ -528,7 +528,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getRootLevel(buf); // Can drop only root.
 
-            io.cutRoot(buf);
+            io.cutRoot(pageMem.pageSize(), buf);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
@@ -548,7 +548,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getLevelsCount(buf);
 
-            io.addRoot(buf, rootPageId);
+            io.addRoot(pageMem.pageSize(), buf, rootPageId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
@@ -566,7 +566,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            io.initRoot(buf, rootId);
+            io.initRoot(pageMem.pageSize(), buf, rootId);
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageInitRootRecord(cacheId, meta.id(), rootId));
@@ -2185,7 +2185,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          */
         private L insert(Page page, BPlusIO<L> io, ByteBuffer buf, int idx, int lvl)
             throws IgniteCheckedException {
-            int maxCnt = io.getMaxCount(buf);
+            int maxCnt = io.getMaxCount(pageMem.pageSize(), buf);
             int cnt = io.getCount(buf);
 
             if (cnt == maxCnt) // Need to split page.
@@ -2625,7 +2625,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                         // Exit: we are done.
                     }
                     else if (tail.sibling != null &&
-                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(tail.buf)) {
+                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(pageMem.pageSize(), tail.buf)) {
                         // Release everything lower than tail, we've already merged this path.
                         doReleaseTail(tail.down);
                         tail.down = null;
@@ -2940,7 +2940,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             boolean emptyBranch = needMergeEmptyBranch == TRUE || needMergeEmptyBranch == READY;
 
-            if (!left.io.merge(prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch))
+            if (!left.io.merge(pageMem.pageSize(), prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch))
                 return false;
 
             // Invalidate indexes after successful merge.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
index 5fc3d25..6c739a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 
 /**
@@ -72,7 +72,7 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
+    @Override public void initNewPage(long buf, long pageId) {
         super.initNewPage(buf, pageId);
 
         setCount(buf, 0);
@@ -84,16 +84,16 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param buf Buffer.
      * @return Forward page ID.
      */
-    public final long getForward(ByteBuffer buf) {
-        return buf.getLong(FORWARD_OFF);
+    public final long getForward(long buf) {
+        return PageUtils.getLong(buf, FORWARD_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param pageId Forward page ID.
      */
-    public final void setForward(ByteBuffer buf, long pageId) {
-        buf.putLong(FORWARD_OFF, pageId);
+    public final void setForward(long buf, long pageId) {
+        PageUtils.putLong(buf, FORWARD_OFF, pageId);
 
         assert getForward(buf) == pageId;
     }
@@ -102,16 +102,16 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param buf Buffer.
      * @return Remove ID.
      */
-    public final long getRemoveId(ByteBuffer buf) {
-        return buf.getLong(REMOVE_ID_OFF);
+    public final long getRemoveId(long buf) {
+        return PageUtils.getLong(buf, REMOVE_ID_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param rmvId Remove ID.
      */
-    public final void setRemoveId(ByteBuffer buf, long rmvId) {
-        buf.putLong(REMOVE_ID_OFF, rmvId);
+    public final void setRemoveId(long buf, long rmvId) {
+        PageUtils.putLong(buf, REMOVE_ID_OFF, rmvId);
 
         assert getRemoveId(buf) == rmvId;
     }
@@ -120,8 +120,8 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param buf Buffer.
      * @return Items count in the page.
      */
-    public final int getCount(ByteBuffer buf) {
-        int cnt = buf.getShort(CNT_OFF) & 0xFFFF;
+    public final int getCount(long buf) {
+        int cnt = PageUtils.getShort(buf, CNT_OFF) & 0xFFFF;
 
         assert cnt >= 0: cnt;
 
@@ -132,17 +132,17 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param buf Buffer.
      * @param cnt Count.
      */
-    public final void setCount(ByteBuffer buf, int cnt) {
+    public final void setCount(long buf, int cnt) {
         assert cnt >= 0: cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(buf, CNT_OFF, (short)cnt);
 
         assert getCount(buf) == cnt;
     }
 
     /**
      * @return {@code true} If we can get the full row from this page using
-     * method {@link BPlusTree#getRow(BPlusIO, ByteBuffer, int)}.
+     * method {@link BPlusTree#getRow(BPlusIO, long, int)}.
      * Must always be {@code true} for leaf pages.
      */
     public final boolean canGetRow() {
@@ -160,7 +160,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param buf Buffer.
      * @return Max items count.
      */
-    public abstract int getMaxCount(ByteBuffer buf);
+    public abstract int getMaxCount(int pageSize, long buf);
 
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
@@ -171,7 +171,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param rowBytes Row bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public final void store(ByteBuffer buf, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
+    public final void store(long buf, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
         int off = offset(idx);
 
         if (rowBytes == null)
@@ -194,7 +194,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param row Lookup or full row.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void storeByOffset(ByteBuffer buf, int off, L row) throws IgniteCheckedException;
+    public abstract void storeByOffset(long buf, int off, L row) throws IgniteCheckedException;
 
     /**
      * Store row info from the given source.
@@ -206,7 +206,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param srcIdx Source index.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void store(ByteBuffer dst, int dstIdx, BPlusIO<L> srcIo, ByteBuffer src, int srcIdx)
+    public abstract void store(long dst, int dstIdx, BPlusIO<L> srcIo, long src, int srcIdx)
         throws IgniteCheckedException;
 
     /**
@@ -218,7 +218,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @return Lookup row.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract L getLookupRow(BPlusTree<L, ?> tree, ByteBuffer buf, int idx) throws IgniteCheckedException;
+    public abstract L getLookupRow(BPlusTree<L, ?> tree, long buf, int idx) throws IgniteCheckedException;
 
     /**
      * Copy items from source buffer to destination buffer.
@@ -232,7 +232,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param cpLeft Copy leftmost link (makes sense only for inner pages).
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
+    public abstract void copyItems(long src, long dst, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
         throws IgniteCheckedException;
 
     // Methods for B+Tree logic.
@@ -245,7 +245,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param rightId Page ID which will be to the right child for the inserted item.
      * @throws IgniteCheckedException If failed.
      */
-    public void insert(ByteBuffer buf, int idx, L row, byte[] rowBytes, long rightId)
+    public void insert(long buf, int idx, L row, byte[] rowBytes, long rightId)
         throws IgniteCheckedException {
         int cnt = getCount(buf);
 
@@ -266,9 +266,9 @@ public abstract class BPlusIO<L> extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void splitForwardPage(
-        ByteBuffer buf,
+        long buf,
         long fwdId,
-        ByteBuffer fwdBuf,
+        long fwdBuf,
         int mid,
         int cnt
     ) throws IgniteCheckedException {
@@ -291,7 +291,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param mid Bisection index.
      * @param fwdId New forward page ID.
      */
-    public void splitExistingPage(ByteBuffer buf, int mid, long fwdId) {
+    public void splitExistingPage(long buf, int mid, long fwdId) {
         setCount(buf, mid);
         setForward(buf, fwdId);
     }
@@ -302,7 +302,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param cnt Count.
      * @throws IgniteCheckedException If failed.
      */
-    public void remove(ByteBuffer buf, int idx, int cnt) throws IgniteCheckedException {
+    public void remove(long buf, int idx, int cnt) throws IgniteCheckedException {
         cnt--;
 
         copyItems(buf, buf, idx + 1, idx, cnt - idx, false);
@@ -320,11 +320,12 @@ public abstract class BPlusIO<L> extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public boolean merge(
+        int pageSize,
         BPlusIO<L> prntIo,
-        ByteBuffer prnt,
+        long prnt,
         int prntIdx,
-        ByteBuffer left,
-        ByteBuffer right,
+        long left,
+        long right,
         boolean emptyBranch
     ) throws IgniteCheckedException {
         int prntCnt = prntIo.getCount(prnt);
@@ -337,7 +338,7 @@ public abstract class BPlusIO<L> extends PageIO {
         if (!isLeaf() && !emptyBranch)
             newCnt++;
 
-        if (newCnt > getMaxCount(left)) {
+        if (newCnt > getMaxCount(pageSize, left)) {
             assert !emptyBranch;
 
             return false;
@@ -373,15 +374,7 @@ public abstract class BPlusIO<L> extends PageIO {
      * @param pos Position in buffer.
      * @param bytes Bytes.
      */
-    private static void putBytes(ByteBuffer buf, int pos, byte[] bytes) {
-        int oldPos = buf.position();
-
-        try {
-            buf.position(pos);
-            buf.put(bytes);
-        }
-        finally {
-            buf.position(oldPos);
-        }
+    private static void putBytes(long buf, int pos, byte[] bytes) {
+        PageUtils.putBytes(buf, pos, bytes);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
index 90b0f37..d0ead51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 
 /**
@@ -45,11 +45,11 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(ByteBuffer buf) {
+    @Override public int getMaxCount(int pageSize, long buf) {
         // The structure of the page is the following:
         // |ITEMS_OFF|w|A|x|B|y|C|z|
         // where capital letters are data items, lowercase letters are 8 byte page references.
-        return (buf.capacity() - ITEMS_OFF - 8) / (itemSize + 8);
+        return (pageSize - ITEMS_OFF - 8) / (itemSize + 8);
     }
 
     /**
@@ -57,8 +57,8 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getLeft(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx, SHIFT_LEFT));
+    public final long getLeft(long buf, int idx) {
+        return PageUtils.getLong(buf, (offset(idx, SHIFT_LEFT)));
     }
 
     /**
@@ -66,8 +66,8 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setLeft(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx, SHIFT_LEFT), pageId);
+    public final void setLeft(long buf, int idx, long pageId) {
+        PageUtils.putLong(buf, offset(idx, SHIFT_LEFT), pageId);
 
         assert pageId == getLeft(buf, idx);
     }
@@ -77,8 +77,8 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getRight(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx, SHIFT_RIGHT));
+    public final long getRight(long buf, int idx) {
+        return PageUtils.getLong(buf, offset(idx, SHIFT_RIGHT));
     }
 
     /**
@@ -86,14 +86,14 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setRight(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx, SHIFT_RIGHT), pageId);
+    public final void setRight(long buf, int idx, long pageId) {
+        PageUtils.putLong(buf, offset(idx, SHIFT_RIGHT), pageId);
 
         assert pageId == getRight(buf, idx);
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long src, long dst, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
         assert srcIdx != dstIdx || src != dst;
 
@@ -103,11 +103,11 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
             PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
 
             if (cpLeft)
-                dst.putLong(offset(dstIdx, SHIFT_LEFT), src.getLong(offset(srcIdx, SHIFT_LEFT)));
+                PageUtils.putLong(dst, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(src, (offset(srcIdx, SHIFT_LEFT))));
         }
         else {
             if (cpLeft)
-                dst.putLong(offset(dstIdx, SHIFT_LEFT), src.getLong(offset(srcIdx, SHIFT_LEFT)));
+                PageUtils.putLong(dst, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(src, (offset(srcIdx, SHIFT_LEFT))));
 
             PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
         }
@@ -131,7 +131,7 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
 
     /** {@inheritDoc} */
     @Override public void insert(
-        ByteBuffer buf,
+        long buf,
         int idx,
         L row,
         byte[] rowBytes,
@@ -152,7 +152,7 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
      * @throws IgniteCheckedException If failed.
      */
     public void initNewRoot(
-        ByteBuffer newRootBuf,
+        long newRootBuf,
         long newRootId,
         long leftChildId,
         L row,

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
index 824c943..76c621c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
@@ -35,12 +35,12 @@ public abstract class BPlusLeafIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(ByteBuffer buf) {
-        return (buf.capacity() - ITEMS_OFF) / itemSize;
+    @Override public int getMaxCount(int pageSize, long buf) {
+        return (pageSize - ITEMS_OFF) / itemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long src, long dst, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
         assert srcIdx != dstIdx || src != dst;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
index 15a49ef..ccaa952 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
  * IO routines for B+Tree meta pages.
@@ -45,8 +45,8 @@ public class BPlusMetaIO extends PageIO {
      * @param buf Buffer.
      * @param rootId Root page ID.
      */
-    public void initRoot(ByteBuffer buf, long rootId) {
-        setLevelsCount(buf, 1);
+    public void initRoot(int pageSize, long buf, long rootId) {
+        setLevelsCount(pageSize, buf, 1);
         setFirstPageId(buf, 0, rootId);
     }
 
@@ -54,26 +54,26 @@ public class BPlusMetaIO extends PageIO {
      * @param buf Buffer.
      * @return Number of levels in this tree.
      */
-    public int getLevelsCount(ByteBuffer buf) {
-        return buf.get(LVLS_OFF);
+    public int getLevelsCount(long buf) {
+        return PageUtils.getByte(buf, LVLS_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @return Max levels possible for this page size.
      */
-    public int getMaxLevels(ByteBuffer buf) {
-        return (buf.capacity() - REFS_OFF) / 8;
+    public int getMaxLevels(int pageSize, long buf) {
+        return (pageSize - REFS_OFF) / 8;
     }
 
     /**
      * @param buf  Buffer.
      * @param lvls Number of levels in this tree.
      */
-    public void setLevelsCount(ByteBuffer buf, int lvls) {
-        assert lvls >= 0 && lvls <= getMaxLevels(buf): lvls;
+    public void setLevelsCount(int pageSize, long buf, int lvls) {
+        assert lvls >= 0 && lvls <= getMaxLevels(pageSize, buf) : lvls;
 
-        buf.put(LVLS_OFF, (byte)lvls);
+        PageUtils.putByte(buf, LVLS_OFF, (byte)lvls);
 
         assert getLevelsCount(buf) == lvls;
     }
@@ -91,8 +91,8 @@ public class BPlusMetaIO extends PageIO {
      * @param lvl Level.
      * @return First page ID at that level.
      */
-    public long getFirstPageId(ByteBuffer buf, int lvl) {
-        return buf.getLong(offset(lvl));
+    public long getFirstPageId(long buf, int lvl) {
+        return PageUtils.getLong(buf, offset(lvl));
     }
 
     /**
@@ -100,10 +100,10 @@ public class BPlusMetaIO extends PageIO {
      * @param lvl    Level.
      * @param pageId Page ID.
      */
-    public void setFirstPageId(ByteBuffer buf, int lvl, long pageId) {
+    public void setFirstPageId(long buf, int lvl, long pageId) {
         assert lvl >= 0 && lvl < getLevelsCount(buf);
 
-        buf.putLong(offset(lvl), pageId);
+        PageUtils.putLong(buf, offset(lvl), pageId);
 
         assert getFirstPageId(buf, lvl) == pageId;
     }
@@ -112,7 +112,7 @@ public class BPlusMetaIO extends PageIO {
      * @param buf Buffer.
      * @return Root level.
      */
-    public int getRootLevel(ByteBuffer buf) {
+    public int getRootLevel(long buf) {
         int lvls = getLevelsCount(buf); // The highest level page is root.
 
         assert lvls > 0 : lvls;
@@ -124,19 +124,19 @@ public class BPlusMetaIO extends PageIO {
      * @param buf Buffer.
      * @param rootPageId New root page ID.
      */
-    public void addRoot(ByteBuffer buf, long rootPageId) {
+    public void addRoot(int pageSize, long buf, long rootPageId) {
         int lvl = getLevelsCount(buf);
 
-        setLevelsCount(buf, lvl + 1);
+        setLevelsCount(pageSize, buf, lvl + 1);
         setFirstPageId(buf, lvl, rootPageId);
     }
 
     /**
      * @param buf Buffer.
      */
-    public void cutRoot(ByteBuffer buf) {
+    public void cutRoot(int pageSize, long buf) {
         int lvl = getRootLevel(buf);
 
-        setLevelsCount(buf, lvl); // Decrease tree height.
+        setLevelsCount(pageSize, buf, lvl); // Decrease tree height.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
index 5e9fd6d..ed75d12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
 /**
@@ -58,21 +58,21 @@ public class CacheVersionIO {
      * @param ver Version to write.
      * @param allowNull Is {@code null} version allowed.
      */
-    public static void write(ByteBuffer buf, GridCacheVersion ver, boolean allowNull) {
+    public static void write(long buf, GridCacheVersion ver, boolean allowNull) {
         if (ver == null) {
             if (allowNull)
-                buf.put(NULL_PROTO_VER);
+                PageUtils.putByte(buf, 0, NULL_PROTO_VER);
             else
                 throw new IllegalStateException("Cache version is null");
         }
         else {
             byte protoVer = 1; // Version of serialization protocol.
 
-            buf.put(protoVer);
-            buf.putInt(ver.topologyVersion());
-            buf.putInt(ver.nodeOrderAndDrIdRaw());
-            buf.putLong(ver.globalTime());
-            buf.putLong(ver.order());
+            PageUtils.putByte(buf, 0, protoVer);
+            PageUtils.putInt(buf, 1, ver.topologyVersion());
+            PageUtils.putInt(buf, 5, ver.nodeOrderAndDrIdRaw());
+            PageUtils.putLong(buf, 9, ver.globalTime());
+            PageUtils.putLong(buf, 17, ver.order());
         }
     }
 
@@ -102,8 +102,8 @@ public class CacheVersionIO {
      * @return Size of serialized version.
      * @throws IgniteCheckedException If failed.
      */
-    public static int readSize(ByteBuffer buf, boolean allowNull) throws IgniteCheckedException {
-        byte protoVer = checkProtocolVersion(buf.get(buf.position()), allowNull);
+    public static int readSize(long buf, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(PageUtils.getByte(buf, 0), allowNull);
 
         switch (protoVer) {
             case NULL_PROTO_VER:
@@ -126,16 +126,16 @@ public class CacheVersionIO {
      * @return Version.
      * @throws IgniteCheckedException If failed.
      */
-    public static GridCacheVersion read(ByteBuffer buf, boolean allowNull) throws IgniteCheckedException {
-        byte protoVer = checkProtocolVersion(buf.get(), allowNull);
+    public static GridCacheVersion read(long buf, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(PageUtils.getByte(buf, 0), allowNull);
 
         if (protoVer == NULL_PROTO_VER)
             return null;
 
-        int topVer = buf.getInt();
-        int nodeOrderDrId = buf.getInt();
-        long globalTime = buf.getLong();
-        long order = buf.getLong();
+        int topVer = PageUtils.getInt(buf, 1);
+        int nodeOrderDrId = PageUtils.getInt(buf, 5);
+        long globalTime = PageUtils.getLong(buf, 9);
+        long order = PageUtils.getLong(buf, 17);
 
         return new GridCacheVersion(topVer, nodeOrderDrId, globalTime, order);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index a69caab..e1bab5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -17,12 +17,12 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
@@ -109,16 +109,16 @@ public class DataPageIO extends PageIO {
      * @param buf Byte buffer.
      * @param freeListPageId Free list page ID.
      */
-    public void setFreeListPageId(ByteBuffer buf, long freeListPageId) {
-        buf.putLong(FREE_LIST_PAGE_ID_OFF, freeListPageId);
+    public void setFreeListPageId(long buf, long freeListPageId) {
+        PageUtils.putLong(buf, FREE_LIST_PAGE_ID_OFF, freeListPageId);
     }
 
     /**
      * @param buf Byte buffer.
      * @return Free list page ID.
      */
-    public long getFreeListPageId(ByteBuffer buf) {
-        return buf.getLong(FREE_LIST_PAGE_ID_OFF);
+    public long getFreeListPageId(long buf) {
+        return PageUtils.getLong(buf, FREE_LIST_PAGE_ID_OFF);
     }
 
     /**
@@ -213,7 +213,7 @@ public class DataPageIO extends PageIO {
     /**
      * @return {@code true} If there is no useful data in this page.
      */
-    public boolean isEmpty(ByteBuffer buf) {
+    public boolean isEmpty(long buf) {
         return getDirectCount(buf) == 0;
     }
 
@@ -241,8 +241,8 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @return Direct count.
      */
-    private int getDirectCount(ByteBuffer buf) {
-        return buf.get(DIRECT_CNT_OFF) & 0xFF;
+    private int getDirectCount(long buf) {
+        return PageUtils.getByte(buf, DIRECT_CNT_OFF) & 0xFF;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
index bd1139d..f5eb4b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
-
 /**
  * Registry for IO versions.
  */
@@ -83,7 +81,7 @@ public final class IOVersions<V extends PageIO> {
      * @param buf Buffer.
      * @return IO.
      */
-    public V forPage(ByteBuffer buf) {
+    public V forPage(long buf) {
         int ver = PageIO.getVersion(buf);
 
         V res = forVersion(ver);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8fcac6f2/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 aa2d368..12646ae 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
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl;
 import org.apache.ignite.internal.processors.cache.database.MetadataStorage;
@@ -36,10 +36,10 @@ import org.apache.ignite.internal.processors.cache.database.tree.util.PageLockLi
  * 1. IO should not have any `public static` methods.
  *    We have versioned IOs and any static method will mean that it have to always work in backward
  *    compatible way between all the IO versions. The base class {@link PageIO} has
- *    static methods (like {@code {@link #getPageId(ByteBuffer)}}) intentionally:
+ *    static methods (like {@code {@link #getPageId(long)}}) intentionally:
  *    this base format can not be changed between versions.
  *
- * 2. IO must correctly override {@link #initNewPage(ByteBuffer, long)} method and call super.
+ * 2. IO must correctly override {@link #initNewPage(long, long)} method and call super.
  *    We have logic that relies on this behavior.
  *
  * 3. Page IO type ID constant must be declared in this class to have a list of all the
@@ -166,16 +166,16 @@ public abstract class PageIO {
     /**
      * @return Page type.
      */
-    public static int getType(ByteBuffer buf) {
-        return buf.getShort(TYPE_OFF) & 0xFFFF;
+    public static int getType(long buf) {
+        return PageUtils.getShort(buf, TYPE_OFF) & 0xFFFF;
     }
 
     /**
      * @param buf Buffer.
      * @param type Type.
      */
-    public static void setType(ByteBuffer buf, int type) {
-        buf.putShort(TYPE_OFF, (short)type);
+    public static void setType(long buf, int type) {
+        PageUtils.putShort(buf, TYPE_OFF, (short)type);
 
         assert getType(buf) == type;
     }
@@ -184,16 +184,16 @@ public abstract class PageIO {
      * @param buf Buffer.
      * @return Version.
      */
-    public static int getVersion(ByteBuffer buf) {
-        return buf.getShort(VER_OFF) & 0xFFFF;
+    public static int getVersion(long buf) {
+        return PageUtils.getShort(buf, VER_OFF) & 0xFFFF;
     }
 
     /**
      * @param buf Buffer.
      * @param ver Version.
      */
-    public static void setVersion(ByteBuffer buf, int ver) {
-        buf.putShort(VER_OFF, (short)ver);
+    public static void setVersion(long buf, int ver) {
+        PageUtils.putShort(buf, VER_OFF, (short)ver);
 
         assert getVersion(buf) == ver;
     }
@@ -202,16 +202,16 @@ public abstract class PageIO {
      * @param buf Buffer.
      * @return Page ID.
      */
-    public static long getPageId(ByteBuffer buf) {
-        return buf.getLong(PAGE_ID_OFF);
+    public static long getPageId(long buf) {
+        return PageUtils.getLong(buf, PAGE_ID_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param pageId Page ID.
      */
-    public static void setPageId(ByteBuffer buf, long pageId) {
-        buf.putLong(PAGE_ID_OFF, pageId);
+    public static void setPageId(long buf, long pageId) {
+        PageUtils.putLong(buf, PAGE_ID_OFF, pageId);
 
         assert getPageId(buf) == pageId;
     }
@@ -220,16 +220,16 @@ public abstract class PageIO {
      * @param buf Buffer.
      * @return Checksum.
      */
-    public static int getCrc(ByteBuffer buf) {
-        return buf.getInt(CRC_OFF);
+    public static int getCrc(long buf) {
+        return PageUtils.getInt(buf, CRC_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param crc Checksum.
      */
-    public static void setCrc(ByteBuffer buf, int crc) {
-        buf.putInt(CRC_OFF, crc);
+    public static void setCrc(long buf, int crc) {
+        PageUtils.putInt(buf, CRC_OFF, crc);
     }
 
     /**
@@ -275,15 +275,15 @@ public abstract class PageIO {
      * @param buf Buffer.
      * @param pageId Page ID.
      */
-    public void initNewPage(ByteBuffer buf, long pageId) {
+    public void initNewPage(long buf, long pageId) {
         setType(buf, getType());
         setVersion(buf, getVersion());
         setPageId(buf, pageId);
         setCrc(buf, 0);
 
-        buf.putLong(RESERVED_1_OFF, 0L);
-        buf.putLong(RESERVED_2_OFF, 0L);
-        buf.putLong(RESERVED_3_OFF, 0L);
+        PageUtils.putLong(buf, RESERVED_1_OFF, 0L);
+        PageUtils.putLong(buf, RESERVED_2_OFF, 0L);
+        PageUtils.putLong(buf, RESERVED_3_OFF, 0L);
     }
 
     /** {@inheritDoc} */
@@ -296,7 +296,7 @@ public abstract class PageIO {
      * @return IO.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends PageIO> Q getPageIO(ByteBuffer buf) throws IgniteCheckedException {
+    public static <Q extends PageIO> Q getPageIO(long buf) throws IgniteCheckedException {
         int type = getType(buf);
         int ver = getVersion(buf);
 
@@ -343,7 +343,7 @@ public abstract class PageIO {
      * @return IO for either inner or leaf B+Tree page.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends BPlusIO<?>> Q getBPlusIO(ByteBuffer buf) throws IgniteCheckedException {
+    public static <Q extends BPlusIO<?>> Q getBPlusIO(long buf) throws IgniteCheckedException {
         int type = getType(buf);
         int ver = getVersion(buf);
 


[22/25] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-gg-8.0.2.ea2' into ignite-gg-11810

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/community/ignite-gg-8.0.2.ea2' into ignite-gg-11810

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogNoopManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java


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

Branch: refs/heads/ignite-gg-11810
Commit: 9876950e729113dd590def6c16309b7d91c753b6
Parents: 4183186
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 14:17:26 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 14:17:26 2016 +0300

----------------------------------------------------------------------
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |  1 -
 .../record/delta/TrackingPageDeltaRecord.java   |  3 +-
 .../cache/database/tree/io/PageMetaIO.java      | 72 ++++++++---------
 .../cache/database/tree/io/TrackingPageIO.java  | 82 ++++++++++----------
 4 files changed, 78 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9876950e/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
index ce2453e..98502a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9876950e/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
index 7daac81..a76ed48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.TrackingPageIO;
 
@@ -71,7 +70,7 @@ public class TrackingPageDeltaRecord extends PageDeltaRecord {
 
     /** {@inheritDoc} */
     @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
-        TrackingPageIO.VERSIONS.forPage(pageAddr).markChanged(pageAddr, pageIdToMark, nextSnapshotId, lastSuccessfulSnapshotId, buf.capacity());
+        TrackingPageIO.VERSIONS.forPage(pageAddr).markChanged(pageAddr, pageIdToMark, nextSnapshotId, lastSuccessfulSnapshotId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9876950e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index 5e6dd87..11a2643 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@ -74,30 +74,30 @@ public class PageMetaIO extends PageIO {
     @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
         super.initNewPage(pageAddr, pageId, pageSize);
 
-        setTreeRoot(buf, 0);
-        setReuseListRoot(buf, 0);
-        setLastSuccessfulFullSnapshotId(buf, 0);
-        setLastSuccessfulSnapshotId(buf, 0);
-        setNextSnapshotTag(buf, 1);
-        setLastSuccessfulSnapshotTag(buf, 0);
-        setLastAllocatedIndex(buf, 0);
-        setCandidateAllocatedIndex(buf, 0);
+        setTreeRoot(pageAddr, 0);
+        setReuseListRoot(pageAddr, 0);
+        setLastSuccessfulFullSnapshotId(pageAddr, 0);
+        setLastSuccessfulSnapshotId(pageAddr, 0);
+        setNextSnapshotTag(pageAddr, 1);
+        setLastSuccessfulSnapshotTag(pageAddr, 0);
+        setLastAllocatedIndex(pageAddr, 0);
+        setCandidateAllocatedIndex(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Tree root page.
      */
-    public long getTreeRoot(ByteBuffer buf) {
-        return buf.getLong(TREE_ROOT_OFF);
+    public long getTreeRoot(long pageAddr) {
+        return PageUtils.getLong(pageAddr, TREE_ROOT_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param treeRoot Tree root
      */
-    public void setTreeRoot(@NotNull ByteBuffer buf, long treeRoot) {
-        buf.putLong(TREE_ROOT_OFF, treeRoot);
+    public void setTreeRoot(long pageAddr, long treeRoot) {
+        PageUtils.putLong(pageAddr, TREE_ROOT_OFF, treeRoot);
     }
 
     /**
@@ -117,63 +117,63 @@ public class PageMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulSnapshotId Last successful snapshot id.
      */
-    public void setLastSuccessfulSnapshotId(@NotNull ByteBuffer buf, long lastSuccessfulSnapshotId) {
-        buf.putLong(LAST_SUCCESSFUL_SNAPSHOT_ID_OFF, lastSuccessfulSnapshotId);
+    public void setLastSuccessfulSnapshotId(long pageAddr, long lastSuccessfulSnapshotId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_SNAPSHOT_ID_OFF, lastSuccessfulSnapshotId);
     }
 
     /**
      * @param pageAddr Page address.
      */
-    public long getLastSuccessfulSnapshotId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_SNAPSHOT_ID_OFF);
+    public long getLastSuccessfulSnapshotId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_SNAPSHOT_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulFullSnapshotId Last successful full snapshot id.
      */
-    public void setLastSuccessfulFullSnapshotId(@NotNull ByteBuffer buf, long lastSuccessfulFullSnapshotId) {
-        buf.putLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF, lastSuccessfulFullSnapshotId);
+    public void setLastSuccessfulFullSnapshotId(long pageAddr, long lastSuccessfulFullSnapshotId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF, lastSuccessfulFullSnapshotId);
     }
 
     /**
      * @param pageAddr Page address.
      */
-    public long getLastSuccessfulFullSnapshotId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF);
+    public long getLastSuccessfulFullSnapshotId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextSnapshotId Next snapshot id.
      */
-    public void setNextSnapshotTag(@NotNull ByteBuffer buf, long nextSnapshotId) {
-        buf.putLong(NEXT_SNAPSHOT_TAG_OFF, nextSnapshotId);
+    public void setNextSnapshotTag(long pageAddr, long nextSnapshotId) {
+        PageUtils.putLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF, nextSnapshotId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulSnapshotTag(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF);
+    public long getLastSuccessfulSnapshotTag(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulSnapshotTag Last successful snapshot tag.
      */
-    public void setLastSuccessfulSnapshotTag(@NotNull ByteBuffer buf, long lastSuccessfulSnapshotTag) {
-        buf.putLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF, lastSuccessfulSnapshotTag);
+    public void setLastSuccessfulSnapshotTag(long pageAddr, long lastSuccessfulSnapshotTag) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF, lastSuccessfulSnapshotTag);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getNextSnapshotTag(@NotNull ByteBuffer buf) {
-        return buf.getLong(NEXT_SNAPSHOT_TAG_OFF);
+    public long getNextSnapshotTag(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9876950e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
index a2fed21..6ecd9ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
@@ -71,13 +71,13 @@ public class TrackingPageIO extends PageIO {
     /**
      * Will mark pageId as changed for next (!) snapshotId
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Page id.
      * @param nextSnapshotTag tag of next snapshot.
      * @param pageSize Page size.
      */
-    public boolean markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
-        validateSnapshotId(buf, nextSnapshotTag, lastSuccessfulSnapshotTag, pageSize);
+    public boolean markChanged(long pageAddr, long pageId, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
+        validateSnapshotId(pageAddr, nextSnapshotTag, lastSuccessfulSnapshotTag, pageSize);
 
         int cntOfPage = countOfPageToTrack(pageSize);
 
@@ -87,7 +87,7 @@ public class TrackingPageIO extends PageIO {
 
         int idx = sizeOff + SIZE_FIELD_SIZE + (idxToUpdate >> 3);
 
-        byte byteToUpdate = PageUtils.getByte(buf, idx);
+        byte byteToUpdate = PageUtils.getByte(pageAddr, idx);
 
         int updateTemplate = 1 << (idxToUpdate & 0b111);
 
@@ -96,28 +96,28 @@ public class TrackingPageIO extends PageIO {
         if (byteToUpdate == newVal)
             return false;
 
-        buf.put(idx, newVal);
+        PageUtils.putByte(pageAddr, idx, newVal);
 
-        short newSize = (short)(buf.getShort(sizeOff) + 1);
+        short newSize = (short)(PageUtils.getShort(pageAddr, sizeOff) + 1);
 
-        buf.putShort(sizeOff, newSize);
+        PageUtils.putShort(pageAddr, sizeOff, newSize);
 
-        assert newSize == countOfChangedPage(buf, nextSnapshotTag, pageSize);
+        assert newSize == countOfChangedPage(pageAddr, nextSnapshotTag, pageSize);
 
         return true;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextSnapshotTag Next snapshot id.
      * @param lastSuccessfulSnapshotId Last successful snapshot id.
      * @param pageSize Page size.
      */
-    private void validateSnapshotId(ByteBuffer buf, long nextSnapshotTag, long lastSuccessfulSnapshotId, int pageSize) {
+    private void validateSnapshotId(long pageAddr, long nextSnapshotTag, long lastSuccessfulSnapshotId, int pageSize) {
         assert nextSnapshotTag != lastSuccessfulSnapshotId : "nextSnapshotTag = " + nextSnapshotTag +
             ", lastSuccessfulSnapshotId = " + lastSuccessfulSnapshotId;
 
-        long last = getLastSnapshotTag(buf);
+        long last = getLastSnapshotTag(pageAddr);
 
         assert last <= nextSnapshotTag : "last = " + last + ", nextSnapshotTag = " + nextSnapshotTag;
 
@@ -127,9 +127,9 @@ public class TrackingPageIO extends PageIO {
         int cntOfPage = countOfPageToTrack(pageSize);
 
         if (last <= lastSuccessfulSnapshotId) { //we can drop our data
-            buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag);
+            PageUtils.putLong(pageAddr, LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag);
 
-            PageHandler.zeroMemory(buf, SIZE_FIELD_OFFSET, buf.capacity() - SIZE_FIELD_OFFSET);
+            PageHandler.zeroMemory(pageAddr, SIZE_FIELD_OFFSET, pageSize - SIZE_FIELD_OFFSET);
         } else { //we can't drop data, it is still necessary for incremental snapshots
             int len = cntOfPage >> 3;
 
@@ -139,55 +139,55 @@ public class TrackingPageIO extends PageIO {
             if (last - lastSuccessfulSnapshotId == 1) { //we should keep only data in last half
                 //new data will be written in the same half, we should move old data to another half
                 if ((nextSnapshotTag - last) % 2 == 0)
-                    PageHandler.copyMemory(buf, buf, sizeOff, sizeOff2, len + SIZE_FIELD_SIZE);
+                    PageHandler.copyMemory(pageAddr, pageAddr, sizeOff, sizeOff2, len + SIZE_FIELD_SIZE);
             } else { //last - lastSuccessfulSnapshotId > 1, e.g. we should merge two half in one
                 int newSize = 0;
                 int i = 0;
 
                 for (; i < len - 8; i += 8) {
-                    long newVal = PageUtils.getLong(buf, sizeOff + SIZE_FIELD_SIZE + i) | PageUtils.getLong(buf, sizeOff2 + SIZE_FIELD_SIZE + i);
+                    long newVal = PageUtils.getLong(pageAddr, sizeOff + SIZE_FIELD_SIZE + i) | PageUtils.getLong(pageAddr, sizeOff2 + SIZE_FIELD_SIZE + i);
 
                     newSize += Long.bitCount(newVal);
 
-                    PageUtils.putLong(buf, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
+                    PageUtils.putLong(pageAddr, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
                 }
 
                 for (; i < len; i ++) {
-                    byte newVal = (byte) (buf.get(sizeOff + SIZE_FIELD_SIZE + i) | buf.get(sizeOff2 + SIZE_FIELD_SIZE + i));
+                    byte newVal = (byte)(PageUtils.getByte(pageAddr, sizeOff + SIZE_FIELD_SIZE + i) | PageUtils.getByte(pageAddr, sizeOff2 + SIZE_FIELD_SIZE + i));
 
                     newSize += Integer.bitCount(newVal & 0xFF);
 
-                    PageUtils.putByte(buf, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
+                    PageUtils.putByte(pageAddr, sizeOff2 + SIZE_FIELD_SIZE + i, newVal);
                 }
 
-                PageUtils.putShort(buf, sizeOff2, (short)newSize);
+                PageUtils.putShort(pageAddr, sizeOff2, (short)newSize);
             }
 
-            buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag);
+            PageUtils.putLong(pageAddr, LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag);
 
-            PageHandler.zeroMemory(buf, sizeOff, len + SIZE_FIELD_SIZE);
+            PageHandler.zeroMemory(pageAddr, sizeOff, len + SIZE_FIELD_SIZE);
         }
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    long getLastSnapshotTag(ByteBuffer buf) {
-        return buf.getLong(LAST_SNAPSHOT_TAG_OFFSET);
+    long getLastSnapshotTag(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SNAPSHOT_TAG_OFFSET);
     }
 
     /**
      * Check that pageId was marked as changed between previous snapshot finish and current snapshot start.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Page id.
      * @param curSnapshotTag Snapshot tag.
      * @param pageSize Page size.
      */
-    public boolean wasChanged(ByteBuffer buf, long pageId, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
-        validateSnapshotId(buf, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize);
+    public boolean wasChanged(long pageAddr, long pageId, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
+        validateSnapshotId(pageAddr, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize);
 
-        if (countOfChangedPage(buf, curSnapshotTag, pageSize) < 1)
+        if (countOfChangedPage(pageAddr, curSnapshotTag, pageSize) < 1)
             return false;
 
         int cntOfPage = countOfPageToTrack(pageSize);
@@ -197,9 +197,9 @@ public class TrackingPageIO extends PageIO {
         byte byteToTest;
 
         if (useLeftHalf(curSnapshotTag))
-            byteToTest = buf.get(BITMAP_OFFSET + (idxToTest >> 3));
+            byteToTest = PageUtils.getByte(pageAddr, BITMAP_OFFSET + (idxToTest >> 3));
         else
-            byteToTest = PageUtils.getByte(buf, BITMAP_OFFSET + SIZE_FIELD_SIZE + ((idxToTest + cntOfPage) >> 3));
+            byteToTest = PageUtils.getByte(pageAddr, BITMAP_OFFSET + SIZE_FIELD_SIZE + ((idxToTest + cntOfPage) >> 3));
 
         int testTemplate = 1 << (idxToTest & 0b111);
 
@@ -207,22 +207,22 @@ public class TrackingPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param snapshotTag Snapshot tag.
      * @param pageSize Page size.
      *
      * @return count of pages which were marked as change for given snapshotTag
      */
-    public short countOfChangedPage(ByteBuffer buf, long snapshotTag, int pageSize) {
-        long dif = getLastSnapshotTag(buf) - snapshotTag;
+    public short countOfChangedPage(long pageAddr, long snapshotTag, int pageSize) {
+        long dif = getLastSnapshotTag(pageAddr) - snapshotTag;
 
         if (dif != 0 && dif != 1)
             return -1;
 
         if (useLeftHalf(snapshotTag))
-            return buf.getShort(SIZE_FIELD_OFFSET);
+            return PageUtils.getShort(pageAddr, SIZE_FIELD_OFFSET);
         else
-            return PageUtils.getShort(buf, BITMAP_OFFSET + (countOfPageToTrack(pageSize) >> 3));
+            return PageUtils.getShort(pageAddr, BITMAP_OFFSET + (countOfPageToTrack(pageSize) >> 3));
     }
 
     /**
@@ -262,14 +262,14 @@ public class TrackingPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param start Start.
      * @param curSnapshotTag Snapshot id.
      * @param pageSize Page size.
      * @return set pageId if it was changed or next closest one, if there is no changed page null will be returned
      */
-    public Long findNextChangedPage(ByteBuffer buf, long start, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
-        validateSnapshotId(buf, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize);
+    public Long findNextChangedPage(long pageAddr, long start, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
+        validateSnapshotId(pageAddr, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize);
 
         int cntOfPage = countOfPageToTrack(pageSize);
 
@@ -278,7 +278,7 @@ public class TrackingPageIO extends PageIO {
         if (start == trackingPage)
             return trackingPage;
 
-        if (countOfChangedPage(buf, curSnapshotTag, pageSize) <= 0)
+        if (countOfChangedPage(pageAddr, curSnapshotTag, pageSize) <= 0)
             return null;
 
         int idxToStartTest = (PageIdUtils.pageIndex(start) - COUNT_OF_EXTRA_PAGE) % cntOfPage;
@@ -292,7 +292,7 @@ public class TrackingPageIO extends PageIO {
         int stopIdx = zeroIdx + (cntOfPage >> 3);
 
         while (idx < stopIdx) {
-            byte byteToTest = PageUtils.getByte(buf, idx);
+            byte byteToTest = PageUtils.getByte(pageAddr, idx);
 
             if (byteToTest != 0) {
                 int foundSetBit;
@@ -302,7 +302,7 @@ public class TrackingPageIO extends PageIO {
                         PageIdUtils.flag(start),
                         PageIdUtils.pageIndex(trackingPage) + ((idx - zeroIdx) << 3) + foundSetBit);
 
-                    assert wasChanged(buf, foundPageId, curSnapshotTag, lastSuccessfulSnapshotTag, pageSize);
+                    assert wasChanged(pageAddr, foundPageId, curSnapshotTag, lastSuccessfulSnapshotTag, pageSize);
                     assert trackingPageFor(foundPageId, pageSize) == trackingPage;
 
                     return foundPageId;


[17/25] ignite git commit: 11810

Posted by sb...@apache.org.
11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 061a059265d30b2bd04083977522da2e037026c3
Parents: 22c1d1a
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 12:59:11 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 12:59:11 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/pagemem/Page.java    |  2 -
 .../ignite/internal/pagemem/PageMemory.java     |  7 ++
 .../ignite/internal/pagemem/PageUtils.java      | 81 ++++++++++++++++----
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |  5 ++
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  8 +-
 .../delta/DataPageInsertFragmentRecord.java     |  7 +-
 .../wal/record/delta/DataPageInsertRecord.java  |  6 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |  6 +-
 .../delta/DataPageSetFreeListPageRecord.java    |  6 +-
 .../wal/record/delta/FixCountRecord.java        |  6 +-
 .../record/delta/FixLeftmostChildRecord.java    |  6 +-
 .../pagemem/wal/record/delta/FixRemoveId.java   |  6 +-
 .../wal/record/delta/InitNewPageRecord.java     |  4 +-
 .../wal/record/delta/InnerReplaceRecord.java    |  2 +-
 .../pagemem/wal/record/delta/InsertRecord.java  |  4 +-
 .../pagemem/wal/record/delta/MergeRecord.java   |  2 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |  6 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |  6 +-
 .../wal/record/delta/MetaPageInitRecord.java    | 12 +--
 .../record/delta/MetaPageInitRootRecord.java    |  6 +-
 .../MetaPageUpdateCandidateAllocatedIndex.java  |  8 +-
 .../delta/MetaPageUpdateLastAllocatedIndex.java |  8 +-
 .../MetaPageUpdateLastSuccessfulBackupId.java   |  6 +-
 ...etaPageUpdateLastSuccessfulFullBackupId.java |  6 +-
 .../delta/MetaPageUpdateNextBackupId.java       |  6 +-
 .../MetaPageUpdatePartitionDataRecord.java      | 10 +--
 .../wal/record/delta/NewRootInitRecord.java     |  4 +-
 .../wal/record/delta/PageDeltaRecord.java       |  5 +-
 .../record/delta/PagesListAddPageRecord.java    |  6 +-
 .../delta/PagesListInitNewPageRecord.java       |  8 +-
 .../record/delta/PagesListRemovePageRecord.java |  6 +-
 .../record/delta/PagesListSetNextRecord.java    |  6 +-
 .../delta/PagesListSetPreviousRecord.java       |  6 +-
 .../pagemem/wal/record/delta/RecycleRecord.java |  4 +-
 .../pagemem/wal/record/delta/RemoveRecord.java  | 10 +--
 .../pagemem/wal/record/delta/ReplaceRecord.java |  6 +-
 .../record/delta/SplitExistingPageRecord.java   |  6 +-
 .../record/delta/SplitForwardPageRecord.java    |  2 +-
 .../record/delta/TrackingPageDeltaRecord.java   |  4 +-
 .../cache/database/CacheDataRowAdapter.java     | 15 ++--
 .../cache/database/freelist/FreeListImpl.java   |  2 +-
 .../cache/database/tree/io/DataPageIO.java      | 24 +++---
 42 files changed, 203 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/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
index 9b0d865..4f32a7b 100644
--- 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
@@ -85,8 +85,6 @@ public interface Page extends AutoCloseable {
      */
     public Boolean fullPageWalRecordPolicy();
 
-    public ByteBuffer pageBuffer();
-
     /**
      * Release page.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/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 27c090e..1954376 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.pagemem;
 
+import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.lifecycle.LifecycleAware;
@@ -62,4 +63,10 @@ public interface PageMemory extends LifecycleAware, PageIdAllocator {
      * @return Page size with system overhead, in bytes.
      */
     public int systemPageSize();
+
+    /**
+     * @param pageAddr Page address.
+     * @return Page byte buffer.
+     */
+    public ByteBuffer pageBuffer(long pageAddr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index c74fd63..e9e228a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -18,16 +18,12 @@
 package org.apache.ignite.internal.pagemem;
 
 import org.apache.ignite.internal.util.GridUnsafe;
-import sun.misc.Unsafe;
 
 /**
  *
  */
 @SuppressWarnings("deprecation")
 public class PageUtils {
-    /** */
-    private static final Unsafe unsafe = GridUnsafe.UNSAFE;
-
     /**
      * @param addr Start address. 
      * @param off Offset.
@@ -37,7 +33,7 @@ public class PageUtils {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getByte(addr + off);
+        return GridUnsafe.getByte(addr + off);
     }
 
     /**
@@ -53,11 +49,18 @@ public class PageUtils {
 
         byte[] bytes = new byte[len];
 
-        unsafe.copyMemory(null, addr + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
+        GridUnsafe.copyMemory(null, addr + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
 
         return bytes;
     }
 
+    /**
+     * @param srcAddr Source address.
+     * @param srcOff Source offset.
+     * @param dst Destination array.
+     * @param dstOff Destination offset.
+     * @param len Length.
+     */
     public static void getBytes(long srcAddr, int srcOff, byte[] dst, int dstOff, int len) {
         assert srcAddr > 0;
         assert srcOff > 0;
@@ -65,72 +68,118 @@ public class PageUtils {
         assert dstOff >= 0;
         assert len >= 0;
 
-        unsafe.copyMemory(null, srcAddr + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
+        GridUnsafe.copyMemory(null, srcAddr + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
     public static short getShort(long addr, int off) {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getShort(addr + off);
+        return GridUnsafe.getShort(addr + off);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
     public static int getInt(long addr, int off) {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getInt(addr + off);
+        return GridUnsafe.getInt(addr + off);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
     public static long getLong(long addr, int off) {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        return unsafe.getLong(addr + off);
+        return GridUnsafe.getLong(addr + off);
     }
 
+    /**
+     * @param addr Address/
+     * @param off Offset.
+     * @param bytes Bytes.
+     */
     public static void putBytes(long addr, int off, byte[] bytes) {
         assert addr > 0 : addr;
         assert off >= 0;
         assert bytes != null;
 
-        unsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, addr + off, bytes.length);
+        GridUnsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, addr + off, bytes.length);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param bytes Bytes array.
+     * @param bytesOff Bytes array offset.
+     */
     public static void putBytes(long addr, int off, byte[] bytes, int bytesOff) {
         assert addr > 0 : addr;
         assert off >= 0;
         assert bytes != null;
         assert bytesOff >= 0 && bytesOff < bytes.length : bytesOff;
 
-        unsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF + bytesOff, null, addr + off, bytes.length - bytesOff);
+        GridUnsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF + bytesOff, null, addr + off, bytes.length - bytesOff);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
     public static void putByte(long addr, int off, byte v) {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putByte(addr + off, v);
+        GridUnsafe.putByte(addr + off, v);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
     public static void putShort(long addr, int off, short v) {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putShort(addr + off, v);
+        GridUnsafe.putShort(addr + off, v);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
     public static void putInt(long addr, int off, int v) {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putInt(addr + off, v);
+        GridUnsafe.putInt(addr + off, v);
     }
 
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
     public static void putLong(long addr, int off, long v) {
         assert addr > 0 : addr;
         assert off >= 0;
 
-        unsafe.putLong(addr + off, v);
+        GridUnsafe.putLong(addr + off, v);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/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 3106866..409703e 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
@@ -205,6 +205,11 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /** {@inheritDoc} */
+    @Override public ByteBuffer pageBuffer(long pageAddr) {
+        return wrapPointer(pageAddr, pageSize());
+    }
+
+    /** {@inheritDoc} */
     @Override public long allocatePage(int cacheId, int partId, byte flags) {
         long relPtr = INVALID_REL_PTR;
         long absPtr = 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/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
index ac7ca93..5748acd 100644
--- 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
@@ -66,11 +66,9 @@ public class PageNoStoreImpl implements Page {
         this.noTagCheck = noTagCheck;
     }
 
-    /** {@inheritDoc} */
-    @Override public ByteBuffer pageBuffer() {
-        return pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
-    }
-
+    /**
+     * @return Data pointer.
+     */
     private long pointer() {
         return absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
index 96bf4ba..919bf04 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 
@@ -50,10 +49,10 @@ public class DataPageInsertFragmentRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.addRowFragment(buf, payload, lastLink, pageSize);
+        io.addRowFragment(pageAddr, payload, lastLink, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
index 66b07bb..1ba22d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
@@ -51,12 +51,12 @@ public class DataPageInsertRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
         assert payload != null;
 
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.addRow(buf, payload, pageSize);
+        io.addRow(pageAddr, payload, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
index 3a6023e..09f83b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
@@ -48,11 +48,11 @@ public class DataPageRemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize)
+    @Override public void applyDelta(long pageAddr, int pageSize)
         throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.removeRow(buf, itemId, pageSize);
+        io.removeRow(pageAddr, itemId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
index 09d1442..19878b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
@@ -49,10 +49,10 @@ public class DataPageSetFreeListPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.setFreeListPageId(buf, freeListPage);
+        io.setFreeListPageId(pageAddr, freeListPage);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
index e95fdf1..032f634 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
@@ -40,10 +40,10 @@ public class FixCountRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setCount(buf, cnt);
+        io.setCount(pageAddr, cnt);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
index c8a68d4..b345a43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
@@ -41,10 +41,10 @@ public class FixLeftmostChildRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        BPlusInnerIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        BPlusInnerIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setLeft(buf, 0, rightId);
+        io.setLeft(pageAddr, 0, rightId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
index b9febc9..0f8fe48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
@@ -41,11 +41,11 @@ public class FixRemoveId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize)
+    @Override public void applyDelta(long pageAddr, int pageSize)
         throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setRemoveId(buf, rmvId);
+        io.setRemoveId(pageAddr, rmvId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
index ac4dda0..ceff1b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
@@ -54,10 +54,10 @@ public class InitNewPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
         PageIO io = PageIO.getPageIO(ioType, ioVer);
 
-        io.initNewPage(buf, newPageId, pageSize);
+        io.initNewPage(pageAddr, newPageId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
index a9140ef..bad3fba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
@@ -57,7 +57,7 @@ public class InnerReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long dstBuf, int pageSize) throws IgniteCheckedException {
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
         throw new IgniteCheckedException("Inner replace record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
index 948f829..27ad5c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
@@ -72,8 +72,8 @@ public class InsertRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        io.insert(buf, idx, row, rowBytes, rightId);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        io.insert(pageAddr, idx, row, rowBytes, rightId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
index c3b5934..5932ee5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
@@ -63,7 +63,7 @@ public class MergeRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long leftBuf, int pageSize) throws IgniteCheckedException {
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
         throw new IgniteCheckedException("Merge record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
index 9f289e0..db82bcf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
@@ -39,10 +39,10 @@ public class MetaPageAddRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.addRoot(buf, rootId, pageSize);
+        io.addRoot(pageAddr, rootId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
index 1036c37..34d08c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
@@ -34,10 +34,10 @@ public class MetaPageCutRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.cutRoot(buf, pageSize);
+        io.cutRoot(pageAddr, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
index fecf80d..f5d3d06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
@@ -74,15 +74,15 @@ public class MetaPageInitRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
         PageMetaIO io = ioType == PageIO.T_META ?
-            PageMetaIO.VERSIONS.forPage(buf) :
-            PagePartitionMetaIO.VERSIONS.forPage(buf);
+            PageMetaIO.VERSIONS.forPage(pageAddr) :
+            PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.initNewPage(buf, newPageId, pageSize);
+        io.initNewPage(pageAddr, newPageId, pageSize);
 
-        io.setTreeRoot(buf, treeRoot);
-        io.setReuseListRoot(buf, reuseListRoot);
+        io.setTreeRoot(pageAddr, treeRoot);
+        io.setReuseListRoot(pageAddr, reuseListRoot);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
index 06595fd..1c29c8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
@@ -39,10 +39,10 @@ public class MetaPageInitRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.initRoot(buf, rootId, pageSize);
+        io.initRoot(pageAddr, rootId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
index e9e1792..3a19f63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateCandidateAllocatedIndex.java
@@ -38,12 +38,12 @@ public class MetaPageUpdateCandidateAllocatedIndex extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        assert PageIO.getType(buf) == PageIO.T_META || PageIO.getType(buf) == PageIO.T_PART_META;
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        assert PageIO.getType(pageAddr) == PageIO.T_META || PageIO.getType(pageAddr) == PageIO.T_PART_META;
 
-        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(buf));
+        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(pageAddr));
 
-        io.setCandidateAllocatedIndex(buf, candidateAllocatedIdx);
+        io.setCandidateAllocatedIndex(pageAddr, candidateAllocatedIdx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
index 727ed23..c505009 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
@@ -39,12 +39,12 @@ public class MetaPageUpdateLastAllocatedIndex extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        assert PageIO.getType(buf) == PageIO.T_META || PageIO.getType(buf) == PageIO.T_PART_META;
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        assert PageIO.getType(pageAddr) == PageIO.T_META || PageIO.getType(pageAddr) == PageIO.T_PART_META;
 
-        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(buf));
+        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(pageAddr));
 
-        io.setLastAllocatedIndex(buf, lastAllocatedIdx);
+        io.setLastAllocatedIndex(pageAddr, lastAllocatedIdx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
index f9b9e8d..8163a32 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulBackupId.java
@@ -37,10 +37,10 @@ public class MetaPageUpdateLastSuccessfulBackupId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setLastSuccessfulBackupId(buf, lastSuccessfulBackupId);
+        io.setLastSuccessfulBackupId(pageAddr, lastSuccessfulBackupId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
index c63788d..b7e6a54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullBackupId.java
@@ -37,10 +37,10 @@ public class MetaPageUpdateLastSuccessfulFullBackupId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setLastSuccessfulFullBackupId(buf, lastSuccessfulFullBackupId);
+        io.setLastSuccessfulFullBackupId(pageAddr, lastSuccessfulFullBackupId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
index 6cdf974..e0a5345 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextBackupId.java
@@ -37,10 +37,10 @@ public class MetaPageUpdateNextBackupId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setNextBackupId(buf, nextBackupId);
+        io.setNextBackupId(pageAddr, nextBackupId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
index f959761..39d038b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
@@ -78,12 +78,12 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setUpdateCounter(buf, updateCntr);
-        io.setGlobalRemoveId(buf, globalRmvId);
-        io.setSize(buf, partSize);
+        io.setUpdateCounter(pageAddr, updateCntr);
+        io.setGlobalRemoveId(pageAddr, globalRmvId);
+        io.setSize(pageAddr, partSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
index 756ad93..ce71ac7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
@@ -73,8 +73,8 @@ public class NewRootInitRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        io.initNewRoot(buf, newRootId, leftChildId, row, rowBytes, rightChildId, pageSize);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        io.initNewRoot(pageAddr, newRootId, leftChildId, row, rowBytes, rightChildId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
index 80668a1..9819232 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
@@ -62,10 +62,11 @@ public abstract class PageDeltaRecord extends WALRecord {
      * Apply changes from this delta to the given page.
      * It is assumed that the given buffer represents page state right before this update.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void applyDelta(long buf, int pageSize) throws IgniteCheckedException;
+    public abstract void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException;
 
     /** {@inheritDoc} */
     @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
index 208de66..38779f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
@@ -50,10 +50,10 @@ public class PagesListAddPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        int cnt = io.addPage(buf, dataPageId, pageSize);
+        int cnt = io.addPage(pageAddr, dataPageId, pageSize);
 
         assert cnt >= 0 : cnt;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
index c3ac7ea..66aea1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
@@ -72,14 +72,14 @@ public class PagesListInitNewPageRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
         PagesListNodeIO io = PageIO.getPageIO(PageIO.T_PAGE_LIST_NODE, ioVer);
 
-        io.initNewPage(buf, pageId(), pageSize);
-        io.setPreviousId(buf, prevPageId);
+        io.initNewPage(pageAddr, pageId(), pageSize);
+        io.setPreviousId(pageAddr, prevPageId);
 
         if (addDataPageId != 0L) {
-            int cnt = io.addPage(buf, addDataPageId, pageSize);
+            int cnt = io.addPage(pageAddr, addDataPageId, pageSize);
 
             assert cnt == 0 : cnt;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
index 95c0d47..d41eb09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
@@ -50,10 +50,10 @@ public class PagesListRemovePageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        boolean rmvd = io.removePage(buf, rmvdPageId);
+        boolean rmvd = io.removePage(pageAddr, rmvdPageId);
 
         assert rmvd;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
index 801d679..62f737a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
@@ -46,10 +46,10 @@ public class PagesListSetNextRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        io.setNextId(buf, nextPageId);
+        io.setNextId(pageAddr, nextPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
index bfac436..081bcb8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
@@ -46,10 +46,10 @@ public class PagesListSetPreviousRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        io.setPreviousId(buf, prevPageId);
+        io.setPreviousId(pageAddr, prevPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
index c40f014..6e75a5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
@@ -39,8 +39,8 @@ public class RecycleRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        PageIO.setPageId(buf, newPageId);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        PageIO.setPageId(pageAddr, newPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
index c9c8ea3..0a09ea0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
@@ -46,13 +46,13 @@ public class RemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        if (io.getCount(buf) != cnt)
-            throw new DeltaApplicationException("Count is wrong [expCnt=" + cnt + ", actual=" + io.getCount(buf) + ']');
+        if (io.getCount(pageAddr) != cnt)
+            throw new DeltaApplicationException("Count is wrong [expCnt=" + cnt + ", actual=" + io.getCount(pageAddr) + ']');
 
-        io.remove(buf, idx, cnt);
+        io.remove(pageAddr, idx, cnt);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
index 4d65121..266f208 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
@@ -55,12 +55,12 @@ public class ReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize)
+    @Override public void applyDelta(long pageAddr, int pageSize)
         throws IgniteCheckedException {
-        if (io.getCount(buf) < idx)
+        if (io.getCount(pageAddr) < idx)
             throw new DeltaApplicationException("Index is greater than count: " + idx);
 
-        io.store(buf, idx, row, rowBytes);
+        io.store(pageAddr, idx, row, rowBytes);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
index f9dfd37..2a43391 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
@@ -49,10 +49,10 @@ public class SplitExistingPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.splitExistingPage(buf, mid, fwdId);
+        io.splitExistingPage(pageAddr, mid, fwdId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
index f67e8d5..8fdd3d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
@@ -79,7 +79,7 @@ public class SplitForwardPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long fwdBuf, int pageSize) throws IgniteCheckedException {
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
         throw new IgniteCheckedException("Split forward page record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
index e95afbb..bc5082e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
@@ -70,8 +70,8 @@ public class TrackingPageDeltaRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
-        TrackingPageIO.VERSIONS.forPage(buf).markChanged(buf, pageIdToMark, nextBackupId, lastSuccessfulBackupId, pageSize);
+    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
+        TrackingPageIO.VERSIONS.forPage(pageAddr).markChanged(pageAddr, pageIdToMark, nextBackupId, lastSuccessfulBackupId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/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 3f5b03a..d940d62 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
@@ -21,6 +21,7 @@ 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;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -90,6 +91,8 @@ public class CacheDataRowAdapter implements CacheDataRow {
         boolean first = true;
 
         do {
+            PageMemory pageMem = cctx.shared().database().pageMemory();
+
             try (Page page = page(pageId(nextLink), cctx)) {
                 long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
 
@@ -100,7 +103,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
                     DataPagePayload data = io.readPayload(pageAddr,
                         itemId(nextLink),
-                        cctx.shared().database().pageMemory().pageSize());
+                        pageMem.pageSize());
 
                     nextLink = data.nextLink();
 
@@ -115,12 +118,12 @@ public class CacheDataRowAdapter implements CacheDataRow {
                         first = false;
                     }
 
-                    ByteBuffer buf0 = page.pageBuffer();
+                    ByteBuffer buf = pageMem.pageBuffer(pageAddr);
 
-                    buf0.position(data.offset());
-                    buf0.limit(data.offset() + data.payloadSize());
+                    buf.position(data.offset());
+                    buf.limit(data.offset() + data.payloadSize());
 
-                    incomplete = readFragment(coctx, buf0, keyOnly, incomplete);
+                    incomplete = readFragment(coctx, buf, keyOnly, incomplete);
 
                     if (keyOnly && key != null)
                         return;
@@ -293,6 +296,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param buf Buffer.
      * @param incomplete Incomplete object.
      * @return Incomplete object.
+     * @throws IgniteCheckedException If failed.
      */
     private IncompleteObject<?> readIncompleteExpireTime(
         ByteBuffer buf,
@@ -336,6 +340,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param buf Buffer.
      * @param incomplete Incomplete object.
      * @return Incomplete object.
+     * @throws IgniteCheckedException If failed.
      */
     private IncompleteObject<?> readIncompleteVersion(
         ByteBuffer buf,

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/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 8cee71c..6c1b21b 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
@@ -157,7 +157,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 // Read last link before the fragment write, because it will be updated there.
                 long lastLink = row.link();
 
-                int payloadSize = io.addRowFragment(buf, row, written, rowSize, pageSize());
+                int payloadSize = io.addRowFragment(pageMem, buf, row, written, rowSize, pageSize());
 
                 assert payloadSize > 0 : payloadSize;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/061a0592/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index d45c326..548e300 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -30,8 +30,6 @@ import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.typedef.internal.SB;
-import sun.misc.JavaNioAccess;
-import sun.misc.SharedSecrets;
 
 /**
  * Data pages IO.
@@ -845,6 +843,7 @@ public class DataPageIO extends PageIO {
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
+     * @param pageMem Page memory.
      * @param pageAddr Page address.
      * @param row Cache data row.
      * @param written Number of bytes of row size that was already written.
@@ -854,13 +853,14 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public int addRowFragment(
+        PageMemory pageMem,
         long pageAddr,
         CacheDataRow row,
         int written,
         int rowSize,
         int pageSize
     ) throws IgniteCheckedException {
-        return addRowFragment(pageAddr, written, rowSize, row.link(), row, null, pageSize);
+        return addRowFragment(pageMem, pageAddr, written, rowSize, row.link(), row, null, pageSize);
     }
 
     /**
@@ -878,15 +878,13 @@ public class DataPageIO extends PageIO {
         long lastLink,
         int pageSize
     ) throws IgniteCheckedException {
-        addRowFragment(pageAddr, 0, 0, lastLink, null, payload, pageSize);
+        addRowFragment(null, pageAddr, 0, 0, lastLink, null, payload, pageSize);
     }
 
-    // TODO GG-11810.
-    private JavaNioAccess nioAccess = SharedSecrets.getJavaNioAccess();
-
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
+     * @param pageMem Page memory.
      * @param pageAddr Page address.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
@@ -898,6 +896,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     private int addRowFragment(
+        PageMemory pageMem,
         long pageAddr,
         int written,
         int rowSize,
@@ -918,19 +917,18 @@ public class DataPageIO extends PageIO {
         int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         if (payload == null) {
-            ByteBuffer buf0 = nioAccess.newDirectByteBuffer(pageAddr, pageSize, null);
-            buf0.order(PageMemory.NATIVE_BYTE_ORDER);
+            ByteBuffer buf = pageMem.pageBuffer(pageAddr);
 
-            buf0.position(dataOff);
+            buf.position(dataOff);
 
             short p = (short)(payloadSize | FRAGMENTED_FLAG);
 
-            buf0.putShort(p);
-            buf0.putLong(lastLink);
+            buf.putShort(p);
+            buf.putLong(lastLink);
 
             int rowOff = rowSize - written - payloadSize;
 
-            writeFragmentData(row, buf0, rowOff, payloadSize);
+            writeFragmentData(row, buf, rowOff, payloadSize);
         }
         else {
             PageUtils.putShort(pageAddr, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));


[16/25] ignite git commit: 11810

Posted by sb...@apache.org.
11810


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

Branch: refs/heads/ignite-gg-11810
Commit: 22c1d1a2090c8be6530c726b428b1b50bf6f6d94
Parents: c41f751
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 12:28:32 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 12:28:32 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/pagemem/Page.java    |   7 -
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  10 -
 .../wal/record/delta/MetaPageAddRootRecord.java |   2 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |   2 +-
 .../record/delta/MetaPageInitRootRecord.java    |   2 +-
 .../cache/database/CacheDataRowAdapter.java     |  33 +-
 .../cache/database/freelist/FreeListImpl.java   |   6 +-
 .../cache/database/freelist/PagesList.java      |  10 +-
 .../cache/database/tree/BPlusTree.java          |  30 +-
 .../cache/database/tree/io/BPlusIO.java         |  40 +-
 .../cache/database/tree/io/BPlusInnerIO.java    |  20 +-
 .../cache/database/tree/io/BPlusMetaIO.java     |  45 +-
 .../cache/database/tree/io/DataPageIO.java      | 420 ++++++++++---------
 .../cache/database/tree/io/DataPagePayload.java |  14 +
 .../cache/database/tree/io/PageIO.java          |  84 ++--
 .../cache/database/tree/io/PageMetaIO.java      | 102 ++---
 .../database/tree/io/PagePartitionMetaIO.java   |  62 +--
 .../cache/database/tree/util/PageHandler.java   |  16 +-
 18 files changed, 461 insertions(+), 444 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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
index eef61c8..9b0d865 100644
--- 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
@@ -39,11 +39,6 @@ public interface Page extends AutoCloseable {
     public FullPageId fullId();
 
     /**
-     * @return ByteBuffer for modifying the page.
-     */
-    public ByteBuffer getForRead();
-
-    /**
      * @return Pointer for modifying the page.
      */
     public long getForReadPointer();
@@ -90,8 +85,6 @@ public interface Page extends AutoCloseable {
      */
     public Boolean fullPageWalRecordPolicy();
 
-    public int size();
-
     public ByteBuffer pageBuffer();
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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
index 2fa1caf..ac7ca93 100644
--- 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
@@ -86,11 +86,6 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer getForRead() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public long getForReadPointer() {
         if (pageMem.readLockPage(absPtr, PageIdUtils.tag(pageId)))
             return pointer();
@@ -154,11 +149,6 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
-    @Override public int size() {
-        return pageMem.pageSize();
-    }
-
-    /** {@inheritDoc} */
     @Override public void close() {
         pageMem.releasePage(this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
index 19d8140..9f289e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
@@ -42,7 +42,7 @@ public class MetaPageAddRootRecord extends PageDeltaRecord {
     @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
-        io.addRoot(0, buf, rootId);
+        io.addRoot(buf, rootId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
index a2957e5..1036c37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
@@ -37,7 +37,7 @@ public class MetaPageCutRootRecord extends PageDeltaRecord {
     @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
-        io.cutRoot(0, buf);
+        io.cutRoot(buf, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
index e67e974..06595fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
@@ -42,7 +42,7 @@ public class MetaPageInitRootRecord extends PageDeltaRecord {
     @Override public void applyDelta(long buf, int pageSize) throws IgniteCheckedException {
         BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
 
-        io.initRoot(0, buf, rootId);
+        io.initRoot(buf, rootId, pageSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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 c989eee..3f5b03a 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
@@ -91,21 +91,23 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
         do {
             try (Page page = page(pageId(nextLink), cctx)) {
-                long buf = page.getForReadPointer(); // Non-empty data page must not be recycled.
+                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
 
-                assert buf != 0L : nextLink;
+                assert pageAddr != 0L : nextLink;
 
                 try {
-                    DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+                    DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-                    DataPagePayload data = io.readPayload(buf, itemId(nextLink), page.size());
+                    DataPagePayload data = io.readPayload(pageAddr,
+                        itemId(nextLink),
+                        cctx.shared().database().pageMemory().pageSize());
 
                     nextLink = data.nextLink();
 
                     if (first) {
                         if (nextLink == 0) {
                             // Fast path for a single page row.
-                            readFullRow(coctx, buf + data.offset(), keyOnly);
+                            readFullRow(coctx, pageAddr + data.offset(), keyOnly);
 
                             return;
                         }
@@ -139,6 +141,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param keyOnly {@code true} If need to read only key object.
      * @param incomplete Incomplete object.
      * @throws IgniteCheckedException If failed.
+     * @return Read object.
      */
     private IncompleteObject<?> readFragment(
         CacheObjectContext coctx,
@@ -184,23 +187,23 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
     /**
      * @param coctx Cache object context.
-     * @param buf Buffer.
+     * @param addr Address.
      * @param keyOnly {@code true} If need to read only key object.
      * @throws IgniteCheckedException If failed.
      */
-    private void readFullRow(CacheObjectContext coctx, long buf, boolean keyOnly) throws IgniteCheckedException {
+    private void readFullRow(CacheObjectContext coctx, long addr, boolean keyOnly) throws IgniteCheckedException {
         int off = 0;
 
-        int len = PageUtils.getInt(buf, off);
+        int len = PageUtils.getInt(addr, off);
         off += 4;
 
         if (len == 0)
             key = null;
         else {
-            byte type = PageUtils.getByte(buf, off);
+            byte type = PageUtils.getByte(addr, off);
             off++;
 
-            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            byte[] bytes = PageUtils.getBytes(addr, off, len);
             off += len;
 
             key = coctx.processor().toKeyCacheObject(coctx, type, bytes);
@@ -212,26 +215,26 @@ public class CacheDataRowAdapter implements CacheDataRow {
             return;
         }
 
-        len = PageUtils.getInt(buf, off);
+        len = PageUtils.getInt(addr, off);
         off += 4;
 
         if (len == 0)
             val = null;
         else {
-            byte type = PageUtils.getByte(buf, off);
+            byte type = PageUtils.getByte(addr, off);
             off++;
 
-            byte[] bytes = PageUtils.getBytes(buf, off, len);
+            byte[] bytes = PageUtils.getBytes(addr, off, len);
             off += len;
 
             val = coctx.processor().toCacheObject(coctx, type, bytes);
         }
 
-        ver = CacheVersionIO.read(buf + off, false);
+        ver = CacheVersionIO.read(addr + off, false);
 
         off += CacheVersionIO.size(ver, false);
 
-        expireTime = PageUtils.getLong(buf, off);
+        expireTime = PageUtils.getLong(addr, off);
 
         assert isReady(): "ready";
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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 5b3e781..8cee71c 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
@@ -325,7 +325,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 // If it is an existing page, we do not need to initialize it.
                 DataPageIO init = reuseBucket || pageId == 0L ? DataPageIO.VERSIONS.latest() : null;
 
-                written = writePage(page, this, writeRow, init, wal, row, written, FAIL_I);
+                written = writePage(pageMem, page, this, writeRow, init, wal, row, written, FAIL_I);
 
                 assert written != FAIL_I; // We can't fail here.
             }
@@ -343,7 +343,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         long nextLink;
 
         try (Page page = pageMem.page(cacheId, pageId)) {
-            nextLink = writePage(page, this, rmvRow, null, itemId, FAIL_L);
+            nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
 
             assert nextLink != FAIL_L; // Can't fail here.
         }
@@ -353,7 +353,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
             pageId = PageIdUtils.pageId(nextLink);
 
             try (Page page = pageMem.page(cacheId, pageId)) {
-                nextLink = writePage(page, this, rmvRow, null, itemId, FAIL_L);
+                nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
 
                 assert nextLink != FAIL_L; // Can't fail here.
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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 9e7f9ff..f33818a 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
@@ -139,7 +139,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 {
@@ -338,7 +338,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);
@@ -855,7 +855,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;
                                 }
@@ -874,7 +874,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;
                             }
@@ -1012,7 +1012,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.
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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 4a7a68a..56bebbc 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
@@ -527,7 +527,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getRootLevel(pageAddr); // Can drop only root.
 
-            io.cutRoot(pageSize(), pageAddr);
+            io.cutRoot(pageAddr, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
@@ -547,7 +547,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
             assert lvl == io.getLevelsCount(pageAddr);
 
-            io.addRoot(pageSize(), pageAddr, rootPageId);
+            io.addRoot(pageAddr, rootPageId, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
@@ -565,7 +565,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            io.initRoot(pageSize(), pageAddr, rootId);
+            io.initRoot(pageAddr, rootId, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageInitRootRecord(cacheId, meta.id(), rootId));
@@ -643,12 +643,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         long rootId = allocatePage(null);
 
         try (Page root = page(rootId)) {
-            initPage(root, this, latestLeafIO(), wal);
+            initPage(pageMem, root, this, latestLeafIO(), wal);
         }
 
         // Initialize meta page with new root page.
         try (Page meta = page(metaPageId)) {
-            Bool res = writePage(meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, 0, FALSE);
+            Bool res = writePage(pageMem, meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, 0, FALSE);
 
             assert res == TRUE: res;
         }
@@ -1806,7 +1806,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                             p.fwdId = fwdId;
                             p.pageId = pageId;
 
-                            res = writePage(page, this, replace, p, lvl, RETRY);
+                            res = writePage(pageMem, page, this, replace, p, lvl, RETRY);
 
                             // Restore args.
                             p.pageId = oldPageId;
@@ -1836,7 +1836,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                         p.pageId = pageId;
                         p.fwdId = fwdId;
 
-                        return writePage(page, this, replace, p, lvl, RETRY);
+                        return writePage(pageMem, page, this, replace, p, lvl, RETRY);
 
                     case NOT_FOUND: // Do insert.
                         assert lvl == p.btmLvl : "must insert at the bottom level";
@@ -1846,7 +1846,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                         p.pageId = pageId;
                         p.fwdId = fwdId;
 
-                        return writePage(page, this, insert, p, lvl, RETRY);
+                        return writePage(pageMem, page, this, insert, p, lvl, RETRY);
 
                     default:
                         return res;
@@ -2307,7 +2307,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                             }
                         }
 
-                        Bool res = writePage(meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
+                        Bool res = writePage(pageMem, meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
 
                         assert res == TRUE: res;
 
@@ -2692,7 +2692,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             Page back = page(backId);
 
             try {
-                return writePage(back, BPlusTree.this, lockBackAndRmvFromLeaf, this, 0, RETRY);
+                return writePage(pageMem, back, BPlusTree.this, lockBackAndRmvFromLeaf, this, 0, RETRY);
             }
             finally {
                 if (canRelease(back, 0))
@@ -2707,7 +2707,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         private Result doRemoveFromLeaf() throws IgniteCheckedException {
             assert page != null;
 
-            return writePage(page, BPlusTree.this, rmvFromLeaf, this, 0, RETRY);
+            return writePage(pageMem, page, BPlusTree.this, rmvFromLeaf, this, 0, RETRY);
         }
 
         /**
@@ -2718,7 +2718,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         private Result doLockTail(int lvl) throws IgniteCheckedException {
             assert page != null;
 
-            return writePage(page, BPlusTree.this, lockTail, this, lvl, RETRY);
+            return writePage(pageMem, page, BPlusTree.this, lockTail, this, lvl, RETRY);
         }
 
         /**
@@ -2746,7 +2746,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             Page back = page(backId);
 
             try {
-                return writePage(back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
+                return writePage(pageMem, back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
             }
             finally {
                 if (canRelease(back, lvl))
@@ -2766,7 +2766,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             Page fwd = page(fwdId);
 
             try {
-                return writePage(fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
+                return writePage(pageMem, fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
             }
             finally {
                 // If we were not able to lock forward page as tail, release the page.
@@ -2999,7 +2999,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @throws IgniteCheckedException If failed.
          */
         private void cutRoot(int lvl) throws IgniteCheckedException {
-            Bool res = writePage(meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
+            Bool res = writePage(pageMem, meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
 
             assert res == TRUE: res;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
index ff3efb8..0cf1385 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
@@ -222,7 +222,7 @@ public abstract class BPlusIO<L> extends PageIO {
     public abstract L getLookupRow(BPlusTree<L, ?> tree, long pageAddr, int idx) throws IgniteCheckedException;
 
     /**
-     * Copy items from source buffer to destination buffer.
+     * Copy items from source page to destination page.
      * Both pages must be of the same type and the same version.
      *
      * @param srcPageAddr Source page address.
@@ -239,54 +239,54 @@ public abstract class BPlusIO<L> extends PageIO {
     // Methods for B+Tree logic.
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param row Row to insert.
      * @param rowBytes Row bytes.
      * @param rightId Page ID which will be to the right child for the inserted item.
      * @throws IgniteCheckedException If failed.
      */
-    public void insert(long buf, int idx, L row, byte[] rowBytes, long rightId)
+    public void insert(long pageAddr, int idx, L row, byte[] rowBytes, long rightId)
         throws IgniteCheckedException {
-        int cnt = getCount(buf);
+        int cnt = getCount(pageAddr);
 
         // Move right all the greater elements to make a free slot for a new row link.
-        copyItems(buf, buf, idx, idx + 1, cnt - idx, false);
+        copyItems(pageAddr, pageAddr, idx, idx + 1, cnt - idx, false);
 
-        setCount(buf, cnt + 1);
+        setCount(pageAddr, cnt + 1);
 
-        store(buf, idx, row, rowBytes);
+        store(pageAddr, idx, row, rowBytes);
     }
 
     /**
-     * @param buf Splitting buffer.
+     * @param pageAddr Splitting page address.
      * @param fwdId Forward page ID.
-     * @param fwdBuf Forward buffer.
+     * @param fwdPageAddr Forward page address.
      * @param mid Bisection index.
      * @param cnt Initial elements count in the page being split.
      * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void splitForwardPage(
-        long buf,
+        long pageAddr,
         long fwdId,
-        long fwdBuf,
+        long fwdPageAddr,
         int mid,
         int cnt,
         int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(fwdBuf, fwdId, pageSize);
+        initNewPage(fwdPageAddr, fwdId, pageSize);
 
         cnt -= mid;
 
-        copyItems(buf, fwdBuf, mid, 0, cnt, true);
+        copyItems(pageAddr, fwdPageAddr, mid, 0, cnt, true);
 
-        setCount(fwdBuf, cnt);
-        setForward(fwdBuf, getForward(buf));
+        setCount(fwdPageAddr, cnt);
+        setForward(fwdPageAddr, getForward(pageAddr));
 
         // Copy remove ID to make sure that if inner remove touched this page, then retry
         // will happen even for newly allocated forward page.
-        setRemoveId(fwdBuf, getRemoveId(buf));
+        setRemoveId(fwdPageAddr, getRemoveId(pageAddr));
     }
 
     /**
@@ -374,11 +374,11 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
-     * @param pos Position in buffer.
+     * @param pageAddr Page address.
+     * @param pos Position in page.
      * @param bytes Bytes.
      */
-    private static void putBytes(long buf, int pos, byte[] bytes) {
-        PageUtils.putBytes(buf, pos, bytes);
+    private static void putBytes(long pageAddr, int pos, byte[] bytes) {
+        PageUtils.putBytes(pageAddr, pos, bytes);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
index cac7f05..c7018bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
@@ -131,20 +131,20 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
 
     /** {@inheritDoc} */
     @Override public void insert(
-        long buf,
+        long pageAddr,
         int idx,
         L row,
         byte[] rowBytes,
         long rightId
     ) throws IgniteCheckedException {
-        super.insert(buf, idx, row, rowBytes, rightId);
+        super.insert(pageAddr, idx, row, rowBytes, rightId);
 
         // Setup reference to the right page on split.
-        setRight(buf, idx, rightId);
+        setRight(pageAddr, idx, rightId);
     }
 
     /**
-     * @param newRootBuf New root buffer.
+     * @param newRootPageAddr New root page address.
      * @param newRootId New root ID.
      * @param leftChildId Left child ID.
      * @param row Moved up row.
@@ -154,7 +154,7 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
      * @throws IgniteCheckedException If failed.
      */
     public void initNewRoot(
-        long newRootBuf,
+        long newRootPageAddr,
         long newRootId,
         long leftChildId,
         L row,
@@ -162,11 +162,11 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
         long rightChildId,
         int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(newRootBuf, newRootId, pageSize);
+        initNewPage(newRootPageAddr, newRootId, pageSize);
 
-        setCount(newRootBuf, 1);
-        setLeft(newRootBuf, 0, leftChildId);
-        store(newRootBuf, 0, row, rowBytes);
-        setRight(newRootBuf, 0, rightChildId);
+        setCount(newRootPageAddr, 1);
+        setLeft(newRootPageAddr, 0, leftChildId);
+        store(newRootPageAddr, 0, row, rowBytes);
+        setRight(newRootPageAddr, 0, rightChildId);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
index ccaa952..700da39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
@@ -37,45 +37,48 @@ public class BPlusMetaIO extends PageIO {
     /**
      * @param ver Page format version.
      */
-    protected BPlusMetaIO(int ver) {
+    private BPlusMetaIO(int ver) {
         super(T_BPLUS_META, ver);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAdrr Page address.
      * @param rootId Root page ID.
+     * @param pageSize Page size.
      */
-    public void initRoot(int pageSize, long buf, long rootId) {
-        setLevelsCount(pageSize, buf, 1);
-        setFirstPageId(buf, 0, rootId);
+    public void initRoot(long pageAdrr, long rootId, int pageSize) {
+        setLevelsCount(pageAdrr, 1, pageSize);
+        setFirstPageId(pageAdrr, 0, rootId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Number of levels in this tree.
      */
-    public int getLevelsCount(long buf) {
-        return PageUtils.getByte(buf, LVLS_OFF);
+    public int getLevelsCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, LVLS_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return Max levels possible for this page size.
      */
-    public int getMaxLevels(int pageSize, long buf) {
+    private int getMaxLevels(long pageAddr, int pageSize) {
         return (pageSize - REFS_OFF) / 8;
     }
 
     /**
-     * @param buf  Buffer.
+     * @param pageAddr Page address.
      * @param lvls Number of levels in this tree.
+     * @param pageSize Page size.
      */
-    public void setLevelsCount(int pageSize, long buf, int lvls) {
-        assert lvls >= 0 && lvls <= getMaxLevels(pageSize, buf) : lvls;
+    private void setLevelsCount(long pageAddr, int lvls, int pageSize) {
+        assert lvls >= 0 && lvls <= getMaxLevels(pageAddr, pageSize) : lvls;
 
-        PageUtils.putByte(buf, LVLS_OFF, (byte)lvls);
+        PageUtils.putByte(pageAddr, LVLS_OFF, (byte)lvls);
 
-        assert getLevelsCount(buf) == lvls;
+        assert getLevelsCount(pageAddr) == lvls;
     }
 
     /**
@@ -100,7 +103,7 @@ public class BPlusMetaIO extends PageIO {
      * @param lvl    Level.
      * @param pageId Page ID.
      */
-    public void setFirstPageId(long buf, int lvl, long pageId) {
+    private void setFirstPageId(long buf, int lvl, long pageId) {
         assert lvl >= 0 && lvl < getLevelsCount(buf);
 
         PageUtils.putLong(buf, offset(lvl), pageId);
@@ -123,20 +126,22 @@ public class BPlusMetaIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param rootPageId New root page ID.
+     * @param pageSize Page size.
      */
-    public void addRoot(int pageSize, long buf, long rootPageId) {
+    public void addRoot(long buf, long rootPageId, int pageSize) {
         int lvl = getLevelsCount(buf);
 
-        setLevelsCount(pageSize, buf, lvl + 1);
+        setLevelsCount(buf, lvl + 1, pageSize);
         setFirstPageId(buf, lvl, rootPageId);
     }
 
     /**
      * @param buf Buffer.
+     * @param pageSize Page size.
      */
-    public void cutRoot(int pageSize, long buf) {
+    public void cutRoot(long buf, int pageSize) {
         int lvl = getRootLevel(buf);
 
-        setLevelsCount(pageSize, buf, lvl); // Decrease tree height.
+        setLevelsCount(buf, lvl, pageSize); // Decrease tree height.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index c365063..d45c326 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -92,48 +92,48 @@ public class DataPageIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(long buf, long pageId, int pageSize) {
-        super.initNewPage(buf, pageId, pageSize);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setEmptyPage(buf, pageSize);
-        setFreeListPageId(buf, 0L);
+        setEmptyPage(pageAddr, pageSize);
+        setFreeListPageId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageSize Page size.
      */
-    private void setEmptyPage(long buf, int pageSize) {
-        setDirectCount(buf, 0);
-        setIndirectCount(buf, 0);
-        setFirstEntryOffset(buf, pageSize, pageSize);
-        setRealFreeSpace(buf, pageSize - ITEMS_OFF, pageSize);
+    private void setEmptyPage(long pageAddr, int pageSize) {
+        setDirectCount(pageAddr, 0);
+        setIndirectCount(pageAddr, 0);
+        setFirstEntryOffset(pageAddr, pageSize, pageSize);
+        setRealFreeSpace(pageAddr, pageSize - ITEMS_OFF, pageSize);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param freeListPageId Free list page ID.
      */
-    public void setFreeListPageId(long buf, long freeListPageId) {
-        PageUtils.putLong(buf, FREE_LIST_PAGE_ID_OFF, freeListPageId);
+    public void setFreeListPageId(long pageAddr, long freeListPageId) {
+        PageUtils.putLong(pageAddr, FREE_LIST_PAGE_ID_OFF, freeListPageId);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @return Free list page ID.
      */
-    public long getFreeListPageId(long buf) {
-        return PageUtils.getLong(buf, FREE_LIST_PAGE_ID_OFF);
+    public long getFreeListPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, FREE_LIST_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param show What elements of data page entry to show in the result.
      * @return Data page entry size.
      */
-    private int getPageEntrySize(long buf, int dataOff, int show) {
-        int payloadLen = PageUtils.getShort(buf, dataOff) & 0xFFFF;
+    private int getPageEntrySize(long pageAddr, int dataOff, int show) {
+        int payloadLen = PageUtils.getShort(pageAddr, dataOff) & 0xFFFF;
 
         if ((payloadLen & FRAGMENTED_FLAG) != 0)
             payloadLen &= ~FRAGMENTED_FLAG; // We are fragmented and have a link.
@@ -166,47 +166,47 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Entry data offset.
      * @param pageSize Page size.
      */
-    private void setFirstEntryOffset(long buf, int dataOff, int pageSize) {
+    private void setFirstEntryOffset(long pageAddr, int dataOff, int pageSize) {
         assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= pageSize : dataOff;
 
-        PageUtils.putShort(buf, FIRST_ENTRY_OFF, (short)dataOff);
+        PageUtils.putShort(pageAddr, FIRST_ENTRY_OFF, (short)dataOff);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Entry data offset.
      */
-    private int getFirstEntryOffset(long buf) {
-        return PageUtils.getShort(buf, FIRST_ENTRY_OFF) & 0xFFFF;
+    private int getFirstEntryOffset(long pageAddr) {
+        return PageUtils.getShort(pageAddr, FIRST_ENTRY_OFF) & 0xFFFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param freeSpace Free space.
      * @param pageSize Page size.
      */
-    private void setRealFreeSpace(long buf, int freeSpace, int pageSize) {
-        assert freeSpace == actualFreeSpace(buf, pageSize) : freeSpace + " != " + actualFreeSpace(buf, pageSize);
+    private void setRealFreeSpace(long pageAddr, int freeSpace, int pageSize) {
+        assert freeSpace == actualFreeSpace(pageAddr, pageSize) : freeSpace + " != " + actualFreeSpace(pageAddr, pageSize);
 
-        PageUtils.putShort(buf, FREE_SPACE_OFF, (short)freeSpace);
+        PageUtils.putShort(pageAddr, FREE_SPACE_OFF, (short)freeSpace);
     }
 
     /**
      * Free space refers to a "max row size (without any data page specific overhead) which is
      * guaranteed to fit into this data page".
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Free space.
      */
-    public int getFreeSpace(long buf) {
-        if (getFreeItemSlots(buf) == 0)
+    public int getFreeSpace(long pageAddr) {
+        if (getFreeItemSlots(pageAddr) == 0)
             return 0;
 
-        int freeSpace = getRealFreeSpace(buf);
+        int freeSpace = getRealFreeSpace(pageAddr);
 
         // We reserve size here because of getFreeSpace() method semantics (see method javadoc).
         // It means that we must be able to accommodate a row of size which is equal to getFreeSpace(),
@@ -218,48 +218,49 @@ public class DataPageIO extends PageIO {
     }
 
     /**
+     * @param pageAddr Page address.
      * @return {@code true} If there is no useful data in this page.
      */
-    public boolean isEmpty(long buf) {
-        return getDirectCount(buf) == 0;
+    public boolean isEmpty(long pageAddr) {
+        return getDirectCount(pageAddr) == 0;
     }
 
     /**
      * Equivalent for {@link #actualFreeSpace(long, int)} but reads saved value.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Free space.
      */
-    private int getRealFreeSpace(long buf) {
-        return PageUtils.getShort(buf, FREE_SPACE_OFF);
+    private int getRealFreeSpace(long pageAddr) {
+        return PageUtils.getShort(pageAddr, FREE_SPACE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Direct count.
      */
-    private void setDirectCount(long buf, int cnt) {
+    private void setDirectCount(long pageAddr, int cnt) {
         assert checkCount(cnt): cnt;
 
-        PageUtils.putByte(buf, DIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(pageAddr, DIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Direct count.
      */
-    private int getDirectCount(long buf) {
-        return PageUtils.getByte(buf, DIRECT_CNT_OFF) & 0xFF;
+    private int getDirectCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, DIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Indirect count.
      */
-    private void setIndirectCount(long buf, int cnt) {
+    private void setIndirectCount(long pageAddr, int cnt) {
         assert checkCount(cnt): cnt;
 
-        PageUtils.putByte(buf, INDIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(pageAddr, INDIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
@@ -279,36 +280,36 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Indirect count.
      */
-    private int getIndirectCount(long buf) {
-        return PageUtils.getByte(buf, INDIRECT_CNT_OFF) & 0xFF;
+    private int getIndirectCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, INDIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Number of free entry slots.
      */
-    private int getFreeItemSlots(long buf) {
-        return 0xFF - getDirectCount(buf);
+    private int getFreeItemSlots(long pageAddr) {
+        return 0xFF - getDirectCount(pageAddr);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return Found index of indirect item.
      */
-    private int findIndirectItemIndex(long buf, int itemId, int directCnt, int indirectCnt) {
+    private int findIndirectItemIndex(long pageAddr, int itemId, int directCnt, int indirectCnt) {
         int low = directCnt;
         int high = directCnt + indirectCnt - 1;
 
         while (low <= high) {
             int mid = (low + high) >>> 1;
 
-            int cmp = Integer.compare(itemId(getItem(buf, mid)), itemId);
+            int cmp = Integer.compare(itemId(getItem(pageAddr, mid)), itemId);
 
             if (cmp < 0)
                 low = mid + 1;
@@ -322,20 +323,20 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageSize Page size.
      * @return String representation.
      */
-    public String printPageLayout(long buf, int pageSize) {
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
-        int free = getRealFreeSpace(buf);
+    private String printPageLayout(long pageAddr, int pageSize) {
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
+        int free = getRealFreeSpace(pageAddr);
 
         boolean valid = directCnt >= indirectCnt;
 
         SB b = new SB();
 
-        b.appendHex(PageIO.getPageId(buf)).a(" [");
+        b.appendHex(PageIO.getPageId(pageAddr)).a(" [");
 
         int entriesSize = 0;
 
@@ -343,12 +344,12 @@ public class DataPageIO extends PageIO {
             if (i != 0)
                 b.a(", ");
 
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
             if (item < ITEMS_OFF || item >= pageSize)
                 valid = false;
 
-            entriesSize += getPageEntrySize(buf, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            entriesSize += getPageEntrySize(pageAddr, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             b.a(item);
         }
@@ -361,7 +362,7 @@ public class DataPageIO extends PageIO {
             if (i != directCnt)
                 b.a(", ");
 
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
             int itemId = itemId(item);
             int directIdx = directItemIndex(item);
@@ -374,7 +375,7 @@ public class DataPageIO extends PageIO {
             if (itemId < directCnt || directIdx < 0 || directIdx >= directCnt)
                 valid = false;
 
-            if (i > directCnt && itemId(getItem(buf, i - 1)) >= itemId)
+            if (i > directCnt && itemId(getItem(pageAddr, i - 1)) >= itemId)
                 valid = false;
 
 
@@ -399,71 +400,72 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @param pageSize Page size.
      * @return Data entry offset in bytes.
      */
-    private int getDataOffset(long buf, int itemId, int pageSize) {
+    private int getDataOffset(long pageAddr, int itemId, int pageSize) {
         assert checkIndex(itemId): itemId;
 
-        int directCnt = getDirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
 
-        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(buf, pageSize);
+        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(pageAddr, pageSize);
 
         if (itemId >= directCnt) { // Need to do indirect lookup.
-            int indirectCnt = getIndirectCount(buf);
+            int indirectCnt = getIndirectCount(pageAddr);
 
             // Must have indirect items here.
             assert indirectCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", indirectCnt=" + indirectCnt +
-                ", page=" + printPageLayout(buf, pageSize);
+                ", page=" + printPageLayout(pageAddr, pageSize);
 
-            int indirectItemIdx = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
+            int indirectItemIdx = findIndirectItemIndex(pageAddr, itemId, directCnt, indirectCnt);
 
             assert indirectItemIdx >= directCnt : indirectItemIdx + " " + directCnt;
             assert indirectItemIdx < directCnt + indirectCnt: indirectItemIdx + " " + directCnt + " " + indirectCnt;
 
-            itemId = directItemIndex(getItem(buf, indirectItemIdx));
+            itemId = directItemIndex(getItem(pageAddr, indirectItemIdx));
 
             assert itemId >= 0 && itemId < directCnt: itemId + " " + directCnt + " " + indirectCnt; // Direct item.
         }
 
-        return directItemToOffset(getItem(buf, itemId));
+        return directItemToOffset(getItem(pageAddr, itemId));
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param dataOff Points to the entry start.
      * @return Link to the next entry fragment or 0 if no fragments left or if entry is not fragmented.
      */
-    private long getNextFragmentLink(long buf, int dataOff) {
-        assert isFragmented(buf, dataOff);
+    private long getNextFragmentLink(long pageAddr, int dataOff) {
+        assert isFragmented(pageAddr, dataOff);
 
-        return PageUtils.getLong(buf, dataOff + PAYLOAD_LEN_SIZE);
+        return PageUtils.getLong(pageAddr, dataOff + PAYLOAD_LEN_SIZE);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @return {@code true} If the data row is fragmented across multiple pages.
      */
-    private boolean isFragmented(long buf, int dataOff) {
-        return (PageUtils.getShort(buf, dataOff) & FRAGMENTED_FLAG) != 0;
+    private boolean isFragmented(long pageAddr, int dataOff) {
+        return (PageUtils.getShort(pageAddr, dataOff) & FRAGMENTED_FLAG) != 0;
     }
 
     /**
      * Sets position to start of actual fragment data and limit to it's end.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param itemId Item to position on.
+     * @param pageSize Page size.
      * @return Link to the next fragment or {@code 0} if it is the last fragment or the data row is not fragmented.
      */
-    public DataPagePayload readPayload(final long buf, final int itemId, final int pageSize) {
-        int dataOff = getDataOffset(buf, itemId, pageSize);
+    public DataPagePayload readPayload(final long pageAddr, final int itemId, final int pageSize) {
+        int dataOff = getDataOffset(pageAddr, itemId, pageSize);
 
-        boolean fragmented = isFragmented(buf, dataOff);
-        long nextLink = fragmented ? getNextFragmentLink(buf, dataOff) : 0;
-        int payloadSize = getPageEntrySize(buf, dataOff, 0);
+        boolean fragmented = isFragmented(pageAddr, dataOff);
+        long nextLink = fragmented ? getNextFragmentLink(pageAddr, dataOff) : 0;
+        int payloadSize = getPageEntrySize(pageAddr, dataOff, 0);
 
         return new DataPagePayload(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0),
             payloadSize,
@@ -471,21 +473,21 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @return Item.
      */
-    private short getItem(long buf, int idx) {
-        return PageUtils.getShort(buf, itemOffset(idx));
+    private short getItem(long pageAddr, int idx) {
+        return PageUtils.getShort(pageAddr, itemOffset(idx));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @param item Item.
      */
-    private void setItem(long buf, int idx, short item) {
-        PageUtils.putShort(buf, itemOffset(idx), item);
+    private void setItem(long pageAddr, int idx, short item) {
+        PageUtils.putShort(pageAddr, itemOffset(idx), item);
     }
 
     /**
@@ -547,14 +549,14 @@ public class DataPageIO extends PageIO {
     /**
      * Move the last direct item to the free slot and reference it with indirect item on the same place.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param freeDirectIdx Free slot.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return {@code true} If the last direct item already had corresponding indirect item.
      */
-    private boolean moveLastItem(long buf, int freeDirectIdx, int directCnt, int indirectCnt) {
-        int lastIndirectId = findIndirectIndexForLastDirect(buf, directCnt, indirectCnt);
+    private boolean moveLastItem(long pageAddr, int freeDirectIdx, int directCnt, int indirectCnt) {
+        int lastIndirectId = findIndirectIndexForLastDirect(pageAddr, directCnt, indirectCnt);
 
         int lastItemId = directCnt - 1;
 
@@ -564,13 +566,13 @@ public class DataPageIO extends PageIO {
 
         assert itemId(indirectItem) == lastItemId && directItemIndex(indirectItem) == freeDirectIdx;
 
-        setItem(buf, freeDirectIdx, getItem(buf, lastItemId));
-        setItem(buf, lastItemId, indirectItem);
+        setItem(pageAddr, freeDirectIdx, getItem(pageAddr, lastItemId));
+        setItem(pageAddr, lastItemId, indirectItem);
 
-        assert getItem(buf, lastItemId) == indirectItem;
+        assert getItem(pageAddr, lastItemId) == indirectItem;
 
         if (lastIndirectId != -1) { // Fix pointer to direct item.
-            setItem(buf, lastIndirectId, indirectItem(itemId(getItem(buf, lastIndirectId)), freeDirectIdx));
+            setItem(pageAddr, lastIndirectId, indirectItem(itemId(getItem(pageAddr, lastIndirectId)), freeDirectIdx));
 
             return true;
         }
@@ -579,16 +581,16 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return Index of indirect item for the last direct item.
      */
-    private int findIndirectIndexForLastDirect(long buf, int directCnt, int indirectCnt) {
+    private int findIndirectIndexForLastDirect(long pageAddr, int directCnt, int indirectCnt) {
         int lastDirectId = directCnt - 1;
 
         for (int i = directCnt, end = directCnt + indirectCnt; i < end; i++) {
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
             if (directItemIndex(item) == lastDirectId)
                 return i;
@@ -598,21 +600,21 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @param pageSize Page size.
      * @return Next link for fragmented entries or {@code 0} if none.
      * @throws IgniteCheckedException If failed.
      */
-    public long removeRow(long buf, int itemId, int pageSize) throws IgniteCheckedException {
+    public long removeRow(long pageAddr, int itemId, int pageSize) throws IgniteCheckedException {
         assert checkIndex(itemId) : itemId;
 
-        final int dataOff = getDataOffset(buf, itemId, pageSize);
-        final long nextLink = isFragmented(buf, dataOff) ? getNextFragmentLink(buf, dataOff) : 0;
+        final int dataOff = getDataOffset(pageAddr, itemId, pageSize);
+        final long nextLink = isFragmented(pageAddr, dataOff) ? getNextFragmentLink(pageAddr, dataOff) : 0;
 
         // Record original counts to calculate delta in free space in the end of remove.
-        final int directCnt = getDirectCount(buf);
-        final int indirectCnt = getIndirectCount(buf);
+        final int directCnt = getDirectCount(pageAddr);
+        final int indirectCnt = getIndirectCount(pageAddr);
 
         int curIndirectCnt = indirectCnt;
 
@@ -621,24 +623,24 @@ public class DataPageIO extends PageIO {
         // Remove the last item on the page.
         if (directCnt == 1) {
             assert (indirectCnt == 0 && itemId == 0) ||
-                (indirectCnt == 1 && itemId == itemId(getItem(buf, 1))) : itemId;
+                (indirectCnt == 1 && itemId == itemId(getItem(pageAddr, 1))) : itemId;
 
-            setEmptyPage(buf, pageSize);
+            setEmptyPage(pageAddr, pageSize);
         }
         else {
             // Get the entry size before the actual remove.
-            int rmvEntrySize = getPageEntrySize(buf, dataOff, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int rmvEntrySize = getPageEntrySize(pageAddr, dataOff, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             int indirectId = 0;
 
             if (itemId >= directCnt) { // Need to remove indirect item.
                 assert indirectCnt > 0;
 
-                indirectId = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
+                indirectId = findIndirectItemIndex(pageAddr, itemId, directCnt, indirectCnt);
 
                 assert indirectId >= directCnt;
 
-                itemId = directItemIndex(getItem(buf, indirectId));
+                itemId = directItemIndex(getItem(pageAddr, indirectId));
 
                 assert itemId < directCnt;
             }
@@ -646,32 +648,32 @@ public class DataPageIO extends PageIO {
             boolean dropLast = true;
 
             if (itemId + 1 < directCnt) // It is not the last direct item.
-                dropLast = moveLastItem(buf, itemId, directCnt, indirectCnt);
+                dropLast = moveLastItem(pageAddr, itemId, directCnt, indirectCnt);
 
             if (indirectId == 0) {// For the last direct item with no indirect item.
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectCnt, -1, pageSize);
+                    moveItems(pageAddr, directCnt, indirectCnt, -1, pageSize);
                 else
                     curIndirectCnt++;
             }
             else {
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectId - directCnt, -1, pageSize);
+                    moveItems(pageAddr, directCnt, indirectId - directCnt, -1, pageSize);
 
-                moveItems(buf, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1, pageSize);
+                moveItems(pageAddr, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1, pageSize);
 
                 if (dropLast)
                     curIndirectCnt--;
             }
 
-            setIndirectCount(buf, curIndirectCnt);
-            setDirectCount(buf, directCnt - 1);
+            setIndirectCount(pageAddr, curIndirectCnt);
+            setDirectCount(pageAddr, directCnt - 1);
 
-            assert getIndirectCount(buf) <= getDirectCount(buf);
+            assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr);
 
             // Increase free space.
-            setRealFreeSpace(buf,
-                getRealFreeSpace(buf) + rmvEntrySize + ITEM_SIZE * (directCnt - getDirectCount(buf) + indirectCnt - getIndirectCount(buf)),
+            setRealFreeSpace(pageAddr,
+                getRealFreeSpace(pageAddr) + rmvEntrySize + ITEM_SIZE * (directCnt - getDirectCount(pageAddr) + indirectCnt - getIndirectCount(pageAddr)),
                 pageSize);
         }
 
@@ -679,17 +681,17 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param cnt Count.
      * @param step Step.
      * @param pageSize Page size.
      */
-    private void moveItems(long buf, int idx, int cnt, int step, int pageSize) {
+    private void moveItems(long pageAddr, int idx, int cnt, int step, int pageSize) {
         assert cnt >= 0: cnt;
 
         if (cnt != 0)
-            moveBytes(buf, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE, pageSize);
+            moveBytes(pageAddr, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE, pageSize);
     }
 
     /**
@@ -706,63 +708,63 @@ public class DataPageIO extends PageIO {
     /**
      * Adds row to this data page and sets respective link to the given row object.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param row Cache data row.
      * @param rowSize Row size.
      * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        final long buf,
+        final long pageAddr,
         CacheDataRow row,
         final int rowSize,
         final int pageSize
     ) throws IgniteCheckedException {
-        assert rowSize <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
+        assert rowSize <= getFreeSpace(pageAddr): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(rowSize, SHOW_PAYLOAD_LEN | SHOW_ITEM);
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
-        writeRowData(buf, dataOff, rowSize, row);
+        writeRowData(pageAddr, dataOff, rowSize, row);
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
+        int itemId = addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
-        setLink(row, buf, itemId);
+        setLink(row, pageAddr, itemId);
     }
 
     /**
      * Adds row to this data page and sets respective link to the given row object.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param payload Payload.
      * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        long buf,
+        long pageAddr,
         byte[] payload,
         int pageSize
     ) throws IgniteCheckedException {
-        assert payload.length <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
+        assert payload.length <= getFreeSpace(pageAddr): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(payload.length, SHOW_PAYLOAD_LEN | SHOW_ITEM);
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
-        writeRowData(buf, dataOff, payload);
+        writeRowData(pageAddr, dataOff, payload);
 
-        addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
+        addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param entryFullSize New entry full size (with item, length and link).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
@@ -771,7 +773,7 @@ public class DataPageIO extends PageIO {
      * @return First entry offset after compaction.
      */
     private int compactIfNeed(
-        final long buf,
+        final long pageAddr,
         final int entryFullSize,
         final int directCnt,
         final int indirectCnt,
@@ -779,7 +781,7 @@ public class DataPageIO extends PageIO {
         int pageSize
     ) {
         if (!isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt)) {
-            dataOff = compactDataEntries(buf, directCnt, pageSize);
+            dataOff = compactDataEntries(pageAddr, directCnt, pageSize);
 
             assert isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt);
         }
@@ -790,7 +792,7 @@ public class DataPageIO extends PageIO {
     /**
      * Put item reference on entry.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param fullEntrySize Full entry size (with link, payload size and item).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
@@ -798,41 +800,41 @@ public class DataPageIO extends PageIO {
      * @param pageSize Page size.
      * @return Item ID.
      */
-    private int addItem(final long buf,
+    private int addItem(final long pageAddr,
         final int fullEntrySize,
         final int directCnt,
         final int indirectCnt,
         final int dataOff,
         final int pageSize)
     {
-        setFirstEntryOffset(buf, dataOff, pageSize);
+        setFirstEntryOffset(pageAddr, dataOff, pageSize);
 
-        int itemId = insertItem(buf, dataOff, directCnt, indirectCnt, pageSize);
+        int itemId = insertItem(pageAddr, dataOff, directCnt, indirectCnt, pageSize);
 
         assert checkIndex(itemId): itemId;
-        assert getIndirectCount(buf) <= getDirectCount(buf);
+        assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr);
 
         // Update free space. If number of indirect items changed, then we were able to reuse an item slot.
-        setRealFreeSpace(buf,
-            getRealFreeSpace(buf) - fullEntrySize + (getIndirectCount(buf) != indirectCnt ? ITEM_SIZE : 0),
+        setRealFreeSpace(pageAddr,
+            getRealFreeSpace(pageAddr) - fullEntrySize + (getIndirectCount(pageAddr) != indirectCnt ? ITEM_SIZE : 0),
             pageSize);
 
         return itemId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param fullEntrySize Full entry size.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param pageSize Page size.
      * @return Offset in the buffer where the entry must be written.
      */
-    private int getDataOffsetForWrite(long buf, int fullEntrySize, int directCnt, int indirectCnt, int pageSize) {
-        int dataOff = getFirstEntryOffset(buf);
+    private int getDataOffsetForWrite(long pageAddr, int fullEntrySize, int directCnt, int indirectCnt, int pageSize) {
+        int dataOff = getFirstEntryOffset(pageAddr);
 
         // Compact if we do not have enough space for entry.
-        dataOff = compactIfNeed(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
+        dataOff = compactIfNeed(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         // We will write data right before the first entry.
         dataOff -= fullEntrySize - ITEM_SIZE;
@@ -843,7 +845,7 @@ public class DataPageIO extends PageIO {
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param row Cache data row.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
@@ -852,31 +854,31 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public int addRowFragment(
-        long buf,
+        long pageAddr,
         CacheDataRow row,
         int written,
         int rowSize,
         int pageSize
     ) throws IgniteCheckedException {
-        return addRowFragment(buf, written, rowSize, row.link(), row, null, pageSize);
+        return addRowFragment(pageAddr, written, rowSize, row.link(), row, null, pageSize);
     }
 
     /**
      * Adds this payload as a fragment to this data page.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param payload Payload bytes.
      * @param lastLink Link to the previous written fragment (link to the tail).
      * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRowFragment(
-        long buf,
+        long pageAddr,
         byte[] payload,
         long lastLink,
         int pageSize
     ) throws IgniteCheckedException {
-        addRowFragment(buf, 0, 0, lastLink, null, payload, pageSize);
+        addRowFragment(pageAddr, 0, 0, lastLink, null, payload, pageSize);
     }
 
     // TODO GG-11810.
@@ -885,7 +887,7 @@ public class DataPageIO extends PageIO {
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
      * @param lastLink Link to the previous written fragment (link to the tail).
@@ -896,7 +898,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     private int addRowFragment(
-        long buf,
+        long pageAddr,
         int written,
         int rowSize,
         long lastLink,
@@ -906,17 +908,17 @@ public class DataPageIO extends PageIO {
     ) throws IgniteCheckedException {
         assert payload == null ^ row == null;
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
         int payloadSize = payload != null ? payload.length :
-            Math.min(rowSize - written, getFreeSpace(buf));
+            Math.min(rowSize - written, getFreeSpace(pageAddr));
 
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt, pageSize);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
         if (payload == null) {
-            ByteBuffer buf0 = nioAccess.newDirectByteBuffer(buf, pageSize, null);
+            ByteBuffer buf0 = nioAccess.newDirectByteBuffer(pageAddr, pageSize, null);
             buf0.order(PageMemory.NATIVE_BYTE_ORDER);
 
             buf0.position(dataOff);
@@ -931,17 +933,17 @@ public class DataPageIO extends PageIO {
             writeFragmentData(row, buf0, rowOff, payloadSize);
         }
         else {
-            PageUtils.putShort(buf, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
+            PageUtils.putShort(pageAddr, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
 
-            PageUtils.putLong(buf, dataOff + 2, lastLink);
+            PageUtils.putLong(pageAddr, dataOff + 2, lastLink);
 
-            PageUtils.putBytes(buf, dataOff + 10, payload);
+            PageUtils.putBytes(pageAddr, dataOff + 10, payload);
         }
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
+        int itemId = addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         if (row != null)
-            setLink(row, buf, itemId);
+            setLink(row, pageAddr, itemId);
 
         return payloadSize;
     }
@@ -1122,56 +1124,56 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param pageSize Page size.
      * @return Item ID (insertion index).
      */
-    private int insertItem(long buf, int dataOff, int directCnt, int indirectCnt, int pageSize) {
+    private int insertItem(long pageAddr, int dataOff, int directCnt, int indirectCnt, int pageSize) {
         if (indirectCnt > 0) {
             // If the first indirect item is on correct place to become the last direct item, do the transition
             // and insert the new item into the free slot which was referenced by this first indirect item.
-            short item = getItem(buf, directCnt);
+            short item = getItem(pageAddr, directCnt);
 
             if (itemId(item) == directCnt) {
                 int directItemIdx = directItemIndex(item);
 
-                setItem(buf, directCnt, getItem(buf, directItemIdx));
-                setItem(buf, directItemIdx, directItemFromOffset(dataOff));
+                setItem(pageAddr, directCnt, getItem(pageAddr, directItemIdx));
+                setItem(pageAddr, directItemIdx, directItemFromOffset(dataOff));
 
-                setDirectCount(buf, directCnt + 1);
-                setIndirectCount(buf, indirectCnt - 1);
+                setDirectCount(pageAddr, directCnt + 1);
+                setIndirectCount(pageAddr, indirectCnt - 1);
 
                 return directItemIdx;
             }
         }
 
         // Move all the indirect items forward to make a free slot and insert new item at the end of direct items.
-        moveItems(buf, directCnt, indirectCnt, +1, pageSize);
+        moveItems(pageAddr, directCnt, indirectCnt, +1, pageSize);
 
-        setItem(buf, directCnt, directItemFromOffset(dataOff));
+        setItem(pageAddr, directCnt, directItemFromOffset(dataOff));
 
-        setDirectCount(buf, directCnt + 1);
-        assert getDirectCount(buf) == directCnt + 1;
+        setDirectCount(pageAddr, directCnt + 1);
+        assert getDirectCount(pageAddr) == directCnt + 1;
 
         return directCnt; // Previous directCnt will be our itemId.
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param directCnt Direct items count.
      * @param pageSize Page size.
      * @return New first entry offset.
      */
-    private int compactDataEntries(long buf, int directCnt, int pageSize) {
+    private int compactDataEntries(long pageAddr, int directCnt, int pageSize) {
         assert checkCount(directCnt): directCnt;
 
         int[] offs = new int[directCnt];
 
         for (int i = 0; i < directCnt; i++) {
-            int off = directItemToOffset(getItem(buf, i));
+            int off = directItemToOffset(getItem(pageAddr, i));
 
             offs[i] = (off << 8) | i; // This way we'll be able to sort by offset using Arrays.sort(...).
         }
@@ -1186,20 +1188,20 @@ public class DataPageIO extends PageIO {
 
             assert off < prevOff: off;
 
-            int entrySize = getPageEntrySize(buf, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int entrySize = getPageEntrySize(pageAddr, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             int delta = prevOff - (off + entrySize);
 
             if (delta != 0) { // Move right.
                 assert delta > 0: delta;
 
-                moveBytes(buf, off, entrySize, delta, pageSize);
+                moveBytes(pageAddr, off, entrySize, delta, pageSize);
 
                 int itemId = offs[i] & 0xFF;
 
                 off += delta;
 
-                setItem(buf, itemId, directItemFromOffset(off));
+                setItem(pageAddr, itemId, directItemFromOffset(off));
             }
 
             prevOff = off;
@@ -1211,56 +1213,56 @@ public class DataPageIO extends PageIO {
     /**
      * Full-scan free space calculation procedure.
      *
-     * @param buf Buffer to scan.
+     * @param pageAddr Page to scan.
      * @param pageSize Page size.
      * @return Actual free space in the buffer.
      */
-    private int actualFreeSpace(long buf, int pageSize) {
-        int directCnt = getDirectCount(buf);
+    private int actualFreeSpace(long pageAddr, int pageSize) {
+        int directCnt = getDirectCount(pageAddr);
 
         int entriesSize = 0;
 
         for (int i = 0; i < directCnt; i++) {
-            int off = directItemToOffset(getItem(buf, i));
+            int off = directItemToOffset(getItem(pageAddr, i));
 
-            int entrySize = getPageEntrySize(buf, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int entrySize = getPageEntrySize(pageAddr, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             entriesSize += entrySize;
         }
 
-        return pageSize - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(buf)) * ITEM_SIZE;
+        return pageSize - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(pageAddr)) * ITEM_SIZE;
     }
 
     /**
-     * @param buf Buffer.
+     * @param addr Address.
      * @param off Offset.
      * @param cnt Count.
      * @param step Step.
      * @param pageSize Page size.
      */
-    private void moveBytes(long buf, int off, int cnt, int step, int pageSize) {
+    private void moveBytes(long addr, int off, int cnt, int step, int pageSize) {
         assert step != 0: step;
         assert off + step >= 0;
         assert off + step + cnt <= pageSize : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
             ", cap=" + pageSize + ']';
 
-        PageHandler.copyMemory(buf, buf, off, off + step, cnt);
+        PageHandler.copyMemory(addr, addr, off, off + step, cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param payloadSize Payload size.
      * @param row Data row.
      * @throws IgniteCheckedException If failed.
      */
     private void writeRowData(
-        long buf,
+        long pageAddr,
         int dataOff,
         int payloadSize,
         CacheDataRow row
     ) throws IgniteCheckedException {
-        long addr = buf + dataOff;
+        long addr = pageAddr + dataOff;
 
         PageUtils.putShort(addr, 0, (short)payloadSize);
         addr += 2;
@@ -1275,18 +1277,18 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param payload Payload
      */
     private void writeRowData(
-        long buf,
+        long pageAddr,
         int dataOff,
         byte[] payload
     ) {
-        PageUtils.putShort(buf, dataOff, (short)payload.length);
+        PageUtils.putShort(pageAddr, dataOff, (short)payload.length);
         dataOff += 2;
 
-        PageUtils.putBytes(buf, dataOff, payload);
+        PageUtils.putBytes(pageAddr, dataOff, payload);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
index 203429e..7dedc00 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
@@ -30,20 +30,34 @@ public class DataPagePayload {
     /** */
     private final long nextLink;
 
+    /**
+     * @param off Offset.
+     * @param payloadSize Payload size.
+     * @param nextLink Next link.
+     */
     DataPagePayload(int off, int payloadSize, long nextLink) {
         this.off = off;
         this.payloadSize = payloadSize;
         this.nextLink = nextLink;
     }
 
+    /**
+     * @return Offset.
+     */
     public int offset() {
         return off;
     }
 
+    /**
+     * @return Payload size.
+     */
     public int payloadSize() {
         return payloadSize;
     }
 
+    /**
+     * @return Next link.
+     */
     public long nextLink() {
         return nextLink;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/22c1d1a2/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 3aee268..58f2b66b3 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
@@ -164,38 +164,39 @@ public abstract class PageIO {
     }
 
     /**
+     * @param pageAddr Page addres.
      * @return Page type.
      */
-    public static int getType(long buf) {
-        return PageUtils.getShort(buf, TYPE_OFF) & 0xFFFF;
+    public static int getType(long pageAddr) {
+        return PageUtils.getShort(pageAddr, TYPE_OFF) & 0xFFFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param type Type.
      */
-    public static void setType(long buf, int type) {
-        PageUtils.putShort(buf, TYPE_OFF, (short)type);
+    public static void setType(long pageAddr, int type) {
+        PageUtils.putShort(pageAddr, TYPE_OFF, (short)type);
 
-        assert getType(buf) == type;
+        assert getType(pageAddr) == type;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Version.
      */
-    public static int getVersion(long buf) {
-        return PageUtils.getShort(buf, VER_OFF) & 0xFFFF;
+    public static int getVersion(long pageAddr) {
+        return PageUtils.getShort(pageAddr, VER_OFF) & 0xFFFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param ver Version.
      */
-    public static void setVersion(long buf, int ver) {
-        PageUtils.putShort(buf, VER_OFF, (short)ver);
+    private static void setVersion(long pageAddr, int ver) {
+        PageUtils.putShort(pageAddr, VER_OFF, (short)ver);
 
-        assert getVersion(buf) == ver;
+        assert getVersion(pageAddr) == ver;
     }
 
     /**
@@ -207,29 +208,29 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Page ID.
      */
-    public static void setPageId(long buf, long pageId) {
-        PageUtils.putLong(buf, PAGE_ID_OFF, pageId);
+    public static void setPageId(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, PAGE_ID_OFF, pageId);
 
-        assert getPageId(buf) == pageId;
+        assert getPageId(pageAddr) == pageId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Checksum.
      */
-    public static int getCrc(long buf) {
-        return PageUtils.getInt(buf, CRC_OFF);
+    public static int getCrc(long pageAddr) {
+        return PageUtils.getInt(pageAddr, CRC_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param crc Checksum.
      */
-    public static void setCrc(long buf, int crc) {
-        PageUtils.putInt(buf, CRC_OFF, crc);
+    public static void setCrc(long pageAddr, int crc) {
+        PageUtils.putInt(pageAddr, CRC_OFF, crc);
     }
 
     /**
@@ -272,18 +273,19 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Page ID.
+     * @param pageSize Page size.
      */
-    public void initNewPage(long buf, long pageId, int pageSize) {
-        setType(buf, getType());
-        setVersion(buf, getVersion());
-        setPageId(buf, pageId);
-        setCrc(buf, 0);
-
-        PageUtils.putLong(buf, RESERVED_1_OFF, 0L);
-        PageUtils.putLong(buf, RESERVED_2_OFF, 0L);
-        PageUtils.putLong(buf, RESERVED_3_OFF, 0L);
+    public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        setType(pageAddr, getType());
+        setVersion(pageAddr, getVersion());
+        setPageId(pageAddr, pageId);
+        setCrc(pageAddr, 0);
+
+        PageUtils.putLong(pageAddr, RESERVED_1_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_2_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_3_OFF, 0L);
     }
 
     /** {@inheritDoc} */
@@ -292,13 +294,13 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends PageIO> Q getPageIO(long buf) throws IgniteCheckedException {
-        int type = getType(buf);
-        int ver = getVersion(buf);
+    public static <Q extends PageIO> Q getPageIO(long pageAddr) throws IgniteCheckedException {
+        int type = getType(pageAddr);
+        int ver = getVersion(pageAddr);
 
         return getPageIO(type, ver);
     }
@@ -339,13 +341,13 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO for either inner or leaf B+Tree page.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends BPlusIO<?>> Q getBPlusIO(long buf) throws IgniteCheckedException {
-        int type = getType(buf);
-        int ver = getVersion(buf);
+    public static <Q extends BPlusIO<?>> Q getBPlusIO(long pageAddr) throws IgniteCheckedException {
+        int type = getType(pageAddr);
+        int ver = getVersion(pageAddr);
 
         return getBPlusIO(type, ver);
     }


[21/25] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-gg-8.0.2.ea2' into ignite-gg-11810

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/community/ignite-gg-8.0.2.ea2' into ignite-gg-11810

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogNoopManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
#	modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIO.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java


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

Branch: refs/heads/ignite-gg-11810
Commit: 4183186ab1a92ede58b1b5b59ee0c33e6632d87f
Parents: df4f0f0 5c106da
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 14:17:09 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 14:17:09 2016 +0300

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 README.md                                       |    8 +
 RELEASE_NOTES.txt                               |   22 +
 assembly/dependencies-fabric-lgpl.xml           |    1 +
 assembly/dependencies-fabric.xml                |    1 +
 assembly/release-base.xml                       |    5 +
 assembly/release-fabric-base.xml                |    1 +
 bin/control.bat                                 |  229 ++
 bin/control.sh                                  |  180 +
 bin/ignite.sh                                   |    9 +
 config/dotnet/default-dotnet.xml                |   45 +
 examples/pom-standalone.xml                     |    6 +
 examples/pom.xml                                |    2 +-
 examples/redis/redis-example.php                |   82 +
 examples/redis/redis-example.py                 |   62 +
 examples/schema-import/pom.xml                  |    2 +-
 .../ignite/examples/ExampleNodeStartup.java     |    2 +-
 .../examples/datagrid/CachePutGetExample.java   |    2 +-
 .../examples/datagrid/CacheQueryDmlExample.java |  163 +
 .../scalar/examples/ScalarJvmCloudExample.scala |    2 +-
 .../CacheExamplesMultiNodeSelfTest.java         |    7 +
 .../ignite/examples/CacheExamplesSelfTest.java  |    8 +
 modules/aop/pom.xml                             |    2 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/benchmarks/pom.xml                      |    2 +-
 .../jmh/misc/JmhIncrementBenchmark.java         |   95 +
 modules/camel/pom.xml                           |    2 +-
 modules/cassandra/pom.xml                       |   16 +-
 modules/cassandra/serializers/README.txt        |    7 +-
 modules/cassandra/serializers/pom.xml           |    4 +-
 modules/cassandra/store/pom.xml                 |    4 +-
 .../store/cassandra/CassandraCacheStore.java    |    6 +
 .../store/cassandra/datasource/DataSource.java  |    9 +
 modules/clients/pom.xml                         |   12 +-
 .../clients/src/test/config/jdbc-bin-config.xml |   54 +
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  263 ++
 .../JdbcAbstractUpdateStatementSelfTest.java    |   37 +
 ...BinaryMarshallerInsertStatementSelfTest.java |   37 +
 ...cBinaryMarshallerMergeStatementSelfTest.java |   37 +
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |   49 +
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  122 +
 .../jdbc2/JdbcMergeStatementSelfTest.java       |   91 +
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  |   40 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |   18 +
 .../rest/ChangeStateCommandHandlerTest.java     |  125 +
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../JettyRestProcessorAbstractSelfTest.java     |   11 +-
 .../rest/RestProcessorMultiStartSelfTest.java   |   48 +-
 .../rest/protocols/tcp/MockNioSession.java      |   11 +
 .../tcp/redis/RedisProtocolSelfTest.java        |  101 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    5 +
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |    2 +
 modules/core/pom.xml                            |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    8 +-
 .../java/org/apache/ignite/IgniteCache.java     |    9 +
 .../apache/ignite/IgniteCheckedException.java   |   12 +
 .../java/org/apache/ignite/IgniteLogger.java    |    6 +-
 .../java/org/apache/ignite/IgniteServices.java  |   16 +
 .../apache/ignite/IgniteSystemProperties.java   |   58 +
 .../binary/BinaryAbstractIdentityResolver.java  |   53 +
 .../binary/BinaryArrayIdentityResolver.java     |  224 ++
 .../binary/BinaryFieldIdentityResolver.java     |  307 ++
 .../ignite/binary/BinaryIdentityResolver.java   |   42 +
 .../ignite/binary/BinaryTypeConfiguration.java  |   27 +-
 .../org/apache/ignite/cache/QueryEntity.java    |   27 +
 .../rendezvous/RendezvousAffinityFunction.java  |   80 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |    2 +-
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |    6 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |    6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java |  112 +-
 .../configuration/CacheConfiguration.java       |   19 +-
 .../configuration/IgniteConfiguration.java      |   50 +-
 .../apache/ignite/internal/GridComponent.java   |    7 +-
 .../internal/GridEventConsumeHandler.java       |    7 +-
 .../ignite/internal/GridJobContextImpl.java     |    4 +-
 .../ignite/internal/GridKernalContext.java      |   17 +
 .../ignite/internal/GridKernalContextImpl.java  |   28 +-
 .../internal/GridMessageListenHandler.java      |    5 +-
 .../internal/GridPerformanceSuggestions.java    |    2 +-
 .../ignite/internal/GridPluginComponent.java    |    4 +-
 .../org/apache/ignite/internal/GridTopic.java   |    6 +-
 .../ignite/internal/IgniteInternalFuture.java   |   11 +
 .../apache/ignite/internal/IgniteKernal.java    |  187 +-
 .../ignite/internal/IgniteServicesImpl.java     |    9 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   34 +-
 .../ignite/internal/MarshallerContextImpl.java  |   10 +-
 .../internal/binary/BinaryClassDescriptor.java  |   27 +-
 .../ignite/internal/binary/BinaryContext.java   |   80 +-
 .../ignite/internal/binary/BinaryFieldImpl.java |   10 +-
 .../internal/binary/BinaryObjectExImpl.java     |   90 +-
 .../internal/binary/BinaryObjectImpl.java       |   48 +-
 .../binary/BinaryObjectOffheapImpl.java         |   42 +-
 .../internal/binary/BinaryPrimitives.java       |   30 +-
 .../binary/BinarySerializedFieldComparator.java |  343 ++
 .../ignite/internal/binary/BinaryTypeProxy.java |   15 +-
 .../ignite/internal/binary/BinaryUtils.java     |   16 +-
 .../internal/binary/BinaryWriterExImpl.java     |   47 +-
 .../internal/binary/GridBinaryMarshaller.java   |    2 +-
 .../binary/builder/BinaryBuilderReader.java     |   11 +
 .../binary/builder/BinaryBuilderSerializer.java |    4 +
 .../binary/builder/BinaryObjectBuilderImpl.java |   27 +-
 .../streams/BinaryAbstractInputStream.java      |    5 +
 .../streams/BinaryAbstractOutputStream.java     |    5 +
 .../streams/BinaryByteBufferInputStream.java    |   10 +
 .../binary/streams/BinaryHeapInputStream.java   |    9 +-
 .../binary/streams/BinaryHeapOutputStream.java  |    7 +-
 .../streams/BinaryMemoryAllocatorChunk.java     |    3 +-
 .../streams/BinaryOffheapInputStream.java       |   14 +-
 .../streams/BinaryOffheapOutputStream.java      |    8 +-
 .../internal/binary/streams/BinaryStream.java   |   12 +-
 .../ignite/internal/client/GridClient.java      |    9 +
 .../internal/client/GridClientClusterState.java |   33 +
 .../client/impl/GridClientClusterStateImpl.java |   75 +
 .../internal/client/impl/GridClientImpl.java    |   11 +
 .../impl/connection/GridClientConnection.java   |   17 +
 .../GridClientConnectionManagerAdapter.java     |    1 +
 .../connection/GridClientNioTcpConnection.java  |   21 +
 .../router/impl/GridRouterClientImpl.java       |    6 +
 .../internal/commandline/CommandHandler.java    |   89 +
 .../ignite/internal/jdbc2/JdbcConnection.java   |   17 +
 .../internal/jdbc2/JdbcPreparedStatement.java   |   71 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |    5 +-
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |  406 ++
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   72 +-
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |   49 +
 .../ignite/internal/jdbc2/JdbcStatement.java    |  213 +-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |   25 +-
 .../internal/managers/GridManagerAdapter.java   |    2 +-
 .../checkpoint/GridCheckpointManager.java       |    2 +-
 .../collision/GridCollisionManager.java         |    2 +-
 .../managers/communication/GridIoManager.java   |   35 +-
 .../managers/communication/GridIoMessage.java   |   13 +
 .../communication/GridIoMessageFactory.java     |   71 +-
 .../communication/IgniteIoTestMessage.java      |   13 +-
 .../deployment/GridDeploymentManager.java       |    2 +-
 .../discovery/GridDiscoveryManager.java         |  125 +-
 .../eventstorage/GridEventStorageManager.java   |    2 +-
 .../managers/failover/GridFailoverManager.java  |    2 +-
 .../managers/indexing/GridIndexingManager.java  |    2 +-
 .../loadbalancer/GridLoadBalancerManager.java   |    2 +-
 .../ignite/internal/pagemem/FullPageId.java     |   11 +-
 .../ignite/internal/pagemem/PageIdUtils.java    |    8 +
 .../pagemem/backup/BackupFinishedMessage.java   |  136 -
 .../StartFullBackupAckDiscoveryMessage.java     |  128 -
 .../backup/StartFullBackupDiscoveryMessage.java |  135 -
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |    2 +-
 .../snapshot/SnapshotFinishedMessage.java       |  136 +
 .../snapshot/SnapshotProgressMessage.java       |  135 +
 .../StartFullSnapshotAckDiscoveryMessage.java   |  147 +
 .../StartFullSnapshotDiscoveryMessage.java      |  160 +
 .../pagemem/store/IgnitePageStoreManager.java   |   13 +-
 .../internal/pagemem/store/PageStore.java       |    2 +-
 .../pagemem/wal/IgniteWriteAheadLogManager.java |    3 +-
 .../wal/record/MemoryRecoveryRecord.java        |   23 +-
 .../internal/pagemem/wal/record/WALRecord.java  |   23 +-
 .../MetaPageUpdateCandidateAllocatedIndex.java  |   61 -
 .../MetaPageUpdateLastSuccessfulBackupId.java   |   58 -
 ...etaPageUpdateLastSuccessfulFullBackupId.java |   58 -
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |   58 +
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |   70 +
 .../delta/MetaPageUpdateNextBackupId.java       |   58 -
 .../delta/MetaPageUpdateNextSnapshotId.java     |   58 +
 .../MetaPageUpdatePartitionDataRecord.java      |   15 +-
 .../delta/PageListMetaResetCountRecord.java     |   46 +
 .../record/delta/TrackingPageDeltaRecord.java   |   28 +-
 .../processors/GridProcessorAdapter.java        |    4 +-
 .../affinity/GridAffinityProcessor.java         |    4 +-
 .../cache/CacheAffinitySharedManager.java       |    9 +
 .../processors/cache/CacheLockCandidates.java   |   42 +
 .../cache/CacheLockCandidatesList.java          |   71 +
 .../internal/processors/cache/CacheState.java   |   33 -
 .../cache/CacheStoreBalancingWrapper.java       |    6 +
 .../cache/ChangeGlobalStateMessage.java         |  120 +
 .../internal/processors/cache/ClusterState.java |   38 +
 .../cache/DynamicCacheChangeRequest.java        |   23 +-
 .../processors/cache/GridCacheAdapter.java      |  754 ++--
 .../processors/cache/GridCacheContext.java      |    8 +
 .../processors/cache/GridCacheEntryEx.java      |    7 +-
 .../cache/GridCacheLoaderWriterStore.java       |    6 +
 .../processors/cache/GridCacheMapEntry.java     |  158 +-
 .../processors/cache/GridCacheMessage.java      |    7 +
 .../processors/cache/GridCacheMvcc.java         |  376 +-
 .../processors/cache/GridCacheMvccCallback.java |    4 +-
 .../cache/GridCacheMvccCandidate.java           |   80 +-
 .../processors/cache/GridCacheMvccManager.java  |   19 +-
 .../GridCachePartitionExchangeManager.java      |  187 +-
 .../processors/cache/GridCachePreloader.java    |   16 +-
 .../cache/GridCachePreloaderAdapter.java        |   14 +-
 .../processors/cache/GridCacheProcessor.java    |  444 ++-
 .../processors/cache/GridCacheProxyImpl.java    |   24 +-
 .../cache/GridCacheSharedTtlCleanupManager.java |    5 +-
 .../processors/cache/GridCacheTtlManager.java   |    4 +
 .../processors/cache/GridCacheUtils.java        |   62 +-
 .../GridChangeGlobalStateMessageResponse.java   |  177 +
 .../cache/IgniteCacheOffheapManager.java        |   14 +
 .../cache/IgniteCacheOffheapManagerImpl.java    |  134 +-
 .../processors/cache/IgniteCacheProxy.java      |   14 +-
 .../processors/cache/IgniteInternalCache.java   |    8 +
 .../processors/cache/QueryCursorImpl.java       |   31 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |    9 +-
 .../IgniteCacheDatabaseSharedManager.java       |   54 +-
 .../cache/database/freelist/PagesList.java      |    4 +
 .../cache/database/tree/io/PageMetaIO.java      |  115 +-
 .../cache/database/tree/io/TrackingPageIO.java  |  135 +-
 .../cache/database/tree/util/PageHandler.java   |    5 +
 .../CacheDataStructuresManager.java             |    6 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    4 +-
 .../distributed/GridDistributedCacheEntry.java  |  303 +-
 .../distributed/GridDistributedLockRequest.java |    5 +
 .../GridDistributedUnlockRequest.java           |    5 +
 .../dht/GridClientPartitionTopology.java        |   71 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    3 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   34 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   81 +-
 .../distributed/dht/GridDhtLockFuture.java      |   61 +-
 .../distributed/dht/GridDhtLockResponse.java    |    9 +-
 .../dht/GridDhtPartitionTopology.java           |   22 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  261 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |    1 -
 .../distributed/dht/GridDhtTxFinishFuture.java  |   74 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  125 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    7 +-
 .../dht/GridDhtTxOnePhaseCommitAckRequest.java  |    2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  461 ++-
 .../GridDhtAtomicSingleUpdateRequest.java       |    7 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |    5 +
 ...idNearAtomicAbstractSingleUpdateRequest.java |   28 +-
 .../atomic/GridNearAtomicFullUpdateRequest.java |   69 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |    6 +-
 .../GridNearAtomicSingleUpdateRequest.java      |    7 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   13 -
 .../colocated/GridDhtColocatedLockFuture.java   |   12 +-
 .../dht/preloader/GridDhtPartitionDemander.java |  277 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |   18 +-
 .../dht/preloader/GridDhtPartitionMap2.java     |    4 +
 .../GridDhtPartitionsAbstractMessage.java       |    3 +-
 .../GridDhtPartitionsExchangeFuture.java        |  167 +-
 .../preloader/GridDhtPartitionsFullMessage.java |   83 +-
 .../GridDhtPartitionsSingleMessage.java         |   81 +-
 .../GridDhtPartitionsSingleRequest.java         |    3 +-
 .../dht/preloader/GridDhtPreloader.java         |   20 +-
 .../distributed/near/GridNearAtomicCache.java   |   65 +-
 .../distributed/near/GridNearCacheEntry.java    |   44 +-
 .../distributed/near/GridNearGetRequest.java    |    5 +
 .../distributed/near/GridNearLockFuture.java    |    7 +-
 ...arOptimisticSerializableTxPrepareFuture.java |    4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    4 +-
 .../near/GridNearTransactionalCache.java        |    5 +-
 .../near/GridNearTxFinishFuture.java            |   46 +-
 .../cache/distributed/near/GridNearTxLocal.java |   48 +-
 .../cache/local/GridLocalCacheEntry.java        |  173 +-
 .../cache/local/GridLocalLockFuture.java        |    2 +-
 .../local/atomic/GridLocalAtomicCache.java      |  174 +-
 .../query/GridCacheDistributedQueryManager.java |    2 +-
 .../cache/query/GridCacheQueryManager.java      |   25 +-
 .../cache/query/GridCacheQueryRequest.java      |    6 +-
 .../cache/query/GridCacheTwoStepQuery.java      |  253 --
 .../cache/query/IgniteQueryErrorCode.java       |   91 +
 .../continuous/CacheContinuousQueryHandler.java |   19 +-
 .../cache/store/CacheStoreManager.java          |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |   34 +-
 .../cache/store/GridCacheWriteBehindStore.java  |    6 +-
 .../cache/transactions/IgniteTxHandler.java     |   17 +-
 .../transactions/IgniteTxLocalAdapter.java      |   38 +-
 .../cache/transactions/IgniteTxManager.java     |   20 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    2 +-
 .../clock/GridClockSyncProcessor.java           |    8 +-
 .../closure/GridClosureProcessor.java           |   33 +-
 .../processors/cluster/ClusterProcessor.java    |    2 +-
 .../cluster/GridClusterStateProcessor.java      |  936 +++++
 .../cluster/IgniteChangeGlobalStateSupport.java |   40 +
 .../continuous/GridContinuousHandler.java       |    5 +-
 .../continuous/GridContinuousProcessor.java     |    7 +-
 .../StartRoutineAckDiscoveryMessage.java        |   13 +-
 .../StartRoutineDiscoveryMessage.java           |   19 +-
 .../datastreamer/DataStreamProcessor.java       |   24 +-
 .../datastructures/DataStructuresProcessor.java |  132 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   14 +-
 .../GridCacheAtomicReferenceImpl.java           |   16 +-
 .../GridCacheAtomicSequenceImpl.java            |   26 +-
 .../GridCacheAtomicStampedImpl.java             |   14 +-
 .../GridCacheCountDownLatchImpl.java            |   14 +-
 .../datastructures/GridCacheLockImpl.java       |   17 +-
 .../datastructures/GridCacheSemaphoreImpl.java  |   14 +-
 .../processors/hadoop/HadoopClassLoader.java    |   11 +
 .../internal/processors/hadoop/HadoopJobId.java |   79 +-
 .../processors/hadoop/HadoopJobProperty.java    |   64 +-
 .../hadoop/HadoopMapperAwareTaskOutput.java     |   32 +
 .../processors/hadoop/HadoopTaskContext.java    |    8 +
 .../processors/hadoop/HadoopTaskInfo.java       |   43 +
 .../io/PartiallyOffheapRawComparatorEx.java     |   33 +
 .../hadoop/message/HadoopMessage.java           |   27 +
 .../shuffle/HadoopDirectShuffleMessage.java     |  243 ++
 .../hadoop/shuffle/HadoopShuffleAck.java        |  170 +
 .../shuffle/HadoopShuffleFinishRequest.java     |  172 +
 .../shuffle/HadoopShuffleFinishResponse.java    |  142 +
 .../hadoop/shuffle/HadoopShuffleMessage.java    |  361 ++
 .../internal/processors/igfs/IgfsContext.java   |    4 +-
 .../processors/igfs/IgfsDataManager.java        |    6 +-
 .../internal/processors/igfs/IgfsImpl.java      |    2 +-
 .../internal/processors/igfs/IgfsProcessor.java |    4 +-
 .../processors/job/GridJobProcessor.java        |    2 +-
 .../internal/processors/job/GridJobWorker.java  |   76 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |    2 +-
 .../processors/odbc/OdbcMessageParser.java      |   16 +
 .../processors/odbc/OdbcNioListener.java        |    2 +-
 .../internal/processors/odbc/OdbcProcessor.java |   11 +-
 .../odbc/OdbcQueryGetParamsMetaRequest.java     |   60 +
 .../odbc/OdbcQueryGetParamsMetaResult.java      |   40 +
 .../internal/processors/odbc/OdbcRequest.java   |    3 +
 .../processors/odbc/OdbcRequestHandler.java     |  181 +-
 .../internal/processors/odbc/OdbcTableMeta.java |   15 +-
 .../internal/processors/odbc/OdbcUtils.java     |    4 +-
 .../platform/PlatformAbstractTarget.java        |  268 +-
 .../platform/PlatformAsyncTarget.java           |   44 +
 .../platform/PlatformContextImpl.java           |    2 +-
 .../platform/PlatformNoopProcessor.java         |   41 +-
 .../processors/platform/PlatformProcessor.java  |   42 +-
 .../platform/PlatformProcessorImpl.java         |   89 +-
 .../processors/platform/PlatformTarget.java     |  103 +-
 .../platform/PlatformTargetProxy.java           |  126 +
 .../platform/PlatformTargetProxyImpl.java       |  222 ++
 .../binary/PlatformBinaryProcessor.java         |    6 +-
 .../platform/cache/PlatformCache.java           |   15 +-
 .../cache/PlatformCacheEntryFilterImpl.java     |    4 +-
 .../cache/PlatformCacheEntryProcessorImpl.java  |   43 +-
 .../platform/cache/PlatformCacheIterator.java   |    2 +-
 .../cache/affinity/PlatformAffinity.java        |    4 +-
 .../affinity/PlatformAffinityFunction.java      |   59 +-
 .../PlatformAffinityFunctionTarget.java         |    4 +-
 .../query/PlatformAbstractQueryCursor.java      |    4 +-
 .../query/PlatformContinuousQueryProxy.java     |    3 +-
 .../callback/PlatformCallbackGateway.java       |  272 +-
 .../platform/callback/PlatformCallbackOp.java   |  206 +
 .../callback/PlatformCallbackUtils.java         |  544 +--
 .../platform/cluster/PlatformClusterGroup.java  |   18 +-
 .../platform/compute/PlatformAbstractJob.java   |    2 +-
 .../platform/compute/PlatformAbstractTask.java  |   15 +-
 .../platform/compute/PlatformClosureJob.java    |   12 +-
 .../platform/compute/PlatformCompute.java       |   21 +-
 .../platform/compute/PlatformFullJob.java       |   15 +-
 .../platform/compute/PlatformFullTask.java      |   18 +-
 .../datastreamer/PlatformDataStreamer.java      |    4 +-
 .../PlatformStreamReceiverImpl.java             |   11 +-
 .../datastructures/PlatformAtomicLong.java      |    4 +-
 .../datastructures/PlatformAtomicReference.java |    8 +-
 .../datastructures/PlatformAtomicSequence.java  |    2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   17 +-
 .../platform/events/PlatformEvents.java         |   15 +-
 .../memory/PlatformInputStreamImpl.java         |   14 +-
 .../memory/PlatformOutputStreamImpl.java        |   14 +-
 .../platform/messaging/PlatformMessaging.java   |    9 +-
 .../services/PlatformAbstractService.java       |   25 +-
 .../platform/services/PlatformServices.java     |   29 +-
 .../transactions/PlatformTransactions.java      |    8 +-
 .../platform/utils/PlatformFutureUtils.java     |   18 +-
 .../utils/PlatformListenableTarget.java         |   62 +
 .../platform/utils/PlatformUtils.java           |   36 +-
 .../plugin/IgnitePluginProcessor.java           |    2 +-
 .../processors/port/GridPortProcessor.java      |    2 +-
 .../processors/query/GridQueryIndexing.java     |   42 +-
 .../processors/query/GridQueryProcessor.java    |  646 +++-
 .../processors/query/GridQueryProperty.java     |   20 +
 .../query/GridQueryTypeDescriptor.java          |   25 +
 .../processors/query/IgniteSQLException.java    |   89 +
 .../resource/GridResourceProcessor.java         |    2 +-
 .../processors/rest/GridRestCommand.java        |   11 +-
 .../processors/rest/GridRestProcessor.java      |   22 +-
 .../client/message/GridClientStateRequest.java  |   80 +
 .../cluster/GridChangeStateCommandHandler.java  |   89 +
 .../redis/GridRedisRestCommandHandler.java      |    1 +
 .../redis/key/GridRedisDelCommandHandler.java   |    7 +-
 .../key/GridRedisExistsCommandHandler.java      |    7 +-
 .../server/GridRedisDbSizeCommandHandler.java   |    7 +-
 .../string/GridRedisAppendCommandHandler.java   |    7 +-
 .../string/GridRedisGetCommandHandler.java      |   34 +-
 .../string/GridRedisGetRangeCommandHandler.java |    7 +-
 .../string/GridRedisGetSetCommandHandler.java   |    7 +-
 .../string/GridRedisIncrDecrCommandHandler.java |   70 +-
 .../string/GridRedisMGetCommandHandler.java     |    7 +-
 .../string/GridRedisMSetCommandHandler.java     |    7 +-
 .../string/GridRedisSetCommandHandler.java      |   31 +-
 .../string/GridRedisSetRangeCommandHandler.java |    7 +-
 .../string/GridRedisStrlenCommandHandler.java   |    7 +-
 .../tcp/GridTcpMemcachedNioListener.java        |   15 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |   22 +-
 .../tcp/redis/GridRedisNioListener.java         |    4 +-
 .../request/GridRestChangeStateRequest.java     |   57 +
 .../service/GridServiceProcessor.java           |  144 +-
 .../processors/service/GridServiceProxy.java    |   18 +-
 .../session/GridTaskSessionProcessor.java       |    2 +-
 .../processors/task/GridTaskProcessor.java      |    6 +-
 .../processors/task/GridTaskWorker.java         |    7 +
 .../timeout/GridTimeoutProcessor.java           |    2 +-
 .../ignite/internal/util/GridHandleTable.java   |   10 +-
 .../apache/ignite/internal/util/GridUnsafe.java |   70 +-
 .../ignite/internal/util/IgniteUtils.java       |  121 +-
 .../internal/util/OffheapReadWriteLock.java     |    4 +
 .../ignite/internal/util/StripedExecutor.java   |  667 ++++
 .../util/future/GridCompoundFuture.java         |   56 +-
 .../util/future/GridFinishedFuture.java         |   24 +
 .../internal/util/future/GridFutureAdapter.java |   15 +-
 .../util/future/GridFutureChainListener.java    |   30 +-
 .../internal/util/io/GridUnsafeDataInput.java   |   12 +-
 .../internal/util/io/GridUnsafeDataOutput.java  |   12 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    2 +-
 .../util/lang/IgniteSingletonIterator.java      |   56 +
 .../util/nio/GridCommunicationClient.java       |    4 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   15 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   10 +-
 .../internal/util/nio/GridNioCodecFilter.java   |   17 +-
 .../ignite/internal/util/nio/GridNioFilter.java |   16 +-
 .../internal/util/nio/GridNioFilterAdapter.java |   10 +-
 .../internal/util/nio/GridNioFilterChain.java   |   14 +-
 .../ignite/internal/util/nio/GridNioFuture.java |    4 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   95 +-
 .../ignite/internal/util/nio/GridNioServer.java | 1343 +++++--
 .../internal/util/nio/GridNioSession.java       |   11 +
 .../internal/util/nio/GridNioSessionImpl.java   |   49 +-
 .../ignite/internal/util/nio/GridNioWorker.java |   48 +
 .../util/nio/GridSelectorNioSessionImpl.java    |  157 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   49 +-
 .../internal/util/nio/SessionWriteRequest.java  |   85 +
 .../internal/util/nio/ssl/GridNioSslFilter.java |   10 +-
 .../util/nio/ssl/GridNioSslHandler.java         |    4 +-
 .../util/offheap/unsafe/GridUnsafeMemory.java   |   10 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../internal/visor/VisorMultiNodeTask.java      |    2 +-
 .../internal/visor/VisorTaskArgument.java       |    2 +-
 .../ignite/internal/visor/cache/VisorCache.java |   87 +-
 .../visor/cache/VisorCachePartition.java        |   14 +-
 .../visor/cache/VisorCachePartitions.java       |   10 +-
 .../visor/cache/VisorCachePartitionsTask.java   |    4 +-
 .../visor/cache/VisorCacheSwapBackupsTask.java  |  102 -
 .../visor/event/VisorGridDiscoveryEvent.java    |   18 +-
 .../visor/event/VisorGridDiscoveryEventV2.java  |   80 -
 .../misc/VisorChangeGridActiveStateTask.java    |   65 +
 .../visor/node/VisorNodeDataCollectorJob.java   |    3 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    2 +
 .../node/VisorNodeDataCollectorTaskResult.java  |   17 +
 .../internal/visor/query/VisorQueryJob.java     |    2 +-
 .../visor/service/VisorCancelServiceTask.java   |   70 +
 .../visor/service/VisorServiceDescriptor.java   |  132 +
 .../visor/service/VisorServiceTask.java         |   75 +
 .../internal/visor/util/VisorEventMapper.java   |    4 +-
 .../internal/visor/util/VisorTaskUtils.java     |   38 +-
 .../apache/ignite/logger/java/JavaLogger.java   |    4 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    4 +-
 .../optimized/OptimizedMarshaller.java          |    8 +-
 .../optimized/OptimizedObjectOutputStream.java  |   25 +-
 .../org/apache/ignite/mxbean/IgniteMXBean.java  |   21 +
 .../communication/tcp/TcpCommunicationSpi.java  |  324 +-
 .../tcp/TcpCommunicationSpiMBean.java           |   40 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   57 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   22 +-
 .../tcp/internal/TcpDiscoveryNode.java          |    5 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |    4 +
 .../org/apache/ignite/stream/StreamAdapter.java |    4 +-
 .../ignite/thread/IgniteThreadFactory.java      |    8 +-
 .../apache/ignite/util/AttributeNodeFilter.java |  108 +
 .../org/jsr166/ConcurrentLinkedHashMap.java     |    2 +-
 .../resources/META-INF/classnames.properties    |  174 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../AbstractAffinityFunctionSelfTest.java       |    2 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   |  283 ++
 .../IgniteComputeTopologyExceptionTest.java     |    5 +-
 .../BinaryArrayIdentityResolverSelfTest.java    |  300 ++
 .../internal/binary/BinaryEnumsSelfTest.java    |   18 +
 .../BinaryFieldIdentityResolverSelfTest.java    |  333 ++
 .../binary/BinaryFieldsOffheapSelfTest.java     |    2 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |    2 +-
 ...ryIdentityResolverConfigurationSelfTest.java |  138 +
 .../binary/BinaryMarshallerSelfTest.java        |   68 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |  157 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java |    2 +-
 .../BinarySerialiedFieldComparatorSelfTest.java |  568 +++
 ...GridManagerLocalMessageListenerSelfTest.java |    8 +-
 .../managers/GridNoopManagerSelfTest.java       |    2 +-
 ...unicationBalanceMultipleConnectionsTest.java |   28 +
 .../IgniteCommunicationBalanceTest.java         |  339 ++
 .../communication/IgniteIoTestMessagesTest.java |   95 +
 .../IgniteVariousConnectionNumberTest.java      |  166 +
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    2 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |    2 +-
 .../cache/CacheRebalancingSelfTest.java         |   75 +
 .../CacheSerializableTransactionsTest.java      |  604 ++-
 .../cache/CrossCacheTxRandomOperationsTest.java |   30 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |    4 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   11 +-
 ...GridCacheConcurrentGetCacheOnClientTest.java |  129 +
 .../GridCacheMissingCommitVersionSelfTest.java  |    6 +-
 .../cache/GridCacheMvccFlagsTest.java           |    8 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |  334 +-
 .../processors/cache/GridCacheMvccSelfTest.java |  212 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |    6 +-
 .../GridCachePartitionedAffinitySpreadTest.java |    7 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   80 +-
 .../GridCacheTtlManagerEvictionSelfTest.java    |    1 +
 .../IgniteTxStoreExceptionAbstractSelfTest.java |    1 +
 ...heapCacheMetricsForClusterGroupSelfTest.java |  141 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  260 +-
 .../database/tree/io/TrackingPageIOTest.java    |    4 +-
 ...achePartitionedAtomicSequenceTxSelfTest.java |  169 +
 .../CacheLockReleaseNodeLeaveTest.java          |  135 +
 ...eAtomicMessageRecovery10ConnectionsTest.java |   28 +
 ...cMessageRecoveryNoPairedConnectionsTest.java |   47 +
 ...acheConnectionRecovery10ConnectionsTest.java |   35 +
 .../distributed/IgniteCacheCreatePutTest.java   |    2 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |  232 --
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    6 +-
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |    2 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   37 +-
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 .../GridCacheRebalancingSyncSelfTest.java       |    2 +
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |   40 +
 .../CacheOffHeapAndSwapMetricsSelfTest.java     |  617 ---
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |  617 +++
 ...ContinuousQueryFailoverAbstractSelfTest.java |    2 +-
 .../TxDeadlockDetectionNoHangsTest.java         |    2 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   29 +-
 .../closure/GridClosureSerializationTest.java   |  177 +
 ...lockMessageSystemPoolStarvationSelfTest.java |   14 +-
 .../GridServiceProcessorProxySelfTest.java      |    2 +-
 ...gniteServiceProxyTimeoutInitializedTest.java |  284 ++
 .../util/future/GridFutureAdapterSelfTest.java  |  122 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   18 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |    7 +-
 .../loadtests/nio/GridNioBenchmarkClient.java   |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    2 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   37 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   13 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    6 +-
 ...ationSpiRecoveryNoPairedConnectionsTest.java |   28 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    8 +
 ...CommunicationRecoveryAckClosureSelfTest.java |    6 +-
 .../ignite/testframework/GridTestUtils.java     |   80 +-
 .../testframework/junits/GridAbstractTest.java  |    2 +-
 .../junits/GridTestKernalContext.java           |    6 +-
 .../junits/logger/GridTestLog4jLogger.java      |    4 +-
 .../multijvm/IgniteCacheProcessProxy.java       |    5 +
 .../junits/multijvm/IgniteNodeRunner.java       |    2 +-
 .../junits/multijvm/IgniteProcessProxy.java     |    2 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |    5 +
 .../IgniteBinaryObjectsTestSuite.java           |   10 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |    5 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    4 +
 .../IgniteCacheMetricsSelfTestSuite.java        |    6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   17 +
 .../testsuites/IgniteCacheTestSuite4.java       |    3 +
 .../testsuites/IgniteCacheTestSuite5.java       |    4 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    2 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |    2 +
 .../util/AttributeNodeFilterSelfTest.java       |  184 +
 modules/docker/1.8.0/Dockerfile                 |   46 +
 modules/docker/1.8.0/run.sh                     |   51 +
 modules/docker/Dockerfile                       |    6 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flink/pom.xml                           |    2 +-
 modules/flume/pom.xml                           |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    9 +-
 .../hadoop/io/PartiallyRawComparator.java       |   33 +
 .../org/apache/ignite/hadoop/io/RawMemory.java  |   86 +
 .../hadoop/io/TextPartiallyRawComparator.java   |  115 +
 .../apache/ignite/hadoop/io/package-info.java   |   22 +
 .../IgniteHadoopClientProtocolProvider.java     |   70 +-
 .../processors/hadoop/HadoopMapperUtils.java    |   56 +
 .../processors/hadoop/HadoopProcessor.java      |    6 +-
 .../hadoop/impl/fs/HadoopFileSystemsUtils.java  |   11 +
 .../hadoop/impl/proto/HadoopClientProtocol.java |   55 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |   11 +
 ...DelegatingPartiallyOffheapRawComparator.java |   54 +
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   22 +-
 .../impl/v2/HadoopV2JobResourceManager.java     |   25 +-
 .../hadoop/impl/v2/HadoopV2MapTask.java         |   10 +
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |   21 +
 .../processors/hadoop/io/OffheapRawMemory.java  |  131 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    8 +-
 .../hadoop/mapreduce/MapReduceClient.java       |  147 +
 .../hadoop/message/HadoopMessage.java           |   27 -
 .../hadoop/shuffle/HadoopShuffle.java           |  112 +-
 .../hadoop/shuffle/HadoopShuffleAck.java        |   92 -
 .../hadoop/shuffle/HadoopShuffleJob.java        |  748 +++-
 .../hadoop/shuffle/HadoopShuffleLocalState.java |   67 +
 .../hadoop/shuffle/HadoopShuffleMessage.java    |  242 --
 .../shuffle/HadoopShuffleRemoteState.java       |   61 +
 .../shuffle/collections/HadoopMultimapBase.java |    5 +-
 .../shuffle/collections/HadoopSkipList.java     |   14 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |  166 +
 .../shuffle/direct/HadoopDirectDataOutput.java  |  221 ++
 .../direct/HadoopDirectDataOutputContext.java   |  100 +
 .../direct/HadoopDirectDataOutputState.java     |   54 +
 .../shuffle/streams/HadoopDataOutStream.java    |    2 +-
 .../child/HadoopChildProcessRunner.java         |   12 +-
 .../HadoopExternalCommunication.java            |    4 +-
 .../communication/HadoopIpcToNioAdapter.java    |    2 +-
 .../communication/HadoopMarshallerFilter.java   |    6 +-
 .../resources/META-INF/classnames.properties    |  114 +
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   13 +-
 .../impl/HadoopAbstractWordCountTest.java       |    6 +-
 .../hadoop/impl/HadoopFileSystemsTest.java      |    9 +
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |    4 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |   22 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |    4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |  383 ++
 ...opClientProtocolMultipleServersSelfTest.java |   93 +-
 .../client/HadoopClientProtocolSelfTest.java    |  232 +-
 .../collections/HadoopAbstractMapTest.java      |    6 +
 .../HadoopConcurrentHashMultimapSelftest.java   |    2 +-
 .../collections/HadoopSkipListSelfTest.java     |   16 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |    2 +
 .../testsuites/IgniteHadoopTestSuite.java       |    3 +
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../cache/query/GridCacheTwoStepQuery.java      |  253 ++
 .../query/h2/DmlStatementsProcessor.java        | 1083 ++++++
 .../query/h2/GridH2ResultSetIterator.java       |    3 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  377 +-
 .../query/h2/database/H2PkHashIndex.java        |  322 ++
 .../query/h2/dml/FastUpdateArgument.java        |   27 +
 .../query/h2/dml/FastUpdateArguments.java       |   53 +
 .../query/h2/dml/KeyValueSupplier.java          |   30 +
 .../processors/query/h2/dml/UpdateMode.java     |   36 +
 .../processors/query/h2/dml/UpdatePlan.java     |  128 +
 .../query/h2/dml/UpdatePlanBuilder.java         |  486 +++
 .../processors/query/h2/dml/package-info.java   |   22 +
 .../query/h2/opt/GridH2RowDescriptor.java       |   23 +
 .../processors/query/h2/opt/GridH2Table.java    |  141 +-
 .../query/h2/opt/GridH2TreeIndex.java           |    2 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |  616 +++
 .../processors/query/h2/sql/GridSqlArray.java   |    8 +
 .../processors/query/h2/sql/GridSqlConst.java   |    6 +
 .../processors/query/h2/sql/GridSqlDelete.java  |   68 +
 .../query/h2/sql/GridSqlFunction.java           |    5 +-
 .../processors/query/h2/sql/GridSqlInsert.java  |  149 +
 .../processors/query/h2/sql/GridSqlKeyword.java |   46 +
 .../processors/query/h2/sql/GridSqlMerge.java   |  143 +
 .../processors/query/h2/sql/GridSqlQuery.java   |   44 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  323 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    6 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |    3 +-
 .../query/h2/sql/GridSqlStatement.java          |   64 +
 .../processors/query/h2/sql/GridSqlUpdate.java  |  105 +
 .../h2/twostep/GridReduceQueryExecutor.java     |   61 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  559 +++
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |  243 ++
 .../IgniteCacheDeleteSqlQuerySelfTest.java      |  106 +
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  203 +
 .../cache/IgniteCacheLargeResultSelfTest.java   |   15 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  153 +
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |  472 +++
 .../IgniteCacheAtomicFieldsQuerySelfTest.java   |   21 -
 ...niteCachePartitionedFieldsQuerySelfTest.java |   25 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   37 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  109 +-
 .../IgniteCacheQuerySelfTestSuite.java          |  143 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |  111 +-
 .../IgniteH2IndexingSpiTestSuite.java           |   16 +-
 modules/jcl/pom.xml                             |    2 +-
 modules/jms11/pom.xml                           |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/kafka/pom.xml                           |    2 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |   50 +-
 .../kafka/KafkaIgniteStreamerSelfTest.java      |   36 +-
 modules/log4j/pom.xml                           |    2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |    4 +-
 modules/log4j2/pom.xml                          |    2 +-
 modules/mesos/pom.xml                           |    2 +-
 modules/mqtt/pom.xml                            |    2 +-
 modules/osgi-karaf/pom.xml                      |    2 +-
 modules/osgi-paxlogging/pom.xml                 |    2 +-
 modules/osgi/pom.xml                            |    3 +-
 modules/platforms/.gitignore                    |   31 +
 .../include/ignite/binary/binary_raw_reader.h   |   21 +
 .../ignite/impl/binary/binary_reader_impl.h     |    9 +-
 .../src/impl/binary/binary_reader_impl.cpp      |   20 +-
 .../platforms/cpp/common/include/Makefile.am    |    5 +-
 .../common/include/ignite/common/concurrent.h   |   90 +-
 .../include/ignite/common/reference_impl.h      |  286 ++
 .../cpp/common/include/ignite/reference.h       |  564 +++
 .../cpp/common/project/vs/common.vcxproj        |    2 +
 .../common/project/vs/common.vcxproj.filters    |    6 +
 modules/platforms/cpp/configure.ac              |    2 +-
 modules/platforms/cpp/configure.acrel           |    2 +-
 modules/platforms/cpp/core-test/Makefile.am     |    7 +-
 .../core-test/config/cache-query-continuous.xml |   87 +
 .../cpp/core-test/config/cache-query.xml        |   10 +
 .../cpp/core-test/config/cache-test.xml         |    1 +
 .../platforms/cpp/core-test/config/invalid.xml  |   39 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |   13 +-
 .../project/vs/core-test.vcxproj.filters        |   11 +-
 .../cpp/core-test/src/cache_query_test.cpp      |   76 +-
 .../cpp/core-test/src/continuous_query_test.cpp |  611 +++
 .../cpp/core-test/src/handle_registry_test.cpp  |   18 +-
 .../cpp/core-test/src/interop_test.cpp          |   17 +-
 .../cpp/core-test/src/reference_test.cpp        |  412 ++
 modules/platforms/cpp/core/Makefile.am          |    1 +
 .../cpp/core/include/ignite/cache/cache.h       |  104 +-
 .../cpp/core/include/ignite/cache/cache_entry.h |   40 +-
 .../ignite/cache/event/cache_entry_event.h      |  139 +
 .../cache/event/cache_entry_event_listener.h    |   71 +
 .../cache/query/continuous/continuous_query.h   |  239 ++
 .../query/continuous/continuous_query_handle.h  |  133 +
 .../core/include/ignite/impl/cache/cache_impl.h |  116 +-
 .../continuous/continuous_query_handle_impl.h   |  101 +
 .../query/continuous/continuous_query_impl.h    |  351 ++
 .../core/include/ignite/impl/handle_registry.h  |   62 +-
 .../include/ignite/impl/ignite_environment.h    |   34 +-
 modules/platforms/cpp/core/namespaces.dox       |   74 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |    7 +
 .../cpp/core/project/vs/core.vcxproj.filters    |   30 +
 modules/platforms/cpp/core/src/ignition.cpp     |   15 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   31 +
 .../continuous/continuous_query_handle_impl.cpp |   96 +
 .../cpp/core/src/impl/handle_registry.cpp       |  102 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  146 +-
 modules/platforms/cpp/examples/Makefile.am      |    1 +
 modules/platforms/cpp/examples/configure.ac     |    3 +-
 .../continuous-query-example/Makefile.am        |   58 +
 .../config/continuous-query-example.xml         |   52 +
 .../project/vs/continuous-query-example.vcxproj |  110 +
 .../vs/continuous-query-example.vcxproj.filters |   35 +
 .../src/continuous_query_example.cpp            |  142 +
 .../examples/include/ignite/examples/person.h   |    2 +-
 .../odbc-example/config/example-odbc.xml        |   38 +-
 .../project/vs/odbc-example.vcxproj             |    7 +-
 .../project/vs/odbc-example.vcxproj.filters     |    8 +
 .../examples/odbc-example/src/odbc_example.cpp  |  514 ++-
 .../cpp/examples/project/vs/ignite-examples.sln |    6 +
 .../putget-example/src/putget_example.cpp       |    2 +-
 .../query-example/src/query_example.cpp         |    4 +-
 .../cpp/jni/include/ignite/jni/exports.h        |    3 -
 .../platforms/cpp/jni/include/ignite/jni/java.h |   94 +-
 modules/platforms/cpp/jni/project/vs/module.def |    6 +-
 modules/platforms/cpp/jni/src/exports.cpp       |    8 -
 modules/platforms/cpp/jni/src/java.cpp          |  419 +-
 .../cpp/odbc-test/config/queries-default.xml    |  145 +
 .../odbc-test/config/queries-test-noodbc.xml    |   80 +-
 .../cpp/odbc-test/config/queries-test.xml       |   81 +-
 .../platforms/cpp/odbc-test/include/Makefile.am |    1 +
 .../cpp/odbc-test/include/complex_type.h        |  122 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    5 +-
 .../project/vs/odbc-test.vcxproj.filters        |    3 +
 .../src/application_data_buffer_test.cpp        |    2 +-
 .../cpp/odbc-test/src/queries_test.cpp          |  513 ++-
 .../cpp/odbc-test/src/utility_test.cpp          |   27 +-
 .../platforms/cpp/odbc/include/ignite/odbc.h    |   14 +-
 .../ignite/odbc/app/application_data_buffer.h   |    4 +-
 .../odbc/include/ignite/odbc/app/parameter.h    |    2 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |    3 +
 .../cpp/odbc/include/ignite/odbc/message.h      |  138 +-
 .../odbc/include/ignite/odbc/query/data_query.h |   12 +-
 .../cpp/odbc/include/ignite/odbc/query/query.h  |   44 +-
 .../cpp/odbc/include/ignite/odbc/statement.h    |   83 +-
 .../cpp/odbc/include/ignite/odbc/type_traits.h  |    2 +-
 .../cpp/odbc/include/ignite/odbc/utility.h      |   11 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |    2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |    2 +-
 .../odbc/src/app/application_data_buffer.cpp    |   57 +-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |    3 +-
 modules/platforms/cpp/odbc/src/column.cpp       |   41 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |   23 +-
 modules/platforms/cpp/odbc/src/entry_points.cpp |   32 +-
 modules/platforms/cpp/odbc/src/odbc.cpp         |   25 +-
 .../odbc/src/query/column_metadata_query.cpp    |    2 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |    6 +-
 .../cpp/odbc/src/query/foreign_keys_query.cpp   |    2 +-
 .../cpp/odbc/src/query/primary_keys_query.cpp   |    2 +-
 .../odbc/src/query/special_columns_query.cpp    |    2 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |    2 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |    2 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |  230 +-
 modules/platforms/cpp/odbc/src/type_traits.cpp  |    3 +
 modules/platforms/cpp/odbc/src/utility.cpp      |   24 +-
 modules/platforms/cpp/project/vs/ignite.slnrel  |    3 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |    3 +
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.AspNet.nuspec                 |    4 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Binary/BinaryCompactFooterInteropTest.cs    |   31 +-
 .../Cache/Query/CacheLinqTest.cs                |   72 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   17 +-
 .../Cache/Query/CacheQueriesTest.cs             |    8 +
 .../Examples/Example.cs                         |   11 +-
 .../Examples/ExamplesTest.cs                    |  104 +-
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |   64 +-
 .../Process/IgniteProcess.cs                    |   21 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |   23 +-
 .../Services/ServicesTest.cs                    |   46 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |    3 +-
 .../Apache.Ignite.Core.csproj                   |    2 +
 .../dotnet/Apache.Ignite.Core/Binary/IBinary.cs |    3 +-
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   15 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |    1 +
 .../Impl/Binary/BinaryUtils.cs                  |   16 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |    4 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   29 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |   13 +-
 .../Impl/Common/Listenable.cs                   |   49 +
 .../Impl/Compute/ComputeImpl.cs                 |    4 +-
 .../Impl/Compute/ComputeTaskHolder.cs           |   14 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |    2 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |    8 -
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   79 +-
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |   86 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1229 +++---
 .../Impl/Unmanaged/UnmanagedUtils.cs            |    5 -
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../EntityFrameworkCacheTest.cs                 |   58 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Impl/DbCommandInfo.cs                       |   21 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |   12 +-
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |    8 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |   34 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |    7 +
 .../Apache.Ignite/Config/ArgsConfigurator.cs    |    7 +-
 .../dotnet/Apache.Ignite/Config/Configurator.cs |   10 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    6 +-
 modules/platforms/dotnet/DEVNOTES.txt           |   12 +-
 modules/platforms/dotnet/README.md              |  150 +
 modules/platforms/dotnet/build.bat              |   19 +
 modules/platforms/dotnet/build.ps1              |  211 +
 .../Datagrid/MultiTieredCacheExample.cs         |    8 +-
 .../Misc/ClientReconnectExample.cs              |   18 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 .../schedule/IgniteScheduleProcessor.java       |    2 +-
 modules/schema-import-db/pom.xml                |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spark-2.10/pom.xml                      |    2 +-
 modules/spark/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/storm/pom.xml                           |    2 +-
 modules/tools/pom.xml                           |    2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |    8 +-
 modules/twitter/pom.xml                         |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |    2 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../ignite/visor/commands/VisorConsole.scala    |    1 -
 .../commands/cache/VisorCacheCommand.scala      |   37 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  145 -
 .../commands/disco/VisorDiscoveryCommand.scala  |    2 +-
 .../commands/events/VisorEventsCommand.scala    |    8 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |   89 -
 .../testsuites/VisorConsoleSelfTestSuite.scala  |    2 -
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web-console/backend/app/agent.js        |   47 +-
 modules/web-console/backend/app/browser.js      |   26 +
 modules/web-console/backend/app/mongo.js        |   30 +-
 .../backend/config/settings.json.sample         |    7 -
 modules/web-console/backend/index.js            |    6 +-
 modules/web-console/backend/middlewares/host.js |    7 +-
 modules/web-console/backend/routes/agent.js     |    4 +-
 modules/web-console/backend/routes/demo.js      |   17 +-
 modules/web-console/backend/routes/profile.js   |    3 +-
 modules/web-console/backend/services/agents.js  |    2 +-
 modules/web-console/backend/services/caches.js  |    2 +
 .../web-console/backend/services/notebooks.js   |   14 +-
 .../web-console/backend/services/sessions.js    |    6 +-
 modules/web-console/backend/services/spaces.js  |   15 +
 .../docker/compose/backend/.dockerignore        |    2 +
 .../docker/compose/backend/Dockerfile           |    6 +-
 .../web-console/docker/compose/backend/build.sh |    4 +-
 .../docker/compose/docker-compose.yml           |    5 -
 .../docker/compose/frontend/DockerfileBuild     |    4 +-
 .../docker/compose/frontend/build.sh            |    6 +-
 .../compose/frontend/nginx/web-console.conf     |    9 +
 .../web-console/docker/standalone/.dockerignore |    5 +
 .../web-console/docker/standalone/Dockerfile    |   10 +-
 modules/web-console/docker/standalone/build.sh  |    4 +-
 .../docker/standalone/docker-compose.yml        |    7 +-
 .../docker/standalone/nginx/web-console.conf    |    9 +
 modules/web-console/frontend/app/app.js         |    5 -
 .../controllers/reset-password.controller.js    |   14 +-
 .../frontend/app/data/event-groups.json         |  169 +
 .../frontend/app/data/event-types.json          |  169 -
 .../frontend/app/data/pom-dependencies.json     |   14 +-
 .../ui-ace-docker/ui-ace-docker.controller.js   |    2 +-
 .../directives/ui-ace-docker/ui-ace-docker.jade |    2 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |   12 +-
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |    2 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    4 +-
 .../frontend/app/filters/duration.filter.js     |    3 +
 .../helpers/jade/form/form-field-checkbox.jade  |    2 +-
 .../helpers/jade/form/form-field-datalist.jade  |    2 +-
 .../helpers/jade/form/form-field-dropdown.jade  |    5 +-
 .../helpers/jade/form/form-field-number.jade    |    3 +-
 .../helpers/jade/form/form-field-password.jade  |    2 +-
 .../app/helpers/jade/form/form-field-text.jade  |   19 +-
 .../frontend/app/helpers/jade/mixins.jade       |   76 +-
 .../frontend/app/modules/Demo/Demo.module.js    |    6 +-
 .../configuration/EventGroups.provider.js       |   30 -
 .../modules/configuration/Version.service.js    |    6 +-
 .../configuration/configuration.module.js       |   63 +-
 .../generator/AbstractTransformer.js            |   17 +
 .../modules/configuration/generator/Beans.js    |   11 +
 .../generator/ConfigurationGenerator.js         | 2776 +++++++-------
 .../configuration/generator/Custom.service.js   |   23 +
 .../configuration/generator/Docker.service.js   |    4 +-
 .../generator/JavaTransformer.service.js        | 2315 +++++------
 .../configuration/generator/Maven.service.js    |  234 ++
 .../configuration/generator/Pom.service.js      |  233 --
 .../generator/Properties.service.js             |   21 +-
 .../configuration/generator/Readme.service.js   |    2 +-
 .../generator/SharpTransformer.service.js       |  437 ++-
 .../generator/SpringTransformer.service.js      |  489 +--
 .../defaults/Cache.platform.service.js          |   56 +
 .../generator/defaults/Cache.service.js         |  131 +
 .../defaults/Cluster.platform.service.js        |   43 +
 .../generator/defaults/Cluster.service.js       |  289 ++
 .../generator/defaults/Event-groups.service.js  |   27 +
 .../generator/defaults/IGFS.service.js          |   64 +
 .../defaults/cache.platform.provider.js         |   60 -
 .../generator/defaults/cache.provider.js        |  129 -
 .../defaults/cluster.platform.provider.js       |   49 -
 .../generator/defaults/cluster.provider.js      |  293 --
 .../generator/defaults/igfs.provider.js         |   68 -
 .../configuration/generator/generator-common.js |  625 ---
 .../configuration/generator/generator-java.js   | 3617 ------------------
 .../generator/generator-optional.js             |   25 -
 .../configuration/generator/generator-spring.js | 2111 ----------
 .../app/modules/form/field/tooltip.directive.js |    2 +-
 .../app/modules/form/group/tooltip.directive.js |    2 +-
 .../app/modules/form/panel/field.directive.js   |    4 +-
 .../frontend/app/modules/sql/Notebook.data.js   |   11 +-
 .../app/modules/sql/Notebook.service.js         |    2 +-
 .../app/modules/sql/scan-filter-input.jade      |   39 -
 .../modules/sql/scan-filter-input.service.js    |   51 -
 .../frontend/app/modules/sql/sql.controller.js  |  217 +-
 .../frontend/app/modules/sql/sql.module.js      |    2 -
 .../app/modules/states/configuration.state.js   |    2 +
 .../configuration/caches/client-near-cache.jade |    2 +-
 .../configuration/caches/concurrency.jade       |    2 +-
 .../states/configuration/caches/general.jade    |    2 +-
 .../states/configuration/caches/memory.jade     |    2 +-
 .../configuration/caches/near-cache-client.jade |    2 +-
 .../configuration/caches/near-cache-server.jade |    2 +-
 .../configuration/caches/node-filter.jade       |    4 +-
 .../states/configuration/caches/query.jade      |    5 +-
 .../states/configuration/caches/rebalance.jade  |    2 +-
 .../states/configuration/caches/statistics.jade |    2 +-
 .../states/configuration/caches/store.jade      |   20 +-
 .../states/configuration/clusters/atomic.jade   |    2 +-
 .../configuration/clusters/attributes.jade      |    2 +-
 .../states/configuration/clusters/binary.jade   |    2 +-
 .../configuration/clusters/cache-key-cfg.jade   |    2 +-
 .../configuration/clusters/checkpoint.jade      |   17 +-
 .../configuration/clusters/checkpoint/fs.jade   |   10 +-
 .../configuration/clusters/checkpoint/jdbc.jade |   47 +-
 .../configuration/clusters/checkpoint/s3.jade   |  244 +-
 .../configuration/clusters/collision.jade       |   12 +-
 .../clusters/collision/custom.jade              |    4 +-
 .../clusters/collision/fifo-queue.jade          |    2 +-
 .../clusters/collision/job-stealing.jade        |    4 +-
 .../clusters/collision/priority-queue.jade      |    2 +-
 .../configuration/clusters/communication.jade   |    2 +-
 .../configuration/clusters/connector.jade       |    2 +-
 .../configuration/clusters/deployment.jade      |  131 +-
 .../configuration/clusters/discovery.jade       |    2 +-
 .../states/configuration/clusters/events.jade   |    6 +-
 .../states/configuration/clusters/failover.jade |    6 +-
 .../states/configuration/clusters/general.jade  |    2 +-
 .../clusters/general/discovery/cloud.jade       |    2 +-
 .../clusters/general/discovery/google.jade      |    2 +-
 .../clusters/general/discovery/jdbc.jade        |    3 +-
 .../clusters/general/discovery/multicast.jade   |    2 +-
 .../clusters/general/discovery/s3.jade          |    2 +-
 .../clusters/general/discovery/shared.jade      |    2 +-
 .../clusters/general/discovery/vm.jade          |    2 +-
 .../clusters/general/discovery/zookeeper.jade   |    4 +-
 .../bounded-exponential-backoff.jade            |    2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |    4 +-
 .../retrypolicy/exponential-backoff.jade        |    2 +-
 .../zookeeper/retrypolicy/forever.jade          |    2 +-
 .../zookeeper/retrypolicy/n-times.jade          |    2 +-
 .../zookeeper/retrypolicy/one-time.jade         |    2 +-
 .../zookeeper/retrypolicy/until-elapsed.jade    |    2 +-
 .../states/configuration/clusters/igfs.jade     |    2 +-
 .../configuration/clusters/load-balancing.jade  |   25 +-
 .../states/configuration/clusters/logger.jade   |    2 +-
 .../configuration/clusters/logger/custom.jade   |    4 +-
 .../configuration/clusters/logger/log4j.jade    |    2 +-
 .../configuration/clusters/logger/log4j2.jade   |    2 +-
 .../configuration/clusters/marshaller.jade      |    2 +-
 .../states/configuration/clusters/metrics.jade  |    2 +-
 .../states/configuration/clusters/odbc.jade     |    2 +-
 .../states/configuration/clusters/ssl.jade      |    4 +-
 .../states/configuration/clusters/swap.jade     |    2 +-
 .../states/configuration/clusters/thread.jade   |    2 +-
 .../states/configuration/clusters/time.jade     |    2 +-
 .../configuration/clusters/transactions.jade    |    2 +-
 .../states/configuration/domains/general.jade   |    2 +-
 .../states/configuration/domains/query.jade     |   16 +-
 .../states/configuration/domains/store.jade     |   12 +-
 .../modules/states/configuration/igfs/dual.jade |    2 +-
 .../states/configuration/igfs/fragmentizer.jade |    2 +-
 .../states/configuration/igfs/general.jade      |    2 +-
 .../modules/states/configuration/igfs/ipc.jade  |    2 +-
 .../modules/states/configuration/igfs/misc.jade |    6 +-
 .../states/configuration/igfs/secondary.jade    |    2 +-
 .../summary/summary-zipper.service.js           |   37 +
 .../configuration/summary/summary.controller.js |   99 +-
 .../configuration/summary/summary.worker.js     |  123 +
 .../frontend/app/modules/user/Auth.service.js   |   11 +-
 .../frontend/app/services/JavaTypes.service.js  |   13 +-
 .../frontend/app/services/Messages.service.js   |   17 +-
 .../frontend/controllers/admin-controller.js    |  211 +-
 .../frontend/controllers/caches-controller.js   |   22 +-
 .../frontend/controllers/clusters-controller.js |   52 +-
 .../frontend/controllers/domains-controller.js  |   40 +-
 .../frontend/controllers/igfs-controller.js     |   20 +-
 .../frontend/controllers/profile-controller.js  |    3 +-
 .../frontend/gulpfile.babel.js/tasks/jade.js    |    4 +-
 .../frontend/gulpfile.babel.js/tasks/test.js    |   92 -
 .../gulpfile.babel.js/webpack/common.js         |   22 +-
 .../webpack/environments/development.js         |   17 +-
 .../webpack/environments/production.js          |    3 +-
 .../webpack/plugins/progress.js                 |   82 -
 modules/web-console/frontend/package.json       |  180 +-
 .../frontend/public/images/cache.png            |  Bin 23700 -> 24791 bytes
 .../frontend/public/images/domains.png          |  Bin 23828 -> 22131 bytes
 .../web-console/frontend/public/images/igfs.png |  Bin 14307 -> 14139 bytes
 .../frontend/public/images/query-chart.png      |  Bin 16637 -> 17142 bytes
 .../frontend/public/images/query-metadata.png   |  Bin 32298 -> 39361 bytes
 .../frontend/public/images/query-table.png      |  Bin 29189 -> 28065 bytes
 .../frontend/public/images/summary.png          |  Bin 31997 -> 33650 bytes
 .../stylesheets/_font-awesome-custom.scss       |   39 +
 .../frontend/public/stylesheets/form-field.scss |   37 +
 .../frontend/public/stylesheets/style.scss      |  124 +-
 .../frontend/test/unit/JavaTypes.test.js        |   17 +-
 .../frontend/test/unit/Version.test.js          |    8 +-
 .../frontend/views/configuration/caches.jade    |   22 +-
 .../frontend/views/configuration/clusters.jade  |   50 +-
 .../views/configuration/domains-import.jade     |    9 +-
 .../frontend/views/configuration/domains.jade   |    8 +-
 .../frontend/views/configuration/igfs.jade      |   14 +-
 .../frontend/views/configuration/summary.jade   |   27 +-
 .../frontend/views/includes/header.jade         |   15 +-
 .../frontend/views/settings/admin.jade          |   85 +-
 .../frontend/views/settings/profile.jade        |    2 +-
 .../frontend/views/sql/notebook-new.jade        |    2 +-
 modules/web-console/frontend/views/sql/sql.jade |  237 +-
 .../views/templates/agent-download.jade         |    2 +-
 .../frontend/views/templates/alert.jade         |    2 +-
 .../frontend/views/templates/batch-confirm.jade |    2 +-
 .../frontend/views/templates/clone.jade         |    2 +-
 .../frontend/views/templates/confirm.jade       |    2 +-
 .../frontend/views/templates/select.jade        |    2 +-
 modules/web-console/pom.xml                     |    2 +-
 modules/web-console/web-agent/README.txt        |    4 +-
 .../web-agent/bin/ignite-web-agent.bat          |    7 +-
 .../web-agent/bin/ignite-web-agent.sh           |    6 +-
 modules/web-console/web-agent/pom.xml           |    2 +-
 .../console/agent/AgentConfiguration.java       |    5 +-
 .../ignite/console/agent/AgentLauncher.java     |    6 +-
 .../ignite/console/demo/AgentClusterDemo.java   |    1 +
 modules/web/ignite-appserver-test/pom.xml       |    2 +-
 modules/web/ignite-websphere-test/pom.xml       |    2 +-
 modules/web/pom.xml                             |    2 +-
 .../config/benchmark-bin-identity.properties    |   94 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-sql-dml.properties         |   72 +
 modules/yardstick/config/ignite-base-config.xml |   71 +-
 .../config/ignite-bin-multicast-config.xml      |   86 +
 modules/yardstick/pom.xml                       |    2 +-
 .../yardstick/IgniteAbstractBenchmark.java      |   30 +
 .../yardstick/IgniteBenchmarkArguments.java     |   11 +
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |   42 +-
 .../apache/ignite/yardstick/PreloadLogger.java  |  155 +
 .../yardstick/cache/CacheEntryEventProbe.java   |    2 +-
 .../cache/IgniteBinaryIdentityBenchmark.java    |  108 +
 .../cache/IgniteBinaryIdentityGetBenchmark.java |   34 +
 .../cache/IgniteBinaryIdentityPutBenchmark.java |   35 +
 .../IgniteFieldsBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteFieldsBinaryIdentityPutBenchmark.java |   30 +
 .../yardstick/cache/IgniteIoTestBenchmark.java  |   73 +
 .../IgniteLegacyBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteLegacyBinaryIdentityPutBenchmark.java |   30 +
 ...IgnitePutIfAbsentIndexedValue1Benchmark.java |   45 +
 .../IgniteReplaceIndexedValue1Benchmark.java    |   79 +
 .../cache/dml/IgniteSqlDeleteBenchmark.java     |   83 +
 .../dml/IgniteSqlDeleteFilteredBenchmark.java   |   88 +
 .../IgniteSqlInsertIndexedValue1Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue2Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue8Benchmark.java  |   48 +
 .../cache/dml/IgniteSqlMergeAllBenchmark.java   |   82 +
 .../cache/dml/IgniteSqlMergeBenchmark.java      |   42 +
 .../IgniteSqlMergeIndexedValue1Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue2Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue8Benchmark.java   |   43 +
 .../cache/dml/IgniteSqlMergeQueryBenchmark.java |  116 +
 .../cache/dml/IgniteSqlUpdateBenchmark.java     |   82 +
 .../dml/IgniteSqlUpdateFilteredBenchmark.java   |   88 +
 .../IgniteCacheRandomOperationBenchmark.java    |   22 +-
 .../yardstick/cache/model/SampleValue.java      |    2 +
 .../io/IgniteIoTestAbstractBenchmark.java       |   61 +
 .../io/IgniteIoTestSendAllBenchmark.java        |   32 +
 .../io/IgniteIoTestSendRandomBenchmark.java     |   35 +
 modules/yarn/pom.xml                            |    2 +-
 modules/zookeeper/pom.xml                       |    2 +-
 pom.xml                                         |    2 +-
 1129 files changed, 51015 insertions(+), 23421 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index 830f781a,4ee5cce..79ac383
--- 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
@@@ -566,9 -593,9 +566,9 @@@ public class PageMemoryNoStoreImpl impl
  
              pageIdx &= idxMask;
  
-             long off = pageIdx * sysPageSize;
 -            long offset = ((long)pageIdx) * sysPageSize;
++            long off = ((long)pageIdx) * sysPageSize;
  
 -            return pagesBase + offset;
 +            return pagesBase + off;
          }
  
          /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
index 0000000,5633354..db213de
mode 000000,100644..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
@@@ -1,0 -1,59 +1,58 @@@
+ /*
+  * 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.wal.record.delta;
+ 
 -import java.nio.ByteBuffer;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
+ 
+ /**
+  *
+  */
+ public class MetaPageUpdateLastSuccessfulFullSnapshotId extends PageDeltaRecord {
+     /** */
+     private final long lastSuccessfulFullSnapshotId;
+ 
+     /**
+      * @param pageId Meta page ID.
+      */
+     public MetaPageUpdateLastSuccessfulFullSnapshotId(int cacheId, long pageId, long lastSuccessfulFullSnapshotId) {
+         super(cacheId, pageId);
+ 
+         this.lastSuccessfulFullSnapshotId = lastSuccessfulFullSnapshotId;
+     }
+ 
+     /** {@inheritDoc} */
 -    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
 -        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
++    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
++        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
+ 
 -        io.setLastSuccessfulFullSnapshotId(buf, lastSuccessfulFullSnapshotId);
++        io.setLastSuccessfulFullSnapshotId(pageAddr, lastSuccessfulFullSnapshotId);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public RecordType type() {
+         return RecordType.META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID;
+     }
+ 
+     /**
+      * @return Root ID.
+      */
+     public long lastSuccessfulFullSnapshotId() {
+         return lastSuccessfulFullSnapshotId;
+     }
+ }
+ 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
index 0000000,20e136b..ce2453e
mode 000000,100644..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
@@@ -1,0 -1,70 +1,70 @@@
+ /*
+  * 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.wal.record.delta;
+ 
+ import java.nio.ByteBuffer;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
+ 
+ /**
+  *
+  */
+ public class MetaPageUpdateLastSuccessfulSnapshotId extends PageDeltaRecord {
+     /** */
+     private final long lastSuccessfulSnapshotId;
+     /** Last successful snapshot tag. */
+     private final long lastSuccessfulSnapshotTag;
+ 
+     /**
+      * @param pageId Meta page ID.
+      * @param snapshotTag
+      */
+     public MetaPageUpdateLastSuccessfulSnapshotId(int cacheId, long pageId, long lastSuccessfulSnapshotId, long snapshotTag) {
+         super(cacheId, pageId);
+ 
+         this.lastSuccessfulSnapshotId = lastSuccessfulSnapshotId;
+         this.lastSuccessfulSnapshotTag = snapshotTag;
+     }
+ 
+     /** {@inheritDoc} */
 -    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
 -        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
++    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
++        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
+ 
 -        io.setLastSuccessfulSnapshotId(buf, lastSuccessfulSnapshotId);
++        io.setLastSuccessfulSnapshotId(pageAddr, lastSuccessfulSnapshotId);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public RecordType type() {
+         return RecordType.META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID;
+     }
+ 
+     /**
+      * @return lastSuccessfulSnapshotId
+      */
+     public long lastSuccessfulSnapshotId() {
+         return lastSuccessfulSnapshotId;
+     }
+ 
+     /**
+      * @return lastSuccessfulSnapshotTag
+      */
+     public long lastSuccessfulSnapshotTag() {
+         return lastSuccessfulSnapshotTag;
+     }
+ }
+ 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
index 0000000,0a92aef..aafda11
mode 000000,100644..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
@@@ -1,0 -1,59 +1,58 @@@
+ /*
+  * 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.wal.record.delta;
+ 
 -import java.nio.ByteBuffer;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
+ 
+ /**
+  *
+  */
+ public class MetaPageUpdateNextSnapshotId extends PageDeltaRecord {
+     /** */
+     private final long nextSnapshotId;
+ 
+     /**
+      * @param pageId Meta page ID.
+      */
+     public MetaPageUpdateNextSnapshotId(int cacheId, long pageId, long nextSnapshotId) {
+         super(cacheId, pageId);
+ 
+         this.nextSnapshotId = nextSnapshotId;
+     }
+ 
+     /** {@inheritDoc} */
 -    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
 -        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
++    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
++        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
+ 
 -        io.setNextSnapshotTag(buf, nextSnapshotId);
++        io.setNextSnapshotTag(pageAddr, nextSnapshotId);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public RecordType type() {
+         return RecordType.META_PAGE_UPDATE_NEXT_SNAPSHOT_ID;
+     }
+ 
+     /**
+      * @return Root ID.
+      */
+     public long nextSnapshotId() {
+         return nextSnapshotId;
+     }
+ }
+ 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
index 39d038b,ae6210a..ecc2c16
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
@@@ -78,14 -85,21 +84,21 @@@ public class MetaPageUpdatePartitionDat
      }
  
      /** {@inheritDoc} */
 -    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
 -        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(buf);
 +    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
 +        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
  
 -        io.setUpdateCounter(buf, updateCntr);
 -        io.setGlobalRemoveId(buf, globalRmvId);
 -        io.setSize(buf, partSize);
 +        io.setUpdateCounter(pageAddr, updateCntr);
 +        io.setGlobalRemoveId(pageAddr, globalRmvId);
 +        io.setSize(pageAddr, partSize);
      }
  
+     /**
+      *
+      */
+     public int allocatedIndexCandidate() {
+         return allocatedIdxCandidate;
+     }
+ 
      /** {@inheritDoc} */
      @Override public RecordType type() {
          return RecordType.PARTITION_META_PAGE_UPDATE_COUNTERS;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
index 0000000,23ebcee..2618328
mode 000000,100644..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
@@@ -1,0 -1,47 +1,46 @@@
+ /*
+  * 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.wal.record.delta;
+ 
 -import java.nio.ByteBuffer;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListMetaIO;
+ 
+ /**
+  * Delta record for page-list meta count reset
+  */
+ public class PageListMetaResetCountRecord extends PageDeltaRecord  {
+     /**
+      * @param cacheId Cache ID.
+      * @param pageId Page ID.
+      */
+     public PageListMetaResetCountRecord(int cacheId, long pageId) {
+         super(cacheId, pageId);
+     }
+ 
+     /** {@inheritDoc} */
 -    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
 -        PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
++    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
++        PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
+ 
 -        io.resetCount(buf);
++        io.resetCount(pageAddr);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public RecordType type() {
+         return RecordType.PAGE_LIST_META_RESET_COUNT_RECORD;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
index bc5082e,9d00d77..7daac81
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
@@@ -70,8 -70,8 +70,8 @@@ public class TrackingPageDeltaRecord ex
      }
  
      /** {@inheritDoc} */
 -    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
 -        TrackingPageIO.VERSIONS.forPage(buf).markChanged(buf, pageIdToMark, nextSnapshotId, lastSuccessfulSnapshotId, buf.capacity());
 +    @Override public void applyDelta(long pageAddr, int pageSize) throws IgniteCheckedException {
-         TrackingPageIO.VERSIONS.forPage(pageAddr).markChanged(pageAddr, pageIdToMark, nextBackupId, lastSuccessfulBackupId, pageSize);
++        TrackingPageIO.VERSIONS.forPage(pageAddr).markChanged(pageAddr, pageIdToMark, nextSnapshotId, lastSuccessfulSnapshotId, buf.capacity());
      }
  
      /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 85b9de5,4f299dd..bdf2ea1
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@@ -72,9 -72,10 +72,9 @@@ import org.apache.ignite.internal.binar
  import org.apache.ignite.internal.binary.BinaryMarshaller;
  import org.apache.ignite.internal.binary.GridBinaryMarshaller;
  import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
- import org.apache.ignite.internal.pagemem.backup.StartFullBackupAckDiscoveryMessage;
+ import org.apache.ignite.internal.pagemem.snapshot.StartFullSnapshotAckDiscoveryMessage;
  import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
  import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 -import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogNoopManager;
  import org.apache.ignite.internal.processors.GridProcessorAdapter;
  import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
  import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/4183186a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
index f33818a,4da44bc..e44838d
--- 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,17 -258,20 +258,20 @@@ public abstract class PagesList extend
  
          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);
                  }
              }
          }