OSDN Git Service

Make the visibility map crash-safe.
[pg-rex/syncrep.git] / src / backend / access / heap / visibilitymap.c
index e3cbb4e..a193520 100644 (file)
@@ -3,26 +3,27 @@
  * visibilitymap.c
  *       bitmap for tracking visibility of heap tuples
  *
- * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1996-2011, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
- *       $PostgreSQL: pgsql/src/backend/access/heap/visibilitymap.c,v 1.1 2008/12/03 13:05:22 heikki Exp $
+ *       src/backend/access/heap/visibilitymap.c
  *
  * INTERFACE ROUTINES
- *             visibilitymap_clear     - clear a bit in the visibility map
- *             visibilitymap_pin       - pin a map page for setting a bit
- *             visibilitymap_set       - set a bit in a previously pinned page
- *             visibilitymap_test      - test if a bit is set
+ *             visibilitymap_clear  - clear a bit in the visibility map
+ *             visibilitymap_pin        - pin a map page for setting a bit
+ *             visibilitymap_pin_ok - check whether correct map page is already pinned
+ *             visibilitymap_set        - set a bit in a previously pinned page
+ *             visibilitymap_test       - test if a bit is set
  *
  * NOTES
  *
  * The visibility map is a bitmap with one bit per heap page. A set bit means
- * that all tuples on the page are visible to all transactions, and doesn't
- * therefore need to be vacuumed. The map is conservative in the sense that we
- * make sure that whenever a bit is set, we know the condition is true, but if
- * a bit is not set, it might or might not be.
+ * that all tuples on the page are known visible to all transactions, and
+ * therefore the page doesn't need to be vacuumed. The map is conservative in
+ * the sense that we make sure that whenever a bit is set, we know the
+ * condition is true, but if a bit is not set, it might or might not be true.
  *
  * There's no explicit WAL logging in the functions in this file. The callers
  * must make sure that whenever a bit is cleared, the bit is cleared on WAL
  * make VACUUM skip pages that need vacuuming, until the next anti-wraparound
  * vacuum. The visibility map is not used for anti-wraparound vacuums, because
  * an anti-wraparound vacuum needs to freeze tuples and observe the latest xid
- * present in the table, also on pages that don't have any dead tuples.
+ * present in the table, even on pages that don't have any dead tuples.
  *
  * Although the visibility map is just a hint at the moment, the PD_ALL_VISIBLE
- * flag on heap pages *must* be correct.
+ * flag on heap pages *must* be correct, because it is used to skip visibility
+ * checking.
  *
  * LOCKING
  *
  * When a bit is set, the LSN of the visibility map page is updated to make
  * sure that the visibility map update doesn't get written to disk before the
  * WAL record of the changes that made it possible to set the bit is flushed.
- * But when a bit is cleared, we don't have to do that because it's always OK
- * to clear a bit in the map from correctness point of view.
+ * But when a bit is cleared, we don't have to do that because it's always
+ * safe to clear a bit in the map from correctness point of view.
  *
  * TODO
  *
- * It would be nice to use the visibility map to skip visibility checkes in
+ * It would be nice to use the visibility map to skip visibility checks in
  * index scans.
  *
- * Currently, the visibility map is not 100% correct all the time.
- * During updates, the bit in the visibility map is cleared after releasing
- * the lock on the heap page. During the window after releasing the lock
- * and clearing the bit in the visibility map, the bit in the visibility map
- * is set, but the new insertion or deletion is not yet visible to other
- * backends.
- *
- * That might actually be OK for the index scans, though. The newly inserted
- * tuple wouldn't have an index pointer yet, so all tuples reachable from an
- * index would still be visible to all other backends, and deletions wouldn't
- * be visible to other backends yet.
- *
- * There's another hole in the way the PD_ALL_VISIBLE flag is set. When
- * vacuum observes that all tuples are visible to all, it sets the flag on
- * the heap page, and also sets the bit in the visibility map. If we then
- * crash, and only the visibility map page was flushed to disk, we'll have
- * a bit set in the visibility map, but the corresponding flag on the heap
- * page is not set. If the heap page is then updated, the updater won't
- * know to clear the bit in the visibility map.
- *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
+#include "access/heapam.h"
 #include "access/visibilitymap.h"
+#include "miscadmin.h"
 #include "storage/bufmgr.h"
 #include "storage/bufpage.h"
 #include "storage/lmgr.h"
 #include "storage/smgr.h"
-#include "utils/inval.h"
+
 
 /*#define TRACE_VISIBILITYMAP */
 
 /*
  * Size of the bitmap on each visibility map page, in bytes. There's no
- * extra headers, so the whole page minus except for the standard page header
- * is used for the bitmap.
+ * extra headers, so the whole page minus the standard page header is
+ * used for the bitmap.
  */
-#define MAPSIZE (BLCKSZ - SizeOfPageHeaderData)
+#define MAPSIZE (BLCKSZ - MAXALIGN(SizeOfPageHeaderData))
 
 /* Number of bits allocated for each heap block. */
 #define BITS_PER_HEAPBLOCK 1
@@ -125,38 +109,37 @@ static void vm_extend(Relation rel, BlockNumber nvmblocks);
 /*
  *     visibilitymap_clear - clear a bit in visibility map
  *
- * Clear a bit in the visibility map, marking that not all tuples are
- * visible to all transactions anymore.
+ * You must pass a buffer containing the correct map page to this function.
+ * Call visibilitymap_pin first to pin the right one. This function doesn't do
+ * any I/O.
  */
 void
-visibilitymap_clear(Relation rel, BlockNumber heapBlk)
+visibilitymap_clear(Relation rel, BlockNumber heapBlk, Buffer buf)
 {
        BlockNumber mapBlock = HEAPBLK_TO_MAPBLOCK(heapBlk);
        int                     mapByte = HEAPBLK_TO_MAPBYTE(heapBlk);
        int                     mapBit = HEAPBLK_TO_MAPBIT(heapBlk);
        uint8           mask = 1 << mapBit;
-       Buffer          mapBuffer;
        char       *map;
 
 #ifdef TRACE_VISIBILITYMAP
        elog(DEBUG1, "vm_clear %s %d", RelationGetRelationName(rel), heapBlk);
 #endif
 
-       mapBuffer = vm_readbuf(rel, mapBlock, false);
-       if (!BufferIsValid(mapBuffer))
-               return; /* nothing to do */
+       if (!BufferIsValid(buf) || BufferGetBlockNumber(buf) != mapBlock)
+               elog(ERROR, "wrong buffer passed to visibilitymap_clear");
 
-       LockBuffer(mapBuffer, BUFFER_LOCK_EXCLUSIVE);
-       map = PageGetContents(BufferGetPage(mapBuffer));
+       LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
+       map = PageGetContents(BufferGetPage(buf));
 
        if (map[mapByte] & mask)
        {
                map[mapByte] &= ~mask;
 
-               MarkBufferDirty(mapBuffer);
+               MarkBufferDirty(buf);
        }
 
-       UnlockReleaseBuffer(mapBuffer);
+       LockBuffer(buf, BUFFER_LOCK_UNLOCK);
 }
 
 /*
@@ -171,7 +154,7 @@ visibilitymap_clear(Relation rel, BlockNumber heapBlk)
  * On entry, *buf should be InvalidBuffer or a valid buffer returned by
  * an earlier call to visibilitymap_pin or visibilitymap_test on the same
  * relation. On return, *buf is a valid buffer with the map page containing
- * the the bit for heapBlk.
+ * the bit for heapBlk.
  *
  * If the page doesn't exist in the map file yet, it is extended.
  */
@@ -192,19 +175,36 @@ visibilitymap_pin(Relation rel, BlockNumber heapBlk, Buffer *buf)
 }
 
 /*
+ *     visibilitymap_pin_ok - do we already have the correct page pinned?
+ *
+ * On entry, buf should be InvalidBuffer or a valid buffer returned by
+ * an earlier call to visibilitymap_pin or visibilitymap_test on the same
+ * relation.  The return value indicates whether the buffer covers the
+ * given heapBlk.
+ */
+bool
+visibilitymap_pin_ok(BlockNumber heapBlk, Buffer buf)
+{
+       BlockNumber mapBlock = HEAPBLK_TO_MAPBLOCK(heapBlk);
+
+       return BufferIsValid(buf) && BufferGetBlockNumber(buf) == mapBlock;
+}
+
+/*
  *     visibilitymap_set - set a bit on a previously pinned page
  *
- * recptr is the LSN of the heap page. The LSN of the visibility map page is
- * advanced to that, to make sure that the visibility map doesn't get flushed
- * to disk before the update to the heap page that made all tuples visible.
+ * recptr is the LSN of the XLOG record we're replaying, if we're in recovery,
+ * or InvalidXLogRecPtr in normal running.  The page LSN is advanced to the
+ * one provided; in normal running, we generate a new XLOG record and set the
+ * page LSN to that value.
  *
- * This is an opportunistic function. It does nothing, unless *buf
- * contains the bit for heapBlk. Call visibilitymap_pin first to pin
- * the right map page. This function doesn't do any I/O.
+ * You must pass a buffer containing the correct map page to this function.
+ * Call visibilitymap_pin first to pin the right one. This function doesn't do
+ * any I/O.
  */
 void
 visibilitymap_set(Relation rel, BlockNumber heapBlk, XLogRecPtr recptr,
-                                 Buffer *buf)
+                                 Buffer buf)
 {
        BlockNumber mapBlock = HEAPBLK_TO_MAPBLOCK(heapBlk);
        uint32          mapByte = HEAPBLK_TO_MAPBYTE(heapBlk);
@@ -216,25 +216,35 @@ visibilitymap_set(Relation rel, BlockNumber heapBlk, XLogRecPtr recptr,
        elog(DEBUG1, "vm_set %s %d", RelationGetRelationName(rel), heapBlk);
 #endif
 
+       Assert(InRecovery || XLogRecPtrIsInvalid(recptr));
+
        /* Check that we have the right page pinned */
-       if (!BufferIsValid(*buf) || BufferGetBlockNumber(*buf) != mapBlock)
-               return;
+       if (!BufferIsValid(buf) || BufferGetBlockNumber(buf) != mapBlock)
+               elog(ERROR, "wrong buffer passed to visibilitymap_set");
 
-       page = BufferGetPage(*buf);
+       page = BufferGetPage(buf);
        map = PageGetContents(page);
-       LockBuffer(*buf, BUFFER_LOCK_EXCLUSIVE);
+       LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
 
        if (!(map[mapByte] & (1 << mapBit)))
        {
+               START_CRIT_SECTION();
+
                map[mapByte] |= (1 << mapBit);
+               MarkBufferDirty(buf);
 
-               if (XLByteLT(PageGetLSN(page), recptr))
+               if (RelationNeedsWAL(rel))
+               {
+                       if (XLogRecPtrIsInvalid(recptr))
+                               recptr = log_heap_visible(rel->rd_node, heapBlk, buf);
                        PageSetLSN(page, recptr);
-               PageSetTLI(page, ThisTimeLineID);
-               MarkBufferDirty(*buf);
+                       PageSetTLI(page, ThisTimeLineID);
+               }
+
+               END_CRIT_SECTION();
        }
 
-       LockBuffer(*buf, BUFFER_LOCK_UNLOCK);
+       LockBuffer(buf, BUFFER_LOCK_UNLOCK);
 }
 
 /*
@@ -245,7 +255,7 @@ visibilitymap_set(Relation rel, BlockNumber heapBlk, XLogRecPtr recptr,
  * On entry, *buf should be InvalidBuffer or a valid buffer returned by an
  * earlier call to visibilitymap_pin or visibilitymap_test on the same
  * relation. On return, *buf is a valid buffer with the map page containing
- * the the bit for heapBlk, or InvalidBuffer. The caller is responsible for
+ * the bit for heapBlk, or InvalidBuffer. The caller is responsible for
  * releasing *buf after it's done testing and setting bits.
  */
 bool
@@ -289,21 +299,30 @@ visibilitymap_test(Relation rel, BlockNumber heapBlk, Buffer *buf)
 }
 
 /*
- *     visibilitymap_test - truncate the visibility map
+ *     visibilitymap_truncate - truncate the visibility map
+ *
+ * The caller must hold AccessExclusiveLock on the relation, to ensure that
+ * other backends receive the smgr invalidation event that this function sends
+ * before they access the VM again.
+ *
+ * nheapblocks is the new size of the heap.
  */
 void
 visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
 {
        BlockNumber newnblocks;
+
        /* last remaining block, byte, and bit */
        BlockNumber truncBlock = HEAPBLK_TO_MAPBLOCK(nheapblocks);
-       uint32          truncByte  = HEAPBLK_TO_MAPBYTE(nheapblocks);
-       uint8           truncBit   = HEAPBLK_TO_MAPBIT(nheapblocks);
+       uint32          truncByte = HEAPBLK_TO_MAPBYTE(nheapblocks);
+       uint8           truncBit = HEAPBLK_TO_MAPBIT(nheapblocks);
 
 #ifdef TRACE_VISIBILITYMAP
        elog(DEBUG1, "vm_truncate %s %d", RelationGetRelationName(rel), nheapblocks);
 #endif
 
+       RelationOpenSmgr(rel);
+
        /*
         * If no visibility map has been created yet for this relation, there's
         * nothing to truncate.
@@ -315,14 +334,14 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
         * Unless the new size is exactly at a visibility map page boundary, the
         * tail bits in the last remaining map page, representing truncated heap
         * blocks, need to be cleared. This is not only tidy, but also necessary
-        * because we don't get a chance to clear the bits if the heap is
-        * extended again.
+        * because we don't get a chance to clear the bits if the heap is extended
+        * again.
         */
        if (truncByte != 0 || truncBit != 0)
        {
-               Buffer mapBuffer;
-               Page page;
-               char *map;
+               Buffer          mapBuffer;
+               Page            page;
+               char       *map;
 
                newnblocks = truncBlock + 1;
 
@@ -344,11 +363,8 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
                /*
                 * Mask out the unwanted bits of the last remaining byte.
                 *
-                * ((1 << 0) - 1) = 00000000
-                * ((1 << 1) - 1) = 00000001
-                * ...
-                * ((1 << 6) - 1) = 00111111
-                * ((1 << 7) - 1) = 01111111
+                * ((1 << 0) - 1) = 00000000 ((1 << 1) - 1) = 00000001 ... ((1 << 6) -
+                * 1) = 00111111 ((1 << 7) - 1) = 01111111
                 */
                map[truncByte] &= (1 << truncBit) - 1;
 
@@ -358,23 +374,23 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
        else
                newnblocks = truncBlock;
 
-       if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) < newnblocks)
+       if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
        {
                /* nothing to do, the file was already smaller than requested size */
                return;
        }
 
-       smgrtruncate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, newnblocks,
-                                rel->rd_istemp);
+       /* Truncate the unused VM pages, and send smgr inval message */
+       smgrtruncate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, newnblocks);
 
        /*
-        * Need to invalidate the relcache entry, because rd_vm_nblocks
-        * seen by other backends is no longer valid.
+        * We might as well update the local smgr_vm_nblocks setting. smgrtruncate
+        * sent an smgr cache inval message, which will cause other backends to
+        * invalidate their copy of smgr_vm_nblocks, and this one too at the next
+        * command boundary.  But this ensures it isn't outright wrong until then.
         */
-       if (!InRecovery)
-               CacheInvalidateRelcache(rel);
-
-       rel->rd_vm_nblocks = newnblocks;
+       if (rel->rd_smgr)
+               rel->rd_smgr->smgr_vm_nblocks = newnblocks;
 }
 
 /*
@@ -386,26 +402,28 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
 static Buffer
 vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 {
-       Buffer buf;
+       Buffer          buf;
 
        RelationOpenSmgr(rel);
 
        /*
-        * The current size of the visibility map fork is kept in relcache, to
-        * avoid reading beyond EOF. If we haven't cached the size of the map yet,
-        * do that first.
+        * If we haven't cached the size of the visibility map fork yet, check it
+        * first.  Also recheck if the requested block seems to be past end, since
+        * our cached value might be stale.  (We send smgr inval messages on
+        * truncation, but not on extension.)
         */
-       if (rel->rd_vm_nblocks == InvalidBlockNumber)
+       if (rel->rd_smgr->smgr_vm_nblocks == InvalidBlockNumber ||
+               blkno >= rel->rd_smgr->smgr_vm_nblocks)
        {
                if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-                       rel->rd_vm_nblocks = smgrnblocks(rel->rd_smgr,
-                                                                                        VISIBILITYMAP_FORKNUM);
+                       rel->rd_smgr->smgr_vm_nblocks = smgrnblocks(rel->rd_smgr,
+                                                                                                         VISIBILITYMAP_FORKNUM);
                else
-                       rel->rd_vm_nblocks = 0;
+                       rel->rd_smgr->smgr_vm_nblocks = 0;
        }
 
        /* Handle requests beyond EOF */
-       if (blkno >= rel->rd_vm_nblocks)
+       if (blkno >= rel->rd_smgr->smgr_vm_nblocks)
        {
                if (extend)
                        vm_extend(rel, blkno + 1);
@@ -433,46 +451,48 @@ static void
 vm_extend(Relation rel, BlockNumber vm_nblocks)
 {
        BlockNumber vm_nblocks_now;
-       Page pg;
+       Page            pg;
 
        pg = (Page) palloc(BLCKSZ);
        PageInit(pg, BLCKSZ, 0);
 
        /*
-        * We use the relation extension lock to lock out other backends trying
-        * to extend the visibility map at the same time. It also locks out
-        * extension of the main fork, unnecessarily, but extending the
-        * visibility map happens seldom enough that it doesn't seem worthwhile to
-        * have a separate lock tag type for it.
+        * We use the relation extension lock to lock out other backends trying to
+        * extend the visibility map at the same time. It also locks out extension
+        * of the main fork, unnecessarily, but extending the visibility map
+        * happens seldom enough that it doesn't seem worthwhile to have a
+        * separate lock tag type for it.
         *
-        * Note that another backend might have extended or created the
-        * relation before we get the lock.
+        * Note that another backend might have extended or created the relation
+        * by the time we get the lock.
         */
        LockRelationForExtension(rel, ExclusiveLock);
 
-       /* Create the file first if it doesn't exist */
-       if ((rel->rd_vm_nblocks == 0 || rel->rd_vm_nblocks == InvalidBlockNumber)
-               && !smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-       {
+       /* Might have to re-open if a cache flush happened */
+       RelationOpenSmgr(rel);
+
+       /*
+        * Create the file first if it doesn't exist.  If smgr_vm_nblocks is
+        * positive then it must exist, no need for an smgrexists call.
+        */
+       if ((rel->rd_smgr->smgr_vm_nblocks == 0 ||
+                rel->rd_smgr->smgr_vm_nblocks == InvalidBlockNumber) &&
+               !smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
                smgrcreate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, false);
-               vm_nblocks_now = 0;
-       }
-       else
-               vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+
+       vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
 
        while (vm_nblocks_now < vm_nblocks)
        {
                smgrextend(rel->rd_smgr, VISIBILITYMAP_FORKNUM, vm_nblocks_now,
-                                  (char *) pg, rel->rd_istemp);
+                                  (char *) pg, false);
                vm_nblocks_now++;
        }
 
+       /* Update local cache with the up-to-date size */
+       rel->rd_smgr->smgr_vm_nblocks = vm_nblocks_now;
+
        UnlockRelationForExtension(rel, ExclusiveLock);
 
        pfree(pg);
-
-       /* Update the relcache with the up-to-date size */
-       if (!InRecovery)
-               CacheInvalidateRelcache(rel);
-       rel->rd_vm_nblocks = vm_nblocks_now;
 }