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

ignite git commit: Copy page to checkpoint pool (tmp page) if checkpoint in progress, no need flush changes from checkpoint pages to main memory after checkpoint complete.

Repository: ignite
Updated Branches:
  refs/heads/ignite-5267 e31f37171 -> 7058c71f1


Copy page to checkpoint pool (tmp page) if checkpoint in progress, no need flush changes from checkpoint pages to main memory after checkpoint complete.


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

Branch: refs/heads/ignite-5267
Commit: 7058c71f10534d0f84bf9acf8b4690b5d4e45688
Parents: e31f371
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Tue May 23 11:02:01 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Tue May 23 11:02:01 2017 +0300

----------------------------------------------------------------------
 .../database/file/FilePageStoreManager.java     |  14 +-
 .../cache/database/pagemem/PageMemoryImpl.java  | 509 +++++++++----------
 .../PageStoreCheckpointSimulationSelfTest.java  |   2 +-
 3 files changed, 247 insertions(+), 278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7058c71f/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/file/FilePageStoreManager.java b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/file/FilePageStoreManager.java
index 1bb83d2..3a5873d 100755
--- a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/file/FilePageStoreManager.java
+++ b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/file/FilePageStoreManager.java
@@ -117,12 +117,22 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
 
         if (pstCfg.getPersistenceStorePath() != null) {
             File workDir0 = new File(pstCfg.getPersistenceStorePath());
+
             if (!workDir0.isAbsolute())
-                workDir0 = U.resolveWorkDirectory(igniteCfg.getWorkDirectory(), pstCfg.getPersistenceStorePath(), false);
+                workDir0 = U.resolveWorkDirectory(
+                    igniteCfg.getWorkDirectory(),
+                    pstCfg.getPersistenceStorePath(),
+                    false
+                );
+
             storeWorkDir = new File(workDir0, consId);
         }
         else
-            storeWorkDir = new File(U.resolveWorkDirectory(igniteCfg.getWorkDirectory(), "db", false), consId);
+            storeWorkDir = new File(U.resolveWorkDirectory(
+                igniteCfg.getWorkDirectory(),
+                "db",
+                false
+            ), consId);
 
         U.ensureDirectory(storeWorkDir, "page store work directory", log);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7058c71f/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/pagemem/PageMemoryImpl.java b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/pagemem/PageMemoryImpl.java
index 21ebcbf..c05af57 100755
--- a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/pagemem/PageMemoryImpl.java
+++ b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/pagemem/PageMemoryImpl.java
@@ -136,7 +136,7 @@ public class PageMemoryImpl implements PageMemoryEx {
     /** Page relative pointer. Does not change once a page is allocated. */
     public static final int RELATIVE_PTR_OFFSET = 8;
 
-    /** Page ID offset  */
+    /** Page ID offset */
     public static final int PAGE_ID_OFFSET = 16;
 
     /** Page cache ID offset. */
@@ -373,7 +373,7 @@ public class PageMemoryImpl implements PageMemoryEx {
 
     /** {@inheritDoc} */
     @Override public boolean isDirty(int cacheId, long pageId, long page) {
-        return isDirtyVisible(page, new FullPageId(pageId, cacheId));
+        return isDirty(page);
     }
 
     /** {@inheritDoc} */
@@ -417,7 +417,7 @@ public class PageMemoryImpl implements PageMemoryEx {
                 "Pin counter must be 0 for a new page [relPtr=" + U.hexLong(relPtr) +
                     ", absPtr=" + U.hexLong(absPtr) + ']';
 
-            setDirty(fullId, absPtr, true, true, false);
+            setDirty(fullId, absPtr, true, true);
 
             if (isTrackingPage) {
                 long pageAddr = absPtr + PAGE_OVERHEAD;
@@ -428,8 +428,14 @@ public class PageMemoryImpl implements PageMemoryEx {
                     trackingIO.initNewPage(pageAddr, pageId, pageSize());
 
                     if (!sharedCtx.wal().isAlwaysWriteFullPages())
-                        sharedCtx.wal().log(new InitNewPageRecord(cacheId, pageId, trackingIO.getType(),
-                            trackingIO.getVersion(), pageId));
+                        sharedCtx.wal().log(
+                            new InitNewPageRecord(
+                                cacheId,
+                                pageId,
+                                trackingIO.getType(),
+                                trackingIO.getVersion(), pageId
+                            )
+                        );
                     else
                         sharedCtx.wal().log(new PageSnapshot(fullId, absPtr + PAGE_OVERHEAD, pageSize()));
                 }
@@ -483,8 +489,13 @@ public class PageMemoryImpl implements PageMemoryEx {
         seg.readLock().lock();
 
         try {
-            long relPtr = seg.loadedPages.get(cacheId, PageIdUtils.effectivePageId(pageId), seg.partTag(cacheId, partId),
-                INVALID_REL_PTR, INVALID_REL_PTR);
+            long relPtr = seg.loadedPages.get(
+                cacheId,
+                PageIdUtils.effectivePageId(pageId),
+                seg.partTag(cacheId, partId),
+                INVALID_REL_PTR,
+                INVALID_REL_PTR
+            );
 
             // The page is loaded to the memory.
             if (relPtr != INVALID_REL_PTR) {
@@ -503,8 +514,14 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         try {
             // Double-check.
-            long relPtr = seg.loadedPages.get(cacheId, PageIdUtils.effectivePageId(pageId), seg.partTag(cacheId, partId),
-                INVALID_REL_PTR, OUTDATED_REL_PTR);
+            long relPtr = seg.loadedPages.get(
+                cacheId,
+                PageIdUtils.effectivePageId(pageId),
+                seg.partTag(cacheId, partId),
+                INVALID_REL_PTR,
+                OUTDATED_REL_PTR
+            );
+
             long absPtr;
 
             if (relPtr == INVALID_REL_PTR) {
@@ -523,9 +540,14 @@ public class PageMemoryImpl implements PageMemoryEx {
                         ", absPtr=" + U.hexLong(absPtr) + ']';
 
                 // We can clear dirty flag after the page has been allocated.
-                setDirty(fullId, absPtr, false, false, false);
+                setDirty(fullId, absPtr, false, false);
 
-                seg.loadedPages.put(cacheId, PageIdUtils.effectivePageId(pageId), relPtr, seg.partTag(cacheId, partId));
+                seg.loadedPages.put(
+                    cacheId,
+                    PageIdUtils.effectivePageId(pageId),
+                    relPtr,
+                    seg.partTag(cacheId, partId)
+                );
 
                 long pageAddr = absPtr + PAGE_OVERHEAD;
 
@@ -592,10 +614,10 @@ public class PageMemoryImpl implements PageMemoryEx {
      * @param seg Segment.
      * @param cacheId Cache ID.
      * @param pageId Page ID.
-     * @param remove {@code True} if page should be removed.
+     * @param rmv {@code True} if page should be removed.
      * @return Relative pointer to refreshed page.
      */
-    private long refreshOutdatedPage(Segment seg, int cacheId, long pageId, boolean remove) {
+    private long refreshOutdatedPage(Segment seg, int cacheId, long pageId, boolean rmv) {
         assert seg.writeLock().isHeldByCurrentThread();
 
         int tag = seg.partTag(cacheId, PageIdUtils.partId(pageId));
@@ -621,7 +643,7 @@ public class PageMemoryImpl implements PageMemoryEx {
             checkpointPool.releaseFreePage(tmpBufPtr);
         }
 
-        if (remove)
+        if (rmv)
             seg.loadedPages.remove(cacheId, PageIdUtils.effectivePageId(pageId), tag);
 
         return relPtr;
@@ -753,196 +775,164 @@ public class PageMemoryImpl implements PageMemoryEx {
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "TooBroadScope"})
     @Override public void finishCheckpoint() {
-        // Lock segment by segment and flush changes.
-        for (Segment seg : segments) {
-            GridLongList activePages = null;
+        for (Segment seg : segments)
+            seg.segCheckpointPages = null;
+    }
 
-            seg.writeLock().lock();
+    /** {@inheritDoc} */
+    @Override public Integer getForCheckpoint(FullPageId fullId, ByteBuffer tmpBuf) {
+        assert tmpBuf.remaining() == pageSize();
 
-            try {
-                assert seg.segCheckpointPages != null : "Checkpoint has not been started.";
+        Segment seg = segment(fullId.cacheId(), fullId.pageId());
 
-                for (FullPageId fullId : seg.segCheckpointPages) {
-                    int partTag = seg.partTag(fullId.cacheId(), PageIdUtils.partId(fullId.pageId()));
+        long absPtr = 0;
 
-                    long relPtr = seg.loadedPages.get(fullId.cacheId(),
-                        PageIdUtils.effectivePageId(fullId.pageId()), partTag, INVALID_REL_PTR, OUTDATED_REL_PTR);
+        long relPtr;
 
-                    // Checkpoint page may have been written by evict.
-                    if (relPtr == INVALID_REL_PTR)
-                        continue;
+        int tag;
 
-                    if (relPtr == OUTDATED_REL_PTR) {
-                        relPtr = refreshOutdatedPage(seg, fullId.cacheId(),
-                            PageIdUtils.effectivePageId(fullId.pageId()), true);
+        seg.readLock().lock();
 
-                        seg.pool.releaseFreePage(relPtr);
+        try {
+            tag = seg.partTag(fullId.cacheId(), PageIdUtils.partId(fullId.pageId()));
 
-                        continue;
-                    }
+            relPtr = seg.loadedPages.get(
+                fullId.cacheId(),
+                PageIdUtils.effectivePageId(fullId.pageId()),
+                tag,
+                INVALID_REL_PTR,
+                OUTDATED_REL_PTR
+            );
 
-                    long absPtr = seg.absolute(relPtr);
+            // Page may have been cleared during eviction. We have nothing to do in this case.
+            if (relPtr == INVALID_REL_PTR)
+                return null;
 
-                    boolean pinned = PageHeader.isAcquired(absPtr);
+            if (relPtr != OUTDATED_REL_PTR) {
+                absPtr = seg.absolute(relPtr);
 
-                    if (pinned) {
-                        // Pin the page one more time.
-                        seg.acquirePage(absPtr);
+                // Pin the page until page will not be copied.
+                if (PageHeader.tempBufferPointer(absPtr) == INVALID_REL_PTR)
+                    PageHeader.acquirePage(absPtr);
+            }
+        }
+        finally {
+            seg.readLock().unlock();
+        }
 
-                        if (activePages == null)
-                            activePages = new GridLongList(seg.segCheckpointPages.size() / 2 + 1);
+        if (relPtr == OUTDATED_REL_PTR) {
+            seg.writeLock().lock();
 
-                        activePages.add(relPtr);
-                    }
-                    // Page is not pinned and nobody can pin it since we hold the segment write lock.
-                    else {
-                        flushPageTempBuffer(fullId, absPtr);
+            try {
+                // Double-check.
+                relPtr = seg.loadedPages.get(
+                    fullId.cacheId(),
+                    PageIdUtils.effectivePageId(fullId.pageId()),
+                    seg.partTag(
+                        fullId.cacheId(),
+                        PageIdUtils.partId(fullId.pageId())
+                    ),
+                    INVALID_REL_PTR,
+                    OUTDATED_REL_PTR
+                );
 
-                        assert PageHeader.tempBufferPointer(absPtr) == INVALID_REL_PTR;
-                        assert !PageHeader.tempDirty(absPtr) : "ptr=" + U.hexLong(absPtr) + ", fullId=" + fullId;
-                    }
+                if (relPtr == INVALID_REL_PTR)
+                    return null;
+
+                if (relPtr == OUTDATED_REL_PTR) {
+                    relPtr = refreshOutdatedPage(
+                        seg,
+                        fullId.cacheId(),
+                        PageIdUtils.effectivePageId(fullId.pageId()),
+                        true
+                    );
+
+                    seg.pool.releaseFreePage(relPtr);
                 }
+
+                return null;
             }
             finally {
                 seg.writeLock().unlock();
             }
 
-            // Must release active pages outside of segment write lock.
-            if (activePages != null) {
-                for (int p = 0; p < activePages.size(); p++) {
-                    long relPtr = activePages.get(p);
-
-                    long absPtr = seg.absolute(relPtr);
-
-                    flushCheckpoint(absPtr);
-
-                    seg.readLock().lock();
-
-                    try {
-                        seg.releasePage(absPtr);
-                    }
-                    finally {
-                        seg.readLock().unlock();
-                    }
-                }
-            }
+        }
+        else {
+            copyPageForCheckpoint(absPtr, fullId, tmpBuf);
 
-            seg.segCheckpointPages = null;
+            return tag;
         }
     }
 
     /**
-     * Checks if the page represented by the given full ID and absolute pointer has a temp buffer. If it has, this
-     * method will flush temp buffer data to the main page buffer as well as temp buffer dirty flag, release the
-     * temp buffer to segment pool and clear full page ID from checkpoint set.
-     * <p>
-     * This method must be called wither from segment write lock while page is not pinned (thus, no other thread has
-     * access to the page's write buffer, or when this page is pinned and locked for write.
-     *
-     * @param fullId Full page ID.
-     * @param absPtr Page absolute pointer.
+     * @param absPtr Absolute ptr.
+     * @param fullId Full id.
+     * @param tmpBuf Tmp buffer.
      */
-    private void flushPageTempBuffer(FullPageId fullId, long absPtr) {
-        long tmpRelPtr = PageHeader.tempBufferPointer(absPtr);
-
-        // The page has temp buffer, need to flush it to the main memory.
-        if (tmpRelPtr != INVALID_REL_PTR) {
-            long tmpAbsPtr = checkpointPool.absolute(tmpRelPtr);
-
-            boolean tmpDirty = PageHeader.tempDirty(absPtr, false);
-
-            // Page could have a temp write buffer, but be not dirty because
-            // it was not modified after getForWrite.
-            if (tmpDirty)
-                GridUnsafe.copyMemory(tmpAbsPtr + PAGE_OVERHEAD, absPtr + PAGE_OVERHEAD,
-                    sysPageSize - PAGE_OVERHEAD);
-
-            setDirty(fullId, absPtr, tmpDirty, true, false);
-
-            PageHeader.tempBufferPointer(absPtr, INVALID_REL_PTR);
+    private void copyPageForCheckpoint(long absPtr, FullPageId fullId, ByteBuffer tmpBuf) {
+        assert absPtr != 0;
 
-            checkpointPool.releaseFreePage(tmpRelPtr);
+        long tmpRelPtr;
 
-            // We pinned the page when allocated the temp buffer, release it now.
-            int updated = PageHeader.releasePage(absPtr);
+        rwLock.writeLock(absPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS);
 
-            assert updated > 0 : "Checkpoint page should not be released by flushCheckpoint()";
-        }
-        else {
-            // We can get here in two cases.
-            // 1) Page was not modified since the checkpoint started.
-            // 2) Page was dirty and was written to the store by evictPage(). Then it was loaded to memory again
-            //    and may have already modified by a writer.
-            // In both cases we should just set page header dirty flag based on dirtyPages collection.
-            PageHeader.dirty(absPtr, segment(fullId.cacheId(), fullId.pageId()).dirtyPages.contains(fullId));
-        }
+        try {
+            tmpRelPtr = PageHeader.tempBufferPointer(absPtr);
 
-        // It is important to clear checkpoint status before the write lock is released.
-        clearCheckpoint(fullId);
-    }
+            clearCheckpoint(fullId);
 
-    /**
-     * If page was concurrently modified during the checkpoint phase, this method will flush all changes from the
-     * temporary location to main memory.
-     * This method must be called outside of the segment write lock because we can ask for another pages
-     *      while holding a page read or write lock.
-     */
-    private void flushCheckpoint(long absPtr) {
-        rwLock.writeLock(absPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS);
+            if (tmpRelPtr != INVALID_REL_PTR)
+                PageHeader.tempBufferPointer(absPtr, INVALID_REL_PTR);
+            else {
+                copyInBuffer(absPtr, tmpBuf);
 
-        try {
-            assert PageHeader.isAcquired(absPtr);
+                PageHeader.dirty(absPtr, false);
 
-            FullPageId fullId = PageHeader.fullPageId(absPtr);
+                PageHeader.releasePage(absPtr);
 
-            flushPageTempBuffer(fullId, absPtr);
+                return;
+            }
         }
         finally {
             rwLock.writeUnlock(absPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS);
         }
-    }
 
-    /** {@inheritDoc} */
-    @Override public Integer getForCheckpoint(FullPageId fullId, ByteBuffer tmpBuf) {
-        assert tmpBuf.remaining() == pageSize();
-
-        Segment seg = segment(fullId.cacheId(), fullId.pageId());
+        assert tmpRelPtr != 0;
 
-        seg.readLock().lock();
+        long tmpAbsPtr = checkpointPool.absolute(tmpRelPtr);
 
-        try {
-            int tag = seg.partTag(fullId.cacheId(), PageIdUtils.partId(fullId.pageId()));
+        copyInBuffer(tmpAbsPtr, tmpBuf);
 
-            long relPtr = seg.loadedPages.get(fullId.cacheId(),
-                PageIdUtils.effectivePageId(fullId.pageId()), tag, INVALID_REL_PTR, INVALID_REL_PTR);
+        GridUnsafe.setMemory(tmpAbsPtr + PAGE_OVERHEAD, pageSize(), (byte)0);
 
-            // Page may have been cleared during eviction. We have nothing to do in this case.
-            if (relPtr == INVALID_REL_PTR)
-                return null;
+        PageHeader.dirty(tmpAbsPtr, false);
 
-            long absPtr = seg.absolute(relPtr);
+        checkpointPool.releaseFreePage(tmpRelPtr);
 
-            if (tmpBuf.isDirect()) {
-                long tmpPtr = ((DirectBuffer)tmpBuf).address();
+        // We pinned the page when allocated the temp buffer, release it now.
+        PageHeader.releasePage(absPtr);
+    }
 
-                GridUnsafe.copyMemory(absPtr + PAGE_OVERHEAD, tmpPtr, pageSize());
+    /**
+     * @param absPtr Absolute ptr.
+     * @param tmpBuf Tmp buffer.
+     */
+    private void copyInBuffer(long absPtr, ByteBuffer tmpBuf) {
+        if (tmpBuf.isDirect()) {
+            long tmpPtr = ((DirectBuffer)tmpBuf).address();
 
-                assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-                assert GridUnsafe.getInt(tmpPtr + 4) == 0; //TODO GG-11480
-            }
-            else {
-                byte[] arr = tmpBuf.array();
+            GridUnsafe.copyMemory(absPtr + PAGE_OVERHEAD, tmpPtr, pageSize());
 
-                assert arr != null;
-                assert arr.length == pageSize();
+            assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+            assert GridUnsafe.getInt(tmpPtr + 4) == 0; //TODO GG-11480
+        }
+        else {
+            byte[] arr = tmpBuf.array();
 
-                GridUnsafe.copyMemory(null, absPtr + PAGE_OVERHEAD, arr, GridUnsafe.BYTE_ARR_OFF, pageSize());
-            }
+            assert arr != null;
+            assert arr.length == pageSize();
 
-            return tag;
-        }
-        finally {
-            seg.readLock().unlock();
+            GridUnsafe.copyMemory(null, absPtr + PAGE_OVERHEAD, arr, GridUnsafe.BYTE_ARR_OFF, pageSize());
         }
     }
 
@@ -985,13 +975,13 @@ public class PageMemoryImpl implements PageMemoryEx {
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Void> clearAsync(
-        GridPredicate3<Integer, Long, Integer> predicate,
+        GridPredicate3<Integer, Long, Integer> pred,
         boolean cleanDirty
     ) {
         CountDownFuture completeFut = new CountDownFuture(segments.length);
 
         for (Segment seg : segments) {
-            Runnable clear = new ClearSegmentRunnable(seg, predicate, cleanDirty, completeFut, pageSize());
+            Runnable clear = new ClearSegmentRunnable(seg, pred, cleanDirty, completeFut, pageSize());
 
             try {
                 asyncRunner.execute(clear);
@@ -1057,20 +1047,9 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         PageHeader.writeTimestamp(absPtr, U.currentTimeMillis());
 
-        long tmpRelPtr = PageHeader.tempBufferPointer(absPtr);
+        assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
 
-        if (tmpRelPtr == INVALID_REL_PTR) {
-            assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-
-            return absPtr + PAGE_OVERHEAD;
-        }
-        else {
-            long tmpAbsPtr = checkpointPool.absolute(tmpRelPtr);
-
-            assert GridUnsafe.getInt(tmpAbsPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-
-            return tmpAbsPtr + PAGE_OVERHEAD;
-        }
+        return absPtr + PAGE_OVERHEAD;
     }
 
     /** {@inheritDoc} */
@@ -1105,7 +1084,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         if (!rwLock.tryWriteLock(absPtr + PAGE_LOCK_OFFSET, tag))
             return 0;
 
-        return doWriteLockPage(absPtr, fullId);
+        return postWriteLockPage(absPtr, fullId);
     }
 
     /**
@@ -1117,99 +1096,82 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         boolean locked = rwLock.writeLock(absPtr + PAGE_LOCK_OFFSET, tag);
 
-        return locked ? doWriteLockPage(absPtr, fullId) : 0;
+        return locked ? postWriteLockPage(absPtr, fullId) : 0;
     }
 
     /**
      * @param absPtr Absolute pointer.
      * @return Pointer to the page write buffer.
      */
-    private long doWriteLockPage(long absPtr, FullPageId fullId) {
+    private long postWriteLockPage(long absPtr, FullPageId fullId) {
         PageHeader.writeTimestamp(absPtr, U.currentTimeMillis());
 
         // Create a buffer copy if the page is scheduled for a checkpoint.
-        if (isInCheckpoint(fullId)) {
-            long tmpRelPtr = PageHeader.tempBufferPointer(absPtr);
+        if (isInCheckpoint(fullId) && PageHeader.tempBufferPointer(absPtr) == INVALID_REL_PTR) {
+            long tmpRelPtr = checkpointPool.borrowOrAllocateFreePage(fullId.pageId());
 
             if (tmpRelPtr == INVALID_REL_PTR) {
-                tmpRelPtr = checkpointPool.borrowOrAllocateFreePage(fullId.pageId());
+                rwLock.writeUnlock(absPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS);
 
-                if (tmpRelPtr == INVALID_REL_PTR) {
-                    rwLock.writeUnlock(absPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS);
-
-                    throw new IgniteException("Failed to allocate temporary buffer for checkpoint " +
+                throw new IgniteException(
+                    "Failed to allocate temporary buffer for checkpoint " +
                         "(increase checkpointPageBufferSize configuration property)");
-                }
-
-                // Pin the page until checkpoint is not finished.
-                PageHeader.acquirePage(absPtr);
+            }
 
-                long tmpAbsPtr = checkpointPool.absolute(tmpRelPtr);
+            // Pin the page until checkpoint is not finished.
+            PageHeader.acquirePage(absPtr);
 
-                GridUnsafe.copyMemory(null, absPtr + PAGE_OVERHEAD, null, tmpAbsPtr + PAGE_OVERHEAD, pageSize());
+            long tmpAbsPtr = checkpointPool.absolute(tmpRelPtr);
 
-                PageHeader.tempDirty(absPtr, false);
-                PageHeader.tempBufferPointer(absPtr, tmpRelPtr);
+            assert !PageHeader.tempDirty(tmpAbsPtr);
 
-                assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-                assert GridUnsafe.getInt(tmpAbsPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+            GridUnsafe.copyMemory(
+                null,
+                absPtr + PAGE_OVERHEAD,
+                null,
+                tmpAbsPtr + PAGE_OVERHEAD,
+                pageSize()
+            );
 
-                return tmpAbsPtr + PAGE_OVERHEAD;
-            }
-            else {
-                long newAbsPrt = checkpointPool.absolute(tmpRelPtr) + PAGE_OVERHEAD;
-
-                assert GridUnsafe.getInt(newAbsPrt + 4) == 0; //TODO GG-11480
+            PageHeader.dirty(absPtr, false);
+            PageHeader.tempBufferPointer(absPtr, tmpRelPtr);
 
-                return newAbsPrt;
-            }
-        }
-        else {
             assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-
-            return absPtr + PAGE_OVERHEAD;
+            assert GridUnsafe.getInt(tmpAbsPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
         }
+
+        assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+
+        return absPtr + PAGE_OVERHEAD;
     }
 
     /**
      * @param page Page pointer.
-     * @param walPolicy Full page WAL record policy.
+     * @param walPlc Full page WAL record policy.
      */
-    private void writeUnlockPage(long page, FullPageId fullId, Boolean walPolicy, boolean markDirty, boolean restore) {
-        boolean dirtyVisible = isDirtyVisible(page, fullId);
+    private void writeUnlockPage(
+        long page,
+        FullPageId fullId,
+        Boolean walPlc,
+        boolean markDirty,
+        boolean restore
+    ) {
+        boolean dirty = isDirty(page);
 
         //if page is for restore, we shouldn't mark it as changed
-        if (!restore && markDirty && !dirtyVisible) {
+        if (!restore && markDirty && !dirty)
             changeTracker.apply(page, fullId, this);
-        }
-
-        boolean pageWalRec = markDirty && walPolicy != FALSE && (walPolicy == TRUE || !dirtyVisible);
-        long pageId;
-
-        long tmpRel = PageHeader.tempBufferPointer(page);
 
-        if (tmpRel != INVALID_REL_PTR) {
-            long tmpAbsPtr = checkpointPool.absolute(tmpRel);
+        boolean pageWalRec = markDirty && walPlc != FALSE && (walPlc == TRUE || !dirty);
 
-            assert GridUnsafe.getInt(tmpAbsPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-
-            if (markDirty)
-                setDirty(fullId, page, markDirty, false, true);
-
-            beforeReleaseWrite(fullId, tmpAbsPtr + PAGE_OVERHEAD, pageWalRec);
-
-            pageId = PageIO.getPageId(tmpAbsPtr + PAGE_OVERHEAD);
-        }
-        else {
-            assert GridUnsafe.getInt(page + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+        assert GridUnsafe.getInt(page + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
 
-            if (markDirty)
-                setDirty(fullId, page, markDirty, false, false);
+        if (markDirty)
+            setDirty(fullId, page, markDirty, false);
 
-            beforeReleaseWrite(fullId, page + PAGE_OVERHEAD, pageWalRec);
+        beforeReleaseWrite(fullId, page + PAGE_OVERHEAD, pageWalRec);
 
-            pageId = PageIO.getPageId(page + PAGE_OVERHEAD);
-        }
+        long pageId = PageIO.getPageId(page + PAGE_OVERHEAD);
 
         rwLock.writeUnlock(page + PAGE_LOCK_OFFSET, PageIdUtils.tag(pageId));
     }
@@ -1277,31 +1239,6 @@ public class PageMemoryImpl implements PageMemoryEx {
     }
 
     /**
-     * @param absPtr Absolute pointer.
-     * @return {@code True} if page is dirty in temporary buffer.
-     */
-    boolean isTempDirty(long absPtr) {
-        return PageHeader.tempDirty(absPtr);
-    }
-
-    /**
-     * @param absPtr Absolute page pointer.
-     * @param fullId Full page ID.
-     * @return If page is visible to memory user as dirty.
-     */
-    boolean isDirtyVisible(long absPtr, FullPageId fullId) {
-        Collection<FullPageId> cp = segment(fullId.cacheId(), fullId.pageId()).segCheckpointPages;
-
-        if (cp == null || !cp.contains(fullId))
-            return isDirty(absPtr);
-        else {
-            long tmpPtr = PageHeader.tempBufferPointer(absPtr);
-
-            return tmpPtr != INVALID_REL_PTR && PageHeader.tempDirty(absPtr);
-        }
-    }
-
-    /**
      * Gets the number of active pages across all segments. Used for test purposes only.
      *
      * @return Number of active pages.
@@ -1320,11 +1257,11 @@ public class PageMemoryImpl implements PageMemoryEx {
      *
      * @param absPtr Absolute pointer.
      * @param dirty {@code True} dirty flag.
-     * @param forceAdd If this flag is {@code true}, then the page will be added to the dirty set regardless whether
-     *      the old flag was dirty or not.
+     * @param forceAdd If this flag is {@code true}, then the page will be added to the dirty set regardless whether the
+     * old flag was dirty or not.
      */
-    void setDirty(FullPageId pageId, long absPtr, boolean dirty, boolean forceAdd, boolean tmp) {
-        boolean wasDirty = tmp ? PageHeader.tempDirty(absPtr, dirty) : PageHeader.dirty(absPtr, dirty);
+    void setDirty(FullPageId pageId, long absPtr, boolean dirty, boolean forceAdd) {
+        boolean wasDirty = PageHeader.dirty(absPtr, dirty);
 
         if (dirty) {
             if (!wasDirty || forceAdd)
@@ -1420,9 +1357,9 @@ public class PageMemoryImpl implements PageMemoryEx {
         /**
          * Allocates a new free page.
          *
+         * @param pageId Page ID to to initialize.
          * @return Relative pointer to the allocated page.
          * @throws GridOffHeapOutOfMemoryException If failed to allocate new free page.
-         * @param pageId Page ID to to initialize.
          */
         private long borrowOrAllocateFreePage(long pageId) throws GridOffHeapOutOfMemoryException {
             long relPtr = borrowFreePage();
@@ -1458,9 +1395,9 @@ public class PageMemoryImpl implements PageMemoryEx {
         }
 
         /**
+         * @param pageId Page ID.
          * @return Relative pointer of the allocated page.
          * @throws GridOffHeapOutOfMemoryException If failed to allocate new free page.
-         * @param pageId Page ID.
          */
         private long allocateFreePage(long pageId) throws GridOffHeapOutOfMemoryException {
             long limit = region.address() + region.size();
@@ -1574,7 +1511,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         private final int maxDirtyPages;
 
         /** */
-        private final Map<T2<Integer, Integer>, Integer> partitionTagMap = new HashMap<>();
+        private final Map<T2<Integer, Integer>, Integer> partTagMap = new HashMap<>();
 
         /**
          * @param region Memory region.
@@ -1646,8 +1583,8 @@ public class PageMemoryImpl implements PageMemoryEx {
         }
 
         /**
-         * @return Page relative pointer.
          * @param pageId Page ID.
+         * @return Page relative pointer.
          */
         private long borrowOrAllocateFreePage(long pageId) {
             return pool.borrowOrAllocateFreePage(pageId);
@@ -1678,10 +1615,16 @@ public class PageMemoryImpl implements PageMemoryEx {
                 if (cpPages != null && cpPages.contains(fullPageId)) {
                     assert storeMgr != null;
 
-                    flushDirtyPage.applyx(fullPageId, wrapPointer(absPtr + PAGE_OVERHEAD, pageSize()),
-                        partTag(fullPageId.cacheId(), PageIdUtils.partId(fullPageId.pageId())));
+                    flushDirtyPage.applyx(
+                        fullPageId,
+                        wrapPointer(absPtr + PAGE_OVERHEAD, pageSize()),
+                        partTag(
+                            fullPageId.cacheId(),
+                            PageIdUtils.partId(fullPageId.pageId())
+                        )
+                    );
 
-                    setDirty(fullPageId, absPtr, false, true, false);
+                    setDirty(fullPageId, absPtr, false, true);
 
                     cpPages.remove(fullPageId);
 
@@ -1810,8 +1753,14 @@ public class PageMemoryImpl implements PageMemoryEx {
                     continue;
                 }
 
-                loadedPages.remove(fullPageId.cacheId(), PageIdUtils.effectivePageId(fullPageId.pageId()),
-                    partTag(fullPageId.cacheId(), PageIdUtils.partId(fullPageId.pageId())));
+                loadedPages.remove(
+                    fullPageId.cacheId(),
+                    PageIdUtils.effectivePageId(fullPageId.pageId()),
+                    partTag(
+                        fullPageId.cacheId(),
+                        PageIdUtils.partId(fullPageId.pageId())
+                    )
+                );
 
                 return relEvictAddr;
             }
@@ -1819,6 +1768,7 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         /**
          * Will scan all segment pages to find one to evict it
+         *
          * @param cap Capacity.
          */
         private long tryToFindSequentially(int cap) throws IgniteCheckedException {
@@ -1857,8 +1807,14 @@ public class PageMemoryImpl implements PageMemoryEx {
                 final FullPageId fullPageId = PageHeader.fullPageId(absEvictAddr);
 
                 if (prepareEvict(fullPageId, absEvictAddr)) {
-                    loadedPages.remove(fullPageId.cacheId(), PageIdUtils.effectivePageId(fullPageId.pageId()),
-                        partTag(fullPageId.cacheId(), PageIdUtils.partId(fullPageId.pageId())));
+                    loadedPages.remove(
+                        fullPageId.cacheId(),
+                        PageIdUtils.effectivePageId(fullPageId.pageId()),
+                        partTag(
+                            fullPageId.cacheId(),
+                            PageIdUtils.partId(fullPageId.pageId())
+                        )
+                    );
 
                     return addr;
                 }
@@ -1896,7 +1852,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         private int partTag(int cacheId, int partId) {
             assert getReadHoldCount() > 0 || getWriteHoldCount() > 0;
 
-            Integer tag = partitionTagMap.get(new T2<>(cacheId, partId));
+            Integer tag = partTagMap.get(new T2<>(cacheId, partId));
 
             if (tag == null)
                 return 1;
@@ -1914,31 +1870,34 @@ public class PageMemoryImpl implements PageMemoryEx {
 
             T2<Integer, Integer> t = new T2<>(cacheId, partId);
 
-            Integer tag = partitionTagMap.get(t);
+            Integer tag = partTagMap.get(t);
 
             if (tag == null) {
-                partitionTagMap.put(t, 2);
+                partTagMap.put(t, 2);
 
                 return 2;
             }
             else if (tag == Integer.MAX_VALUE) {
                 U.warn(log, "Partition tag overflow [cacheId=" + cacheId + ", partId=" + partId + "]");
 
-                partitionTagMap.put(t, 0);
+                partTagMap.put(t, 0);
 
                 return 0;
             }
             else {
-                partitionTagMap.put(t, tag + 1);
+                partTagMap.put(t, tag + 1);
 
                 return tag + 1;
             }
         }
 
+        /**
+         * @param cacheId Cache id.
+         */
         private void resetPartTags(int cacheId) {
             assert getWriteHoldCount() > 0;
 
-            Iterator<T2<Integer, Integer>> iter = partitionTagMap.keySet().iterator();
+            Iterator<T2<Integer, Integer>> iter = partTagMap.keySet().iterator();
 
             while (iter.hasNext()) {
                 T2<Integer, Integer> t = iter.next();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7058c71f/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreCheckpointSimulationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreCheckpointSimulationSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreCheckpointSimulationSelfTest.java
index 7cc4122..96178ef 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreCheckpointSimulationSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreCheckpointSimulationSelfTest.java
@@ -584,7 +584,7 @@ public class PageStoreCheckpointSimulationSelfTest extends GridCommonAbstractTes
                 long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
 
                 try {
-                    assertFalse(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                    assertTrue(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
 
                     long pageAddr = mem.writeLock(fullId.cacheId(), fullId.pageId(), page);