aboutsummaryrefslogtreecommitdiff
path: root/src/backend/storage
diff options
context:
space:
mode:
authorSimon Riggs <simon@2ndQuadrant.com>2013-03-22 13:54:07 +0000
committerSimon Riggs <simon@2ndQuadrant.com>2013-03-22 13:54:07 +0000
commit96ef3b8ff1cf1950e897fd2f766d4bd9ef0d5d56 (patch)
tree65849014627f4e211c6be8a4e9905b67694ed4ae /src/backend/storage
parente4a05c7512b23c8f48c186e685f2ef186374a20a (diff)
downloadpostgresql-96ef3b8ff1cf1950e897fd2f766d4bd9ef0d5d56.tar.gz
postgresql-96ef3b8ff1cf1950e897fd2f766d4bd9ef0d5d56.zip
Allow I/O reliability checks using 16-bit checksums
Checksums are set immediately prior to flush out of shared buffers and checked when pages are read in again. Hint bit setting will require full page write when block is dirtied, which causes various infrastructure changes. Extensive comments, docs and README. WARNING message thrown if checksum fails on non-all zeroes page; ERROR thrown but can be disabled with ignore_checksum_failure = on. Feature enabled by an initdb option, since transition from option off to option on is long and complex and has not yet been implemented. Default is not to use checksums. Checksum used is WAL CRC-32 truncated to 16-bits. Simon Riggs, Jeff Davis, Greg Smith Wide input and assistance from many community members. Thank you.
Diffstat (limited to 'src/backend/storage')
-rw-r--r--src/backend/storage/buffer/bufmgr.c214
-rw-r--r--src/backend/storage/buffer/localbuf.c7
-rw-r--r--src/backend/storage/freespace/README4
-rw-r--r--src/backend/storage/freespace/freespace.c10
-rw-r--r--src/backend/storage/freespace/fsmpage.c2
-rw-r--r--src/backend/storage/page/bufpage.c183
6 files changed, 357 insertions, 63 deletions
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 0b4c2ed0a01..1cd0ac806a7 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -34,6 +34,7 @@
#include <unistd.h>
#include "catalog/catalog.h"
+#include "catalog/storage.h"
#include "common/relpath.h"
#include "executor/instrument.h"
#include "miscadmin.h"
@@ -431,6 +432,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
{
/* new buffers are zero-filled */
MemSet((char *) bufBlock, 0, BLCKSZ);
+ /* don't set checksum for all-zero page */
smgrextend(smgr, forkNum, blockNum, (char *) bufBlock, false);
}
else
@@ -460,13 +462,13 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
}
/* check for garbage data */
- if (!PageHeaderIsValid((PageHeader) bufBlock))
+ if (!PageIsVerified((Page) bufBlock, blockNum))
{
if (mode == RBM_ZERO_ON_ERROR || zero_damaged_pages)
{
ereport(WARNING,
(errcode(ERRCODE_DATA_CORRUPTED),
- errmsg("invalid page header in block %u of relation %s; zeroing out page",
+ errmsg("invalid page in block %u of relation %s; zeroing out page",
blockNum,
relpath(smgr->smgr_rnode, forkNum))));
MemSet((char *) bufBlock, 0, BLCKSZ);
@@ -474,7 +476,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
else
ereport(ERROR,
(errcode(ERRCODE_DATA_CORRUPTED),
- errmsg("invalid page header in block %u of relation %s",
+ errmsg("invalid page in block %u of relation %s",
blockNum,
relpath(smgr->smgr_rnode, forkNum))));
}
@@ -655,14 +657,23 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
* victim. We need lock to inspect the page LSN, so this
* can't be done inside StrategyGetBuffer.
*/
- if (strategy != NULL &&
- XLogNeedsFlush(BufferGetLSN(buf)) &&
- StrategyRejectBuffer(strategy, buf))
+ if (strategy != NULL)
{
- /* Drop lock/pin and loop around for another buffer */
- LWLockRelease(buf->content_lock);
- UnpinBuffer(buf, true);
- continue;
+ XLogRecPtr lsn;
+
+ /* Read the LSN while holding buffer header lock */
+ LockBufHdr(buf);
+ lsn = BufferGetLSN(buf);
+ UnlockBufHdr(buf);
+
+ if (XLogNeedsFlush(lsn) &&
+ StrategyRejectBuffer(strategy, buf))
+ {
+ /* Drop lock/pin and loop around for another buffer */
+ LWLockRelease(buf->content_lock);
+ UnpinBuffer(buf, true);
+ continue;
+ }
}
/* OK, do the I/O */
@@ -1906,6 +1917,8 @@ FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln)
ErrorContextCallback errcallback;
instr_time io_start,
io_time;
+ Block bufBlock;
+ char *bufToWrite;
/*
* Acquire the buffer's io_in_progress lock. If StartBufferIO returns
@@ -1931,6 +1944,18 @@ FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln)
reln->smgr_rnode.node.dbNode,
reln->smgr_rnode.node.relNode);
+ LockBufHdr(buf);
+
+ /*
+ * Run PageGetLSN while holding header lock, since we don't have the
+ * buffer locked exclusively in all cases.
+ */
+ recptr = BufferGetLSN(buf);
+
+ /* To check if block content changes while flushing. - vadim 01/17/97 */
+ buf->flags &= ~BM_JUST_DIRTIED;
+ UnlockBufHdr(buf);
+
/*
* Force XLOG flush up to buffer's LSN. This implements the basic WAL
* rule that log updates must hit disk before any of the data-file changes
@@ -1949,10 +1974,7 @@ FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln)
* buffer isn't permanent.
*/
if (buf->flags & BM_PERMANENT)
- {
- recptr = BufferGetLSN(buf);
XLogFlush(recptr);
- }
/*
* Now it's safe to write buffer to disk. Note that no one else should
@@ -1960,18 +1982,20 @@ FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln)
* we have the io_in_progress lock.
*/
- /* To check if block content changes while flushing. - vadim 01/17/97 */
- LockBufHdr(buf);
- buf->flags &= ~BM_JUST_DIRTIED;
- UnlockBufHdr(buf);
+ bufBlock = BufHdrGetBlock(buf);
+
+ bufToWrite = PageSetChecksumCopy((Page) bufBlock, buf->tag.blockNum);
if (track_io_timing)
INSTR_TIME_SET_CURRENT(io_start);
+ /*
+ * bufToWrite is either the shared buffer or a copy, as appropriate.
+ */
smgrwrite(reln,
buf->tag.forkNum,
buf->tag.blockNum,
- (char *) BufHdrGetBlock(buf),
+ bufToWrite,
false);
if (track_io_timing)
@@ -2042,6 +2066,34 @@ BufferIsPermanent(Buffer buffer)
return (bufHdr->flags & BM_PERMANENT) != 0;
}
+/*
+ * BufferGetLSNAtomic
+ * Retrieves the LSN of the buffer atomically using a buffer header lock.
+ * This is necessary for some callers who may not have an exclusive lock
+ * on the buffer.
+ */
+XLogRecPtr
+BufferGetLSNAtomic(Buffer buffer)
+{
+ volatile BufferDesc *bufHdr = &BufferDescriptors[buffer - 1];
+ char *page = BufferGetPage(buffer);
+ XLogRecPtr lsn;
+
+ /* Local buffers don't need a lock. */
+ if (BufferIsLocal(buffer))
+ return PageGetLSN(page);
+
+ /* Make sure we've got a real buffer, and that we hold a pin on it. */
+ Assert(BufferIsValid(buffer));
+ Assert(BufferIsPinned(buffer));
+
+ LockBufHdr(bufHdr);
+ lsn = PageGetLSN(page);
+ UnlockBufHdr(bufHdr);
+
+ return lsn;
+}
+
/* ---------------------------------------------------------------------
* DropRelFileNodeBuffers
*
@@ -2343,7 +2395,10 @@ FlushRelationBuffers(Relation rel)
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node) &&
(bufHdr->flags & BM_VALID) && (bufHdr->flags & BM_DIRTY))
{
- ErrorContextCallback errcallback;
+ ErrorContextCallback errcallback;
+ Page localpage;
+
+ localpage = (char *) LocalBufHdrGetBlock(bufHdr);
/* Setup error traceback support for ereport() */
errcallback.callback = local_buffer_write_error_callback;
@@ -2351,10 +2406,12 @@ FlushRelationBuffers(Relation rel)
errcallback.previous = error_context_stack;
error_context_stack = &errcallback;
+ PageSetChecksumInplace(localpage, bufHdr->tag.blockNum);
+
smgrwrite(rel->rd_smgr,
bufHdr->tag.forkNum,
bufHdr->tag.blockNum,
- (char *) LocalBufHdrGetBlock(bufHdr),
+ localpage,
false);
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
@@ -2509,22 +2566,24 @@ IncrBufferRefCount(Buffer buffer)
}
/*
- * SetBufferCommitInfoNeedsSave
+ * MarkBufferDirtyHint
*
- * Mark a buffer dirty when we have updated tuple commit-status bits in it.
+ * Mark a buffer dirty for non-critical changes.
*
- * This is essentially the same as MarkBufferDirty, except that the caller
- * might have only share-lock instead of exclusive-lock on the buffer's
- * content lock. We preserve the distinction mainly as a way of documenting
- * that the caller has not made a critical data change --- the status-bit
- * update could be redone by someone else just as easily. Therefore, no WAL
- * log record need be generated, whereas calls to MarkBufferDirty really ought
- * to be associated with a WAL-entry-creating action.
+ * This is essentially the same as MarkBufferDirty, except:
+ *
+ * 1. The caller does not write WAL; so if checksums are enabled, we may need
+ * to write an XLOG_HINT WAL record to protect against torn pages.
+ * 2. The caller might have only share-lock instead of exclusive-lock on the
+ * buffer's content lock.
+ * 3. This function does not guarantee that the buffer is always marked dirty
+ * (due to a race condition), so it cannot be used for important changes.
*/
void
-SetBufferCommitInfoNeedsSave(Buffer buffer)
+MarkBufferDirtyHint(Buffer buffer)
{
volatile BufferDesc *bufHdr;
+ Page page = BufferGetPage(buffer);
if (!BufferIsValid(buffer))
elog(ERROR, "bad buffer ID: %d", buffer);
@@ -2544,28 +2603,105 @@ SetBufferCommitInfoNeedsSave(Buffer buffer)
/*
* This routine might get called many times on the same page, if we are
* making the first scan after commit of an xact that added/deleted many
- * tuples. So, be as quick as we can if the buffer is already dirty. We
- * do this by not acquiring spinlock if it looks like the status bits are
- * already. Since we make this test unlocked, there's a chance we might
- * fail to notice that the flags have just been cleared, and failed to
- * reset them, due to memory-ordering issues. But since this function is
- * only intended to be used in cases where failing to write out the data
+ * tuples. So, be as quick as we can if the buffer is already dirty. We do
+ * this by not acquiring spinlock if it looks like the status bits are
+ * already set. Since we make this test unlocked, there's a chance we
+ * might fail to notice that the flags have just been cleared, and failed
+ * to reset them, due to memory-ordering issues. But since this function
+ * is only intended to be used in cases where failing to write out the data
* would be harmless anyway, it doesn't really matter.
*/
if ((bufHdr->flags & (BM_DIRTY | BM_JUST_DIRTIED)) !=
(BM_DIRTY | BM_JUST_DIRTIED))
{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ bool dirtied = false;
+ bool delayChkpt = false;
+
+ /*
+ * If checksums are enabled, and the buffer is permanent, then a full
+ * page image may be required even for some hint bit updates to protect
+ * against torn pages. This full page image is only necessary if the
+ * hint bit update is the first change to the page since the last
+ * checkpoint.
+ *
+ * We don't check full_page_writes here because that logic is
+ * included when we call XLogInsert() since the value changes
+ * dynamically.
+ */
+ if (DataChecksumsEnabled() && (bufHdr->flags & BM_PERMANENT))
+ {
+ /*
+ * If we're in recovery we cannot dirty a page because of a hint.
+ * We can set the hint, just not dirty the page as a result so
+ * the hint is lost when we evict the page or shutdown.
+ *
+ * See src/backend/storage/page/README for longer discussion.
+ */
+ if (RecoveryInProgress())
+ return;
+
+ /*
+ * If the block is already dirty because we either made a change
+ * or set a hint already, then we don't need to write a full page
+ * image. Note that aggressive cleaning of blocks
+ * dirtied by hint bit setting would increase the call rate.
+ * Bulk setting of hint bits would reduce the call rate...
+ *
+ * We must issue the WAL record before we mark the buffer dirty.
+ * Otherwise we might write the page before we write the WAL.
+ * That causes a race condition, since a checkpoint might occur
+ * between writing the WAL record and marking the buffer dirty.
+ * We solve that with a kluge, but one that is already in use
+ * during transaction commit to prevent race conditions.
+ * Basically, we simply prevent the checkpoint WAL record from
+ * being written until we have marked the buffer dirty. We don't
+ * start the checkpoint flush until we have marked dirty, so our
+ * checkpoint must flush the change to disk successfully or the
+ * checkpoint never gets written, so crash recovery will fix.
+ *
+ * It's possible we may enter here without an xid, so it is
+ * essential that CreateCheckpoint waits for virtual transactions
+ * rather than full transactionids.
+ */
+ MyPgXact->delayChkpt = delayChkpt = true;
+ lsn = XLogSaveBufferForHint(buffer);
+ }
+
LockBufHdr(bufHdr);
Assert(bufHdr->refcount > 0);
if (!(bufHdr->flags & BM_DIRTY))
{
- /* Do vacuum cost accounting */
+ dirtied = true; /* Means "will be dirtied by this action" */
+
+ /*
+ * Set the page LSN if we wrote a backup block. We aren't
+ * supposed to set this when only holding a share lock but
+ * as long as we serialise it somehow we're OK. We choose to
+ * set LSN while holding the buffer header lock, which causes
+ * any reader of an LSN who holds only a share lock to also
+ * obtain a buffer header lock before using PageGetLSN().
+ * Fortunately, thats not too many places.
+ *
+ * If checksums are enabled, you might think we should reset the
+ * checksum here. That will happen when the page is written
+ * sometime later in this checkpoint cycle.
+ */
+ if (!XLogRecPtrIsInvalid(lsn))
+ PageSetLSN(page, lsn);
+ }
+ bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
+ UnlockBufHdr(bufHdr);
+
+ if (delayChkpt)
+ MyPgXact->delayChkpt = false;
+
+ if (dirtied)
+ {
VacuumPageDirty++;
if (VacuumCostActive)
VacuumCostBalance += VacuumCostPageDirty;
}
- bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
- UnlockBufHdr(bufHdr);
}
}
diff --git a/src/backend/storage/buffer/localbuf.c b/src/backend/storage/buffer/localbuf.c
index 03055c96d85..c67271a4bdf 100644
--- a/src/backend/storage/buffer/localbuf.c
+++ b/src/backend/storage/buffer/localbuf.c
@@ -196,16 +196,19 @@ LocalBufferAlloc(SMgrRelation smgr, ForkNumber forkNum, BlockNumber blockNum,
*/
if (bufHdr->flags & BM_DIRTY)
{
- SMgrRelation oreln;
+ SMgrRelation oreln;
+ Page localpage = (char *) LocalBufHdrGetBlock(bufHdr);
/* Find smgr relation for buffer */
oreln = smgropen(bufHdr->tag.rnode, MyBackendId);
+ PageSetChecksumInplace(localpage, bufHdr->tag.blockNum);
+
/* And write... */
smgrwrite(oreln,
bufHdr->tag.forkNum,
bufHdr->tag.blockNum,
- (char *) LocalBufHdrGetBlock(bufHdr),
+ localpage,
false);
/* Mark not-dirty now in case we error out below */
diff --git a/src/backend/storage/freespace/README b/src/backend/storage/freespace/README
index 9732ad54d3e..bbd1b93fac6 100644
--- a/src/backend/storage/freespace/README
+++ b/src/backend/storage/freespace/README
@@ -169,7 +169,9 @@ Recovery
--------
The FSM is not explicitly WAL-logged. Instead, we rely on a bunch of
-self-correcting measures to repair possible corruption.
+self-correcting measures to repair possible corruption. As a result when
+we write to the FSM we treat that as a hint and thus use MarkBufferDirtyHint()
+rather than MarkBufferDirty().
First of all, whenever a value is set on an FSM page, the root node of the
page is compared against the new value after bubbling up the change is
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 9c2afc5b0e8..b76bf9be6b4 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -216,7 +216,7 @@ XLogRecordPageWithFreeSpace(RelFileNode rnode, BlockNumber heapBlk,
PageInit(page, BLCKSZ, 0);
if (fsm_set_avail(page, slot, new_cat))
- MarkBufferDirty(buf);
+ MarkBufferDirtyHint(buf);
UnlockReleaseBuffer(buf);
}
@@ -286,7 +286,7 @@ FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks)
return; /* nothing to do; the FSM was already smaller */
LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
fsm_truncate_avail(BufferGetPage(buf), first_removed_slot);
- MarkBufferDirty(buf);
+ MarkBufferDirtyHint(buf);
UnlockReleaseBuffer(buf);
new_nfsmblocks = fsm_logical_to_physical(first_removed_address) + 1;
@@ -583,6 +583,8 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
while (fsm_nblocks_now < fsm_nblocks)
{
+ PageSetChecksumInplace(pg, fsm_nblocks_now);
+
smgrextend(rel->rd_smgr, FSM_FORKNUM, fsm_nblocks_now,
(char *) pg, false);
fsm_nblocks_now++;
@@ -617,7 +619,7 @@ fsm_set_and_search(Relation rel, FSMAddress addr, uint16 slot,
page = BufferGetPage(buf);
if (fsm_set_avail(page, slot, newValue))
- MarkBufferDirty(buf);
+ MarkBufferDirtyHint(buf);
if (minValue != 0)
{
@@ -768,7 +770,7 @@ fsm_vacuum_page(Relation rel, FSMAddress addr, bool *eof_p)
{
LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
fsm_set_avail(BufferGetPage(buf), slot, child_avail);
- MarkBufferDirty(buf);
+ MarkBufferDirtyHint(buf);
LockBuffer(buf, BUFFER_LOCK_UNLOCK);
}
}
diff --git a/src/backend/storage/freespace/fsmpage.c b/src/backend/storage/freespace/fsmpage.c
index acb8038a870..19c8e09148b 100644
--- a/src/backend/storage/freespace/fsmpage.c
+++ b/src/backend/storage/freespace/fsmpage.c
@@ -284,7 +284,7 @@ restart:
exclusive_lock_held = true;
}
fsm_rebuild_page(page);
- MarkBufferDirty(buf);
+ MarkBufferDirtyHint(buf);
goto restart;
}
}
diff --git a/src/backend/storage/page/bufpage.c b/src/backend/storage/page/bufpage.c
index 95f3e16bfb1..81cdc6547a3 100644
--- a/src/backend/storage/page/bufpage.c
+++ b/src/backend/storage/page/bufpage.c
@@ -15,7 +15,14 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/xlog.h"
+bool ignore_checksum_failure = false;
+
+static char pageCopyData[BLCKSZ]; /* for checksum calculation */
+static Page pageCopy = pageCopyData;
+
+static uint16 PageCalcChecksum16(Page page, BlockNumber blkno);
/* ----------------------------------------------------------------
* Page support functions
@@ -25,6 +32,8 @@
/*
* PageInit
* Initializes the contents of a page.
+ * Note that we don't calculate an initial checksum here; that's not done
+ * until it's time to write.
*/
void
PageInit(Page page, Size pageSize, Size specialSize)
@@ -39,7 +48,7 @@ PageInit(Page page, Size pageSize, Size specialSize)
/* Make sure all fields of page are zero, as well as unused space */
MemSet(p, 0, pageSize);
- /* p->pd_flags = 0; done by above MemSet */
+ p->pd_flags = 0;
p->pd_lower = SizeOfPageHeaderData;
p->pd_upper = pageSize - specialSize;
p->pd_special = pageSize - specialSize;
@@ -49,8 +58,8 @@ PageInit(Page page, Size pageSize, Size specialSize)
/*
- * PageHeaderIsValid
- * Check that the header fields of a page appear valid.
+ * PageIsVerified
+ * Check that the page header and checksum (if any) appear valid.
*
* This is called when a page has just been read in from disk. The idea is
* to cheaply detect trashed pages before we go nuts following bogus item
@@ -67,30 +76,77 @@ PageInit(Page page, Size pageSize, Size specialSize)
* will clean up such a page and make it usable.
*/
bool
-PageHeaderIsValid(PageHeader page)
+PageIsVerified(Page page, BlockNumber blkno)
{
+ PageHeader p = (PageHeader) page;
char *pagebytes;
int i;
+ bool checksum_failure = false;
+ bool header_sane = false;
+ bool all_zeroes = false;
+ uint16 checksum;
- /* Check normal case */
- if (PageGetPageSize(page) == BLCKSZ &&
- PageGetPageLayoutVersion(page) == PG_PAGE_LAYOUT_VERSION &&
- (page->pd_flags & ~PD_VALID_FLAG_BITS) == 0 &&
- page->pd_lower >= SizeOfPageHeaderData &&
- page->pd_lower <= page->pd_upper &&
- page->pd_upper <= page->pd_special &&
- page->pd_special <= BLCKSZ &&
- page->pd_special == MAXALIGN(page->pd_special))
- return true;
+ /*
+ * Don't verify page data unless the page passes basic non-zero test
+ */
+ if (!PageIsNew(page))
+ {
+ if (DataChecksumsEnabled())
+ {
+ checksum = PageCalcChecksum16(page, blkno);
+
+ if (checksum != p->pd_checksum)
+ checksum_failure = true;
+ }
+
+ /*
+ * The following checks don't prove the header is correct,
+ * only that it looks sane enough to allow into the buffer pool.
+ * Later usage of the block can still reveal problems,
+ * which is why we offer the checksum option.
+ */
+ if ((p->pd_flags & ~PD_VALID_FLAG_BITS) == 0 &&
+ p->pd_lower <= p->pd_upper &&
+ p->pd_upper <= p->pd_special &&
+ p->pd_special <= BLCKSZ &&
+ p->pd_special == MAXALIGN(p->pd_special))
+ header_sane = true;
+
+ if (header_sane && !checksum_failure)
+ return true;
+ }
/* Check all-zeroes case */
+ all_zeroes = true;
pagebytes = (char *) page;
for (i = 0; i < BLCKSZ; i++)
{
if (pagebytes[i] != 0)
- return false;
+ {
+ all_zeroes = false;
+ break;
+ }
+ }
+
+ if (all_zeroes)
+ return true;
+
+ /*
+ * Throw a WARNING if the checksum fails, but only after we've checked for
+ * the all-zeroes case.
+ */
+ if (checksum_failure)
+ {
+ ereport(WARNING,
+ (ERRCODE_DATA_CORRUPTED,
+ errmsg("page verification failed, calculated checksum %u but expected %u",
+ checksum, p->pd_checksum)));
+
+ if (header_sane && ignore_checksum_failure)
+ return true;
}
- return true;
+
+ return false;
}
@@ -827,3 +883,98 @@ PageIndexMultiDelete(Page page, OffsetNumber *itemnos, int nitems)
pfree(itemidbase);
}
+
+/*
+ * Set checksum for page in shared buffers.
+ *
+ * If checksums are disabled, or if the page is not initialized, just return
+ * the input. Otherwise, we must make a copy of the page before calculating the
+ * checksum, to prevent concurrent modifications (e.g. setting hint bits) from
+ * making the final checksum invalid.
+ *
+ * Returns a pointer to the block-sized data that needs to be written. Uses
+ * statically-allocated memory, so the caller must immediately write the
+ * returned page and not refer to it again.
+ */
+char *
+PageSetChecksumCopy(Page page, BlockNumber blkno)
+{
+ if (PageIsNew(page) || !DataChecksumsEnabled())
+ return (char *) page;
+
+ /*
+ * We make a copy iff we need to calculate a checksum because other
+ * backends may set hint bits on this page while we write, which
+ * would mean the checksum differs from the page contents. It doesn't
+ * matter if we include or exclude hints during the copy, as long
+ * as we write a valid page and associated checksum.
+ */
+ memcpy((char *) pageCopy, (char *) page, BLCKSZ);
+ PageSetChecksumInplace(pageCopy, blkno);
+ return (char *) pageCopy;
+}
+
+/*
+ * Set checksum for page in private memory.
+ *
+ * This is a simpler version of PageSetChecksumCopy(). The more explicit API
+ * allows us to more easily see if we're making the correct call and reduces
+ * the amount of additional code specific to page verification.
+ */
+void
+PageSetChecksumInplace(Page page, BlockNumber blkno)
+{
+ if (PageIsNew(page))
+ return;
+
+ if (DataChecksumsEnabled())
+ {
+ PageHeader p = (PageHeader) page;
+ p->pd_checksum = PageCalcChecksum16(page, blkno);
+ }
+
+ return;
+}
+
+/*
+ * Calculate checksum for a PostgreSQL Page. This includes the block number (to
+ * detect the case when a page is somehow moved to a different location), the
+ * page header (excluding the checksum itself), and the page data.
+ *
+ * Note that if the checksum validation fails we cannot tell the difference
+ * between a transposed block and failure from direct on-block corruption,
+ * though that is better than just ignoring transposed blocks altogether.
+ */
+static uint16
+PageCalcChecksum16(Page page, BlockNumber blkno)
+{
+ pg_crc32 crc;
+ PageHeader p = (PageHeader) page;
+
+ /* only calculate the checksum for properly-initialized pages */
+ Assert(!PageIsNew(page));
+
+ INIT_CRC32(crc);
+
+ /*
+ * Initialize the checksum calculation with the block number. This helps
+ * catch corruption from whole blocks being transposed with other whole
+ * blocks.
+ */
+ COMP_CRC32(crc, &blkno, sizeof(blkno));
+
+ /*
+ * Now add in the LSN, which is always the first field on the page.
+ */
+ COMP_CRC32(crc, page, sizeof(p->pd_lsn));
+
+ /*
+ * Now add the rest of the page, skipping the pd_checksum field.
+ */
+ COMP_CRC32(crc, page + sizeof(p->pd_lsn) + sizeof(p->pd_checksum),
+ BLCKSZ - sizeof(p->pd_lsn) - sizeof(p->pd_checksum));
+
+ FIN_CRC32(crc);
+
+ return (uint16) crc;
+}