Remove tupgone special case from vacuumlazy.c.
authorPeter Geoghegan <pg@bowt.ie>
Tue, 6 Apr 2021 15:49:22 +0000 (08:49 -0700)
committerPeter Geoghegan <pg@bowt.ie>
Tue, 6 Apr 2021 15:49:22 +0000 (08:49 -0700)
Retry the call to heap_prune_page() in rare cases where there is
disagreement between the heap_prune_page() call and the call to
HeapTupleSatisfiesVacuum() that immediately follows.  Disagreement is
possible when a concurrently-aborted transaction makes a tuple DEAD
during the tiny window between each step.  This was the only case where
a tuple considered DEAD by VACUUM still had storage following pruning.
VACUUM's definition of dead tuples is now uniformly simple and
unambiguous: dead tuples from each page are always LP_DEAD line pointers
that were encountered just after we performed pruning (and just before
we considered freezing remaining items with tuple storage).

Eliminating the tupgone=true special case enables INDEX_CLEANUP=off
style skipping of index vacuuming that takes place based on flexible,
dynamic criteria.  The INDEX_CLEANUP=off case had to know about skipping
indexes up-front before now, due to a subtle interaction with the
special case (see commit dd695979) -- this was a special case unto
itself.  Now there are no special cases.  And so now it won't matter
when or how we decide to skip index vacuuming: it won't affect how
pruning behaves, and it won't be affected by any of the implementation
details of pruning or freezing.

Also remove XLOG_HEAP2_CLEANUP_INFO records.  These are no longer
necessary because we now rely entirely on heap pruning taking care of
recovery conflicts.  There is no longer any need to generate recovery
conflicts for DEAD tuples that pruning just missed.  This also means
that heap vacuuming now uses exactly the same strategy for recovery
conflicts as index vacuuming always has: REDO routines never need to
process a latestRemovedXid from the WAL record, since earlier REDO of
the WAL record from pruning is sufficient in all cases.  The generic
XLOG_HEAP2_CLEAN record type is now split into two new record types to
reflect this new division (these are called XLOG_HEAP2_PRUNE and
XLOG_HEAP2_VACUUM).

Also stop acquiring a super-exclusive lock for heap pages when they're
vacuumed during VACUUM's second heap pass.  A regular exclusive lock is
enough.  This is correct because heap page vacuuming is now strictly a
matter of setting the LP_DEAD line pointers to LP_UNUSED.  No other
backend can have a pointer to a tuple located in a pinned buffer that
can be invalidated by a concurrent heap page vacuum operation.

Heap vacuuming can now be thought of as conceptually similar to index
vacuuming and conceptually dissimilar to heap pruning.  Heap pruning now
has sole responsibility for anything involving the logical contents of
the database (e.g., managing transaction status information, recovery
conflicts, considering what to do with HOT chains).  Index vacuuming and
heap vacuuming are now only concerned with recycling garbage items from
physical data structures that back the logical database.

Bump XLOG_PAGE_MAGIC due to pruning and heap page vacuum WAL record
changes.

Credit for the idea of retrying pruning a page to avoid the tupgone case
goes to Andres Freund.

Author: Peter Geoghegan <pg@bowt.ie>
Reviewed-By: Andres Freund <andres@anarazel.de>
Reviewed-By: Masahiko Sawada <sawada.mshk@gmail.com>
Discussion: https://postgr.es/m/CAH2-WznneCXTzuFmcwx_EyRQgfsfJAAsu+CsqRFmFXCAar=nJw@mail.gmail.com

12 files changed:
src/backend/access/gist/gistxlog.c
src/backend/access/hash/hash_xlog.c
src/backend/access/heap/heapam.c
src/backend/access/heap/pruneheap.c
src/backend/access/heap/vacuumlazy.c
src/backend/access/nbtree/nbtree.c
src/backend/access/rmgrdesc/heapdesc.c
src/backend/replication/logical/decode.c
src/include/access/heapam.h
src/include/access/heapam_xlog.h
src/include/access/xlog_internal.h
src/tools/pgindent/typedefs.list

index 1c80eae044a94a55d2a79b57c9a2c7f91b46dda2..6464cb9281b90607a66e23ec5fee41f8befed3c8 100644 (file)
@@ -184,10 +184,10 @@ gistRedoDeleteRecord(XLogReaderState *record)
     *
     * GiST delete records can conflict with standby queries.  You might think
     * that vacuum records would conflict as well, but we've handled that
-    * already.  XLOG_HEAP2_CLEANUP_INFO records provide the highest xid
-    * cleaned by the vacuum of the heap and so we can resolve any conflicts
-    * just once when that arrives.  After that we know that no conflicts
-    * exist from individual gist vacuum records on that index.
+    * already.  XLOG_HEAP2_PRUNE records provide the highest xid cleaned by
+    * the vacuum of the heap and so we can resolve any conflicts just once
+    * when that arrives.  After that we know that no conflicts exist from
+    * individual gist vacuum records on that index.
     */
    if (InHotStandby)
    {
index 02d9e6cdfd98965626c7c3839b284838682f2242..af35a991fc300ccb99b48d707cf39a14a47b7f26 100644 (file)
@@ -992,10 +992,10 @@ hash_xlog_vacuum_one_page(XLogReaderState *record)
     * Hash index records that are marked as LP_DEAD and being removed during
     * hash index tuple insertion can conflict with standby queries. You might
     * think that vacuum records would conflict as well, but we've handled
-    * that already.  XLOG_HEAP2_CLEANUP_INFO records provide the highest xid
-    * cleaned by the vacuum of the heap and so we can resolve any conflicts
-    * just once when that arrives.  After that we know that no conflicts
-    * exist from individual hash index vacuum records on that index.
+    * that already.  XLOG_HEAP2_PRUNE records provide the highest xid cleaned
+    * by the vacuum of the heap and so we can resolve any conflicts just once
+    * when that arrives.  After that we know that no conflicts exist from
+    * individual hash index vacuum records on that index.
     */
    if (InHotStandby)
    {
index 595310ba1b206f357defb4452d3fa0dba2a275b9..9cbc161d7a9f732471bf3bdb090480530bd7acea 100644 (file)
@@ -7538,7 +7538,7 @@ heap_index_delete_tuples(Relation rel, TM_IndexDeleteOp *delstate)
             * must have considered the original tuple header as part of
             * generating its own latestRemovedXid value.
             *
-            * Relying on XLOG_HEAP2_CLEAN records like this is the same
+            * Relying on XLOG_HEAP2_PRUNE records like this is the same
             * strategy that index vacuuming uses in all cases.  Index VACUUM
             * WAL records don't even have a latestRemovedXid field of their
             * own for this reason.
@@ -7957,88 +7957,6 @@ bottomup_sort_and_shrink(TM_IndexDeleteOp *delstate)
    return nblocksfavorable;
 }
 
-/*
- * Perform XLogInsert to register a heap cleanup info message. These
- * messages are sent once per VACUUM and are required because
- * of the phasing of removal operations during a lazy VACUUM.
- * see comments for vacuum_log_cleanup_info().
- */
-XLogRecPtr
-log_heap_cleanup_info(RelFileNode rnode, TransactionId latestRemovedXid)
-{
-   xl_heap_cleanup_info xlrec;
-   XLogRecPtr  recptr;
-
-   xlrec.node = rnode;
-   xlrec.latestRemovedXid = latestRemovedXid;
-
-   XLogBeginInsert();
-   XLogRegisterData((char *) &xlrec, SizeOfHeapCleanupInfo);
-
-   recptr = XLogInsert(RM_HEAP2_ID, XLOG_HEAP2_CLEANUP_INFO);
-
-   return recptr;
-}
-
-/*
- * Perform XLogInsert for a heap-clean operation.  Caller must already
- * have modified the buffer and marked it dirty.
- *
- * Note: prior to Postgres 8.3, the entries in the nowunused[] array were
- * zero-based tuple indexes.  Now they are one-based like other uses
- * of OffsetNumber.
- *
- * We also include latestRemovedXid, which is the greatest XID present in
- * the removed tuples. That allows recovery processing to cancel or wait
- * for long standby queries that can still see these tuples.
- */
-XLogRecPtr
-log_heap_clean(Relation reln, Buffer buffer,
-              OffsetNumber *redirected, int nredirected,
-              OffsetNumber *nowdead, int ndead,
-              OffsetNumber *nowunused, int nunused,
-              TransactionId latestRemovedXid)
-{
-   xl_heap_clean xlrec;
-   XLogRecPtr  recptr;
-
-   /* Caller should not call me on a non-WAL-logged relation */
-   Assert(RelationNeedsWAL(reln));
-
-   xlrec.latestRemovedXid = latestRemovedXid;
-   xlrec.nredirected = nredirected;
-   xlrec.ndead = ndead;
-
-   XLogBeginInsert();
-   XLogRegisterData((char *) &xlrec, SizeOfHeapClean);
-
-   XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
-
-   /*
-    * The OffsetNumber arrays are not actually in the buffer, but we pretend
-    * that they are.  When XLogInsert stores the whole buffer, the offset
-    * arrays need not be stored too.  Note that even if all three arrays are
-    * empty, we want to expose the buffer as a candidate for whole-page
-    * storage, since this record type implies a defragmentation operation
-    * even if no line pointers changed state.
-    */
-   if (nredirected > 0)
-       XLogRegisterBufData(0, (char *) redirected,
-                           nredirected * sizeof(OffsetNumber) * 2);
-
-   if (ndead > 0)
-       XLogRegisterBufData(0, (char *) nowdead,
-                           ndead * sizeof(OffsetNumber));
-
-   if (nunused > 0)
-       XLogRegisterBufData(0, (char *) nowunused,
-                           nunused * sizeof(OffsetNumber));
-
-   recptr = XLogInsert(RM_HEAP2_ID, XLOG_HEAP2_CLEAN);
-
-   return recptr;
-}
-
 /*
  * Perform XLogInsert for a heap-freeze operation.  Caller must have already
  * modified the buffer and marked it dirty.
@@ -8510,34 +8428,15 @@ ExtractReplicaIdentity(Relation relation, HeapTuple tp, bool key_changed,
 }
 
 /*
- * Handles CLEANUP_INFO
- */
-static void
-heap_xlog_cleanup_info(XLogReaderState *record)
-{
-   xl_heap_cleanup_info *xlrec = (xl_heap_cleanup_info *) XLogRecGetData(record);
-
-   if (InHotStandby)
-       ResolveRecoveryConflictWithSnapshot(xlrec->latestRemovedXid, xlrec->node);
-
-   /*
-    * Actual operation is a no-op. Record type exists to provide a means for
-    * conflict processing to occur before we begin index vacuum actions. see
-    * vacuumlazy.c and also comments in btvacuumpage()
-    */
-
-   /* Backup blocks are not used in cleanup_info records */
-   Assert(!XLogRecHasAnyBlockRefs(record));
-}
-
-/*
- * Handles XLOG_HEAP2_CLEAN record type
+ * Handles XLOG_HEAP2_PRUNE record type.
+ *
+ * Acquires a super-exclusive lock.
  */
 static void
-heap_xlog_clean(XLogReaderState *record)
+heap_xlog_prune(XLogReaderState *record)
 {
    XLogRecPtr  lsn = record->EndRecPtr;
-   xl_heap_clean *xlrec = (xl_heap_clean *) XLogRecGetData(record);
+   xl_heap_prune *xlrec = (xl_heap_prune *) XLogRecGetData(record);
    Buffer      buffer;
    RelFileNode rnode;
    BlockNumber blkno;
@@ -8548,12 +8447,8 @@ heap_xlog_clean(XLogReaderState *record)
    /*
     * We're about to remove tuples. In Hot Standby mode, ensure that there's
     * no queries running for which the removed tuples are still visible.
-    *
-    * Not all HEAP2_CLEAN records remove tuples with xids, so we only want to
-    * conflict on the records that cause MVCC failures for user queries. If
-    * latestRemovedXid is invalid, skip conflict processing.
     */
-   if (InHotStandby && TransactionIdIsValid(xlrec->latestRemovedXid))
+   if (InHotStandby)
        ResolveRecoveryConflictWithSnapshot(xlrec->latestRemovedXid, rnode);
 
    /*
@@ -8606,7 +8501,7 @@ heap_xlog_clean(XLogReaderState *record)
        UnlockReleaseBuffer(buffer);
 
        /*
-        * After cleaning records from a page, it's useful to update the FSM
+        * After pruning records from a page, it's useful to update the FSM
         * about it, as it may cause the page become target for insertions
         * later even if vacuum decides not to visit it (which is possible if
         * gets marked all-visible.)
@@ -8618,6 +8513,80 @@ heap_xlog_clean(XLogReaderState *record)
    }
 }
 
+/*
+ * Handles XLOG_HEAP2_VACUUM record type.
+ *
+ * Acquires an exclusive lock only.
+ */
+static void
+heap_xlog_vacuum(XLogReaderState *record)
+{
+   XLogRecPtr  lsn = record->EndRecPtr;
+   xl_heap_vacuum *xlrec = (xl_heap_vacuum *) XLogRecGetData(record);
+   Buffer      buffer;
+   BlockNumber blkno;
+   XLogRedoAction action;
+
+   /*
+    * If we have a full-page image, restore it (without using a cleanup lock)
+    * and we're done.
+    */
+   action = XLogReadBufferForRedoExtended(record, 0, RBM_NORMAL, false,
+                                          &buffer);
+   if (action == BLK_NEEDS_REDO)
+   {
+       Page        page = (Page) BufferGetPage(buffer);
+       OffsetNumber *nowunused;
+       Size        datalen;
+       OffsetNumber *offnum;
+
+       nowunused = (OffsetNumber *) XLogRecGetBlockData(record, 0, &datalen);
+
+       /* Shouldn't be a record unless there's something to do */
+       Assert(xlrec->nunused > 0);
+
+       /* Update all now-unused line pointers */
+       offnum = nowunused;
+       for (int i = 0; i < xlrec->nunused; i++)
+       {
+           OffsetNumber off = *offnum++;
+           ItemId      lp = PageGetItemId(page, off);
+
+           Assert(ItemIdIsDead(lp) && !ItemIdHasStorage(lp));
+           ItemIdSetUnused(lp);
+       }
+
+       /*
+        * Update the page's hint bit about whether it has free pointers
+        */
+       PageSetHasFreeLinePointers(page);
+
+       PageSetLSN(page, lsn);
+       MarkBufferDirty(buffer);
+   }
+
+   if (BufferIsValid(buffer))
+   {
+       Size        freespace = PageGetHeapFreeSpace(BufferGetPage(buffer));
+       RelFileNode rnode;
+
+       XLogRecGetBlockTag(record, 0, &rnode, NULL, &blkno);
+
+       UnlockReleaseBuffer(buffer);
+
+       /*
+        * After vacuuming LP_DEAD items from a page, it's useful to update
+        * the FSM about it, as it may cause the page become target for
+        * insertions later even if vacuum decides not to visit it (which is
+        * possible if gets marked all-visible.)
+        *
+        * Do this regardless of a full-page image being applied, since the
+        * FSM data is not in the page anyway.
+        */
+       XLogRecordPageWithFreeSpace(rnode, blkno, freespace);
+   }
+}
+
 /*
  * Replay XLOG_HEAP2_VISIBLE record.
  *
@@ -9722,15 +9691,15 @@ heap2_redo(XLogReaderState *record)
 
    switch (info & XLOG_HEAP_OPMASK)
    {
-       case XLOG_HEAP2_CLEAN:
-           heap_xlog_clean(record);
+       case XLOG_HEAP2_PRUNE:
+           heap_xlog_prune(record);
+           break;
+       case XLOG_HEAP2_VACUUM:
+           heap_xlog_vacuum(record);
            break;
        case XLOG_HEAP2_FREEZE_PAGE:
            heap_xlog_freeze_page(record);
            break;
-       case XLOG_HEAP2_CLEANUP_INFO:
-           heap_xlog_cleanup_info(record);
-           break;
        case XLOG_HEAP2_VISIBLE:
            heap_xlog_visible(record);
            break;
index 8bb38d6406e5e8e0520aa1d1322a94d2c1f7067a..f75502ca2c04688cfc4189a0b8a1aa2afedabba5 100644 (file)
@@ -182,13 +182,10 @@ heap_page_prune_opt(Relation relation, Buffer buffer)
         */
        if (PageIsFull(page) || PageGetHeapFreeSpace(page) < minfree)
        {
-           TransactionId ignore = InvalidTransactionId;    /* return value not
-                                                            * needed */
-
            /* OK to prune */
            (void) heap_page_prune(relation, buffer, vistest,
                                   limited_xmin, limited_ts,
-                                  true, &ignore, NULL);
+                                  true, NULL);
        }
 
        /* And release buffer lock */
@@ -213,8 +210,6 @@ heap_page_prune_opt(Relation relation, Buffer buffer)
  * send its own new total to pgstats, and we don't want this delta applied
  * on top of that.)
  *
- * Sets latestRemovedXid for caller on return.
- *
  * off_loc is the offset location required by the caller to use in error
  * callback.
  *
@@ -225,7 +220,7 @@ heap_page_prune(Relation relation, Buffer buffer,
                GlobalVisState *vistest,
                TransactionId old_snap_xmin,
                TimestampTz old_snap_ts,
-               bool report_stats, TransactionId *latestRemovedXid,
+               bool report_stats,
                OffsetNumber *off_loc)
 {
    int         ndeleted = 0;
@@ -251,7 +246,7 @@ heap_page_prune(Relation relation, Buffer buffer,
    prstate.old_snap_xmin = old_snap_xmin;
    prstate.old_snap_ts = old_snap_ts;
    prstate.old_snap_used = false;
-   prstate.latestRemovedXid = *latestRemovedXid;
+   prstate.latestRemovedXid = InvalidTransactionId;
    prstate.nredirected = prstate.ndead = prstate.nunused = 0;
    memset(prstate.marked, 0, sizeof(prstate.marked));
 
@@ -318,17 +313,41 @@ heap_page_prune(Relation relation, Buffer buffer,
        MarkBufferDirty(buffer);
 
        /*
-        * Emit a WAL XLOG_HEAP2_CLEAN record showing what we did
+        * Emit a WAL XLOG_HEAP2_PRUNE record showing what we did
         */
        if (RelationNeedsWAL(relation))
        {
+           xl_heap_prune xlrec;
            XLogRecPtr  recptr;
 
-           recptr = log_heap_clean(relation, buffer,
-                                   prstate.redirected, prstate.nredirected,
-                                   prstate.nowdead, prstate.ndead,
-                                   prstate.nowunused, prstate.nunused,
-                                   prstate.latestRemovedXid);
+           xlrec.latestRemovedXid = prstate.latestRemovedXid;
+           xlrec.nredirected = prstate.nredirected;
+           xlrec.ndead = prstate.ndead;
+
+           XLogBeginInsert();
+           XLogRegisterData((char *) &xlrec, SizeOfHeapPrune);
+
+           XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
+
+           /*
+            * The OffsetNumber arrays are not actually in the buffer, but we
+            * pretend that they are.  When XLogInsert stores the whole
+            * buffer, the offset arrays need not be stored too.
+            */
+           if (prstate.nredirected > 0)
+               XLogRegisterBufData(0, (char *) prstate.redirected,
+                                   prstate.nredirected *
+                                   sizeof(OffsetNumber) * 2);
+
+           if (prstate.ndead > 0)
+               XLogRegisterBufData(0, (char *) prstate.nowdead,
+                                   prstate.ndead * sizeof(OffsetNumber));
+
+           if (prstate.nunused > 0)
+               XLogRegisterBufData(0, (char *) prstate.nowunused,
+                                   prstate.nunused * sizeof(OffsetNumber));
+
+           recptr = XLogInsert(RM_HEAP2_ID, XLOG_HEAP2_PRUNE);
 
            PageSetLSN(BufferGetPage(buffer), recptr);
        }
@@ -363,8 +382,6 @@ heap_page_prune(Relation relation, Buffer buffer,
    if (report_stats && ndeleted > prstate.ndead)
        pgstat_update_heap_dead_tuples(relation, ndeleted - prstate.ndead);
 
-   *latestRemovedXid = prstate.latestRemovedXid;
-
    /*
     * XXX Should we update the FSM information of this page ?
     *
@@ -809,12 +826,8 @@ heap_prune_record_unused(PruneState *prstate, OffsetNumber offnum)
 
 /*
  * Perform the actual page changes needed by heap_page_prune.
- * It is expected that the caller has suitable pin and lock on the
- * buffer, and is inside a critical section.
- *
- * This is split out because it is also used by heap_xlog_clean()
- * to replay the WAL record when needed after a crash.  Note that the
- * arguments are identical to those of log_heap_clean().
+ * It is expected that the caller has a super-exclusive lock on the
+ * buffer.
  */
 void
 heap_page_prune_execute(Buffer buffer,
@@ -826,6 +839,9 @@ heap_page_prune_execute(Buffer buffer,
    OffsetNumber *offnum;
    int         i;
 
+   /* Shouldn't be called unless there's something to do */
+   Assert(nredirected > 0 || ndead > 0 || nunused > 0);
+
    /* Update all redirected line pointers */
    offnum = redirected;
    for (i = 0; i < nredirected; i++)
index 84700979c35be90047f134e9a5f444a1f2e4f1b8..446e3bc45233a4953ced2a466c9fbab6b3338444 100644 (file)
@@ -310,7 +310,6 @@ typedef struct LVRelState
    /* rel's initial relfrozenxid and relminmxid */
    TransactionId relfrozenxid;
    MultiXactId relminmxid;
-   TransactionId latestRemovedXid;
 
    /* VACUUM operation's cutoff for pruning */
    TransactionId OldestXmin;
@@ -392,8 +391,7 @@ static void lazy_scan_heap(LVRelState *vacrel, VacuumParams *params,
 static void lazy_scan_prune(LVRelState *vacrel, Buffer buf,
                            BlockNumber blkno, Page page,
                            GlobalVisState *vistest,
-                           LVPagePruneState *prunestate,
-                           VacOptTernaryValue index_cleanup);
+                           LVPagePruneState *prunestate);
 static void lazy_vacuum(LVRelState *vacrel);
 static void lazy_vacuum_all_indexes(LVRelState *vacrel);
 static void lazy_vacuum_heap_rel(LVRelState *vacrel);
@@ -556,7 +554,6 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
    vacrel->old_live_tuples = rel->rd_rel->reltuples;
    vacrel->relfrozenxid = rel->rd_rel->relfrozenxid;
    vacrel->relminmxid = rel->rd_rel->relminmxid;
-   vacrel->latestRemovedXid = InvalidTransactionId;
 
    /* Set cutoffs for entire VACUUM */
    vacrel->OldestXmin = OldestXmin;
@@ -804,40 +801,6 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
    }
 }
 
-/*
- * For Hot Standby we need to know the highest transaction id that will
- * be removed by any change. VACUUM proceeds in a number of passes so
- * we need to consider how each pass operates. The first phase runs
- * heap_page_prune(), which can issue XLOG_HEAP2_CLEAN records as it
- * progresses - these will have a latestRemovedXid on each record.
- * In some cases this removes all of the tuples to be removed, though
- * often we have dead tuples with index pointers so we must remember them
- * for removal in phase 3. Index records for those rows are removed
- * in phase 2 and index blocks do not have MVCC information attached.
- * So before we can allow removal of any index tuples we need to issue
- * a WAL record containing the latestRemovedXid of rows that will be
- * removed in phase three. This allows recovery queries to block at the
- * correct place, i.e. before phase two, rather than during phase three
- * which would be after the rows have become inaccessible.
- */
-static void
-vacuum_log_cleanup_info(LVRelState *vacrel)
-{
-   /*
-    * Skip this for relations for which no WAL is to be written, or if we're
-    * not trying to support archive recovery.
-    */
-   if (!RelationNeedsWAL(vacrel->rel) || !XLogIsNeeded())
-       return;
-
-   /*
-    * No need to write the record at all unless it contains a valid value
-    */
-   if (TransactionIdIsValid(vacrel->latestRemovedXid))
-       (void) log_heap_cleanup_info(vacrel->rel->rd_node,
-                                    vacrel->latestRemovedXid);
-}
-
 /*
  * lazy_scan_heap() -- scan an open heap relation
  *
@@ -1319,8 +1282,7 @@ lazy_scan_heap(LVRelState *vacrel, VacuumParams *params, bool aggressive)
         * were pruned some time earlier.  Also considers freezing XIDs in the
         * tuple headers of remaining items with storage.
         */
-       lazy_scan_prune(vacrel, buf, blkno, page, vistest, &prunestate,
-                       params->index_cleanup);
+       lazy_scan_prune(vacrel, buf, blkno, page, vistest, &prunestate);
 
        /* Remember the location of the last page with nonremovable tuples */
        if (prunestate.hastup)
@@ -1599,6 +1561,21 @@ lazy_scan_heap(LVRelState *vacrel, VacuumParams *params, bool aggressive)
  * lazy_scan_prune() -- lazy_scan_heap() pruning and freezing.
  *
  * Caller must hold pin and buffer cleanup lock on the buffer.
+ *
+ * Prior to PostgreSQL 14 there were very rare cases where heap_page_prune()
+ * was allowed to disagree with our HeapTupleSatisfiesVacuum() call about
+ * whether or not a tuple should be considered DEAD.  This happened when an
+ * inserting transaction concurrently aborted (after our heap_page_prune()
+ * call, before our HeapTupleSatisfiesVacuum() call).  There was rather a lot
+ * of complexity just so we could deal with tuples that were DEAD to VACUUM,
+ * but nevertheless were left with storage after pruning.
+ *
+ * The approach we take now is to restart pruning when the race condition is
+ * detected.  This allows heap_page_prune() to prune the tuples inserted by
+ * the now-aborted transaction.  This is a little crude, but it guarantees
+ * that any items that make it into the dead_tuples array are simple LP_DEAD
+ * line pointers, and that every remaining item with tuple storage is
+ * considered as a candidate for freezing.
  */
 static void
 lazy_scan_prune(LVRelState *vacrel,
@@ -1606,14 +1583,14 @@ lazy_scan_prune(LVRelState *vacrel,
                BlockNumber blkno,
                Page page,
                GlobalVisState *vistest,
-               LVPagePruneState *prunestate,
-               VacOptTernaryValue index_cleanup)
+               LVPagePruneState *prunestate)
 {
    Relation    rel = vacrel->rel;
    OffsetNumber offnum,
                maxoff;
    ItemId      itemid;
    HeapTupleData tuple;
+   HTSV_Result res;
    int         tuples_deleted,
                lpdead_items,
                new_dead_tuples,
@@ -1625,6 +1602,8 @@ lazy_scan_prune(LVRelState *vacrel,
 
    maxoff = PageGetMaxOffsetNumber(page);
 
+retry:
+
    /* Initialize (or reset) page-level counters */
    tuples_deleted = 0;
    lpdead_items = 0;
@@ -1643,7 +1622,6 @@ lazy_scan_prune(LVRelState *vacrel,
     */
    tuples_deleted = heap_page_prune(rel, buf, vistest,
                                     InvalidTransactionId, 0, false,
-                                    &vacrel->latestRemovedXid,
                                     &vacrel->offnum);
 
    /*
@@ -1662,7 +1640,6 @@ lazy_scan_prune(LVRelState *vacrel,
         offnum = OffsetNumberNext(offnum))
    {
        bool        tuple_totally_frozen;
-       bool        tupgone = false;
 
        /*
         * Set the offset number so that we can display it along with any
@@ -1713,6 +1690,17 @@ lazy_scan_prune(LVRelState *vacrel,
        tuple.t_len = ItemIdGetLength(itemid);
        tuple.t_tableOid = RelationGetRelid(rel);
 
+       /*
+        * DEAD tuples are almost always pruned into LP_DEAD line pointers by
+        * heap_page_prune(), but it's possible that the tuple state changed
+        * since heap_page_prune() looked.  Handle that here by restarting.
+        * (See comments at the top of function for a full explanation.)
+        */
+       res = HeapTupleSatisfiesVacuum(&tuple, vacrel->OldestXmin, buf);
+
+       if (unlikely(res == HEAPTUPLE_DEAD))
+           goto retry;
+
        /*
         * The criteria for counting a tuple as live in this block need to
         * match what analyze.c's acquire_sample_rows() does, otherwise VACUUM
@@ -1723,42 +1711,8 @@ lazy_scan_prune(LVRelState *vacrel,
         * VACUUM can't run inside a transaction block, which makes some cases
         * impossible (e.g. in-progress insert from the same transaction).
         */
-       switch (HeapTupleSatisfiesVacuum(&tuple, vacrel->OldestXmin, buf))
+       switch (res)
        {
-           case HEAPTUPLE_DEAD:
-
-               /*
-                * Ordinarily, DEAD tuples would have been removed by
-                * heap_page_prune(), but it's possible that the tuple state
-                * changed since heap_page_prune() looked.  In particular an
-                * INSERT_IN_PROGRESS tuple could have changed to DEAD if the
-                * inserter aborted.  So this cannot be considered an error
-                * condition.
-                *
-                * If the tuple is HOT-updated then it must only be removed by
-                * a prune operation; so we keep it just as if it were
-                * RECENTLY_DEAD.  Also, if it's a heap-only tuple, we choose
-                * to keep it, because it'll be a lot cheaper to get rid of it
-                * in the next pruning pass than to treat it like an indexed
-                * tuple. Finally, if index cleanup is disabled, the second
-                * heap pass will not execute, and the tuple will not get
-                * removed, so we must treat it like any other dead tuple that
-                * we choose to keep.
-                *
-                * If this were to happen for a tuple that actually needed to
-                * be deleted, we'd be in trouble, because it'd possibly leave
-                * a tuple below the relation's xmin horizon alive.
-                * heap_prepare_freeze_tuple() is prepared to detect that case
-                * and abort the transaction, preventing corruption.
-                */
-               if (HeapTupleIsHotUpdated(&tuple) ||
-                   HeapTupleIsHeapOnly(&tuple) ||
-                   index_cleanup == VACOPT_TERNARY_DISABLED)
-                   new_dead_tuples++;
-               else
-                   tupgone = true; /* we can delete the tuple */
-               prunestate->all_visible = false;
-               break;
            case HEAPTUPLE_LIVE:
 
                /*
@@ -1838,46 +1792,32 @@ lazy_scan_prune(LVRelState *vacrel,
                break;
        }
 
-       if (tupgone)
+       /*
+        * Non-removable tuple (i.e. tuple with storage).
+        *
+        * Check tuple left behind after pruning to see if needs to be frozen
+        * now.
+        */
+       num_tuples++;
+       prunestate->hastup = true;
+       if (heap_prepare_freeze_tuple(tuple.t_data,
+                                     vacrel->relfrozenxid,
+                                     vacrel->relminmxid,
+                                     vacrel->FreezeLimit,
+                                     vacrel->MultiXactCutoff,
+                                     &frozen[nfrozen],
+                                     &tuple_totally_frozen))
        {
-           /* Pretend that this is an LP_DEAD item  */
-           deadoffsets[lpdead_items++] = offnum;
-           prunestate->all_visible = false;
-           prunestate->has_lpdead_items = true;
-
-           /* But remember it for XLOG_HEAP2_CLEANUP_INFO record */
-           HeapTupleHeaderAdvanceLatestRemovedXid(tuple.t_data,
-                                                  &vacrel->latestRemovedXid);
+           /* Will execute freeze below */
+           frozen[nfrozen++].offset = offnum;
        }
-       else
-       {
-           /*
-            * Non-removable tuple (i.e. tuple with storage).
-            *
-            * Check tuple left behind after pruning to see if needs to be frozen
-            * now.
-            */
-           num_tuples++;
-           prunestate->hastup = true;
-           if (heap_prepare_freeze_tuple(tuple.t_data,
-                                         vacrel->relfrozenxid,
-                                         vacrel->relminmxid,
-                                         vacrel->FreezeLimit,
-                                         vacrel->MultiXactCutoff,
-                                         &frozen[nfrozen],
-                                         &tuple_totally_frozen))
-           {
-               /* Will execute freeze below */
-               frozen[nfrozen++].offset = offnum;
-           }
 
-           /*
-            * If tuple is not frozen (and not about to become frozen) then caller
-            * had better not go on to set this page's VM bit
-            */
-           if (!tuple_totally_frozen)
-               prunestate->all_frozen = false;
-       }
+       /*
+        * If tuple is not frozen (and not about to become frozen) then caller
+        * had better not go on to set this page's VM bit
+        */
+       if (!tuple_totally_frozen)
+           prunestate->all_frozen = false;
    }
 
    /*
@@ -1888,9 +1828,6 @@ lazy_scan_prune(LVRelState *vacrel,
     *
     * Add page level counters to caller's counts, and then actually process
     * LP_DEAD and LP_NORMAL items.
-    *
-    * TODO: Remove tupgone logic entirely in next commit -- we shouldn't have
-    * to pretend that DEAD items are LP_DEAD items.
     */
    vacrel->offnum = InvalidOffsetNumber;
 
@@ -2053,9 +1990,6 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
    Assert(TransactionIdIsNormal(vacrel->relfrozenxid));
    Assert(MultiXactIdIsValid(vacrel->relminmxid));
 
-   /* Log cleanup info before we touch indexes */
-   vacuum_log_cleanup_info(vacrel);
-
    /* Report that we are now vacuuming indexes */
    pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
                                 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
@@ -2078,6 +2012,14 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
        do_parallel_lazy_vacuum_all_indexes(vacrel);
    }
 
+   /*
+    * We delete all LP_DEAD items from the first heap pass in all indexes on
+    * each call here.  This makes the next call to lazy_vacuum_heap_rel()
+    * safe.
+    */
+   Assert(vacrel->num_index_scans > 0 ||
+          vacrel->dead_tuples->num_tuples == vacrel->lpdead_items);
+
    /* Increase and report the number of index scans */
    vacrel->num_index_scans++;
    pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
@@ -2087,9 +2029,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 /*
  * lazy_vacuum_heap_rel() -- second pass over the heap for two pass strategy
  *
- * This routine marks dead tuples as unused and compacts out free space on
- * their pages.  Pages not having dead tuples recorded from lazy_scan_heap are
- * not visited at all.
+ * This routine marks LP_DEAD items in vacrel->dead_tuples array as LP_UNUSED.
+ * Pages that never had lazy_scan_prune record LP_DEAD items are not visited
+ * at all.
  *
  * Note: the reason for doing this as a second pass is we cannot remove the
  * tuples until we've removed their index entries, and we want to process
@@ -2134,16 +2076,11 @@ lazy_vacuum_heap_rel(LVRelState *vacrel)
        vacrel->blkno = tblk;
        buf = ReadBufferExtended(vacrel->rel, MAIN_FORKNUM, tblk, RBM_NORMAL,
                                 vacrel->bstrategy);
-       if (!ConditionalLockBufferForCleanup(buf))
-       {
-           ReleaseBuffer(buf);
-           ++tupindex;
-           continue;
-       }
+       LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
        tupindex = lazy_vacuum_heap_page(vacrel, tblk, buf, tupindex,
                                         &vmbuffer);
 
-       /* Now that we've compacted the page, record its available space */
+       /* Now that we've vacuumed the page, record its available space */
        page = BufferGetPage(buf);
        freespace = PageGetHeapFreeSpace(page);
 
@@ -2161,6 +2098,14 @@ lazy_vacuum_heap_rel(LVRelState *vacrel)
        vmbuffer = InvalidBuffer;
    }
 
+   /*
+    * We set all LP_DEAD items from the first heap pass to LP_UNUSED during
+    * the second heap pass.  No more, no less.
+    */
+   Assert(vacrel->num_index_scans > 1 ||
+          (tupindex == vacrel->lpdead_items &&
+           vacuumed_pages == vacrel->lpdead_item_pages));
+
    ereport(elevel,
            (errmsg("\"%s\": removed %d dead item identifiers in %u pages",
                    vacrel->relname, tupindex, vacuumed_pages),
@@ -2171,14 +2116,22 @@ lazy_vacuum_heap_rel(LVRelState *vacrel)
 }
 
 /*
- * lazy_vacuum_heap_page() -- free dead tuples on a page
- *                       and repair its fragmentation.
+ * lazy_vacuum_heap_page() -- free page's LP_DEAD items listed in the
+ *                       vacrel->dead_tuples array.
  *
- * Caller must hold pin and buffer cleanup lock on the buffer.
+ * Caller must have an exclusive buffer lock on the buffer (though a
+ * super-exclusive lock is also acceptable).
  *
  * tupindex is the index in vacrel->dead_tuples of the first dead tuple for
  * this page.  We assume the rest follow sequentially.  The return value is
  * the first tupindex after the tuples of this page.
+ *
+ * Prior to PostgreSQL 14 there were rare cases where this routine had to set
+ * tuples with storage to unused.  These days it is strictly responsible for
+ * marking LP_DEAD stub line pointers as unused.  This only happens for those
+ * LP_DEAD items on the page that were determined to be LP_DEAD items back
+ * when the same page was visited by lazy_scan_prune() (i.e. those whose TID
+ * was recorded in the dead_tuples array).
  */
 static int
 lazy_vacuum_heap_page(LVRelState *vacrel, BlockNumber blkno, Buffer buffer,
@@ -2214,11 +2167,15 @@ lazy_vacuum_heap_page(LVRelState *vacrel, BlockNumber blkno, Buffer buffer,
            break;              /* past end of tuples for this block */
        toff = ItemPointerGetOffsetNumber(&dead_tuples->itemptrs[tupindex]);
        itemid = PageGetItemId(page, toff);
+
+       Assert(ItemIdIsDead(itemid) && !ItemIdHasStorage(itemid));
        ItemIdSetUnused(itemid);
        unused[uncnt++] = toff;
    }
 
-   PageRepairFragmentation(page);
+   Assert(uncnt > 0);
+
+   PageSetHasFreeLinePointers(page);
 
    /*
     * Mark buffer dirty before we write WAL.
@@ -2228,12 +2185,19 @@ lazy_vacuum_heap_page(LVRelState *vacrel, BlockNumber blkno, Buffer buffer,
    /* XLOG stuff */
    if (RelationNeedsWAL(vacrel->rel))
    {
+       xl_heap_vacuum xlrec;
        XLogRecPtr  recptr;
 
-       recptr = log_heap_clean(vacrel->rel, buffer,
-                               NULL, 0, NULL, 0,
-                               unused, uncnt,
-                               vacrel->latestRemovedXid);
+       xlrec.nunused = uncnt;
+
+       XLogBeginInsert();
+       XLogRegisterData((char *) &xlrec, SizeOfHeapVacuum);
+
+       XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
+       XLogRegisterBufData(0, (char *) unused, uncnt * sizeof(OffsetNumber));
+
+       recptr = XLogInsert(RM_HEAP2_ID, XLOG_HEAP2_VACUUM);
+
        PageSetLSN(page, recptr);
    }
 
@@ -2246,10 +2210,10 @@ lazy_vacuum_heap_page(LVRelState *vacrel, BlockNumber blkno, Buffer buffer,
    END_CRIT_SECTION();
 
    /*
-    * Now that we have removed the dead tuples from the page, once again
+    * Now that we have removed the LD_DEAD items from the page, once again
     * check if the page has become all-visible.  The page is already marked
     * dirty, exclusively locked, and, if needed, a full page image has been
-    * emitted in the log_heap_clean() above.
+    * emitted.
     */
    if (heap_page_is_all_visible(vacrel, buffer, &visibility_cutoff_xid,
                                 &all_frozen))
index 9282c9ea22fb6963b71e8f940e8cab4a88999015..1360ab80c1e80d8c16d7a16f93d5c1e48166deb6 100644 (file)
@@ -1213,10 +1213,10 @@ backtrack:
                 * as long as the callback function only considers whether the
                 * index tuple refers to pre-cutoff heap tuples that were
                 * certainly already pruned away during VACUUM's initial heap
-                * scan by the time we get here. (heapam's XLOG_HEAP2_CLEAN
-                * and XLOG_HEAP2_CLEANUP_INFO records produce conflicts using
-                * a latestRemovedXid value for the pointed-to heap tuples, so
-                * there is no need to produce our own conflict now.)
+                * scan by the time we get here. (heapam's XLOG_HEAP2_PRUNE
+                * records produce conflicts using a latestRemovedXid value
+                * for the pointed-to heap tuples, so there is no need to
+                * produce our own conflict now.)
                 *
                 * Backends with snapshots acquired after a VACUUM starts but
                 * before it finishes could have visibility cutoff with a
index e60e32b9353de6146bc32089df7819db2aa7e3fb..5c29fd9eae2b3095dff64ab1a72adcfcf0112573 100644 (file)
@@ -121,11 +121,20 @@ heap2_desc(StringInfo buf, XLogReaderState *record)
    uint8       info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
 
    info &= XLOG_HEAP_OPMASK;
-   if (info == XLOG_HEAP2_CLEAN)
+   if (info == XLOG_HEAP2_PRUNE)
    {
-       xl_heap_clean *xlrec = (xl_heap_clean *) rec;
+       xl_heap_prune *xlrec = (xl_heap_prune *) rec;
 
-       appendStringInfo(buf, "latestRemovedXid %u", xlrec->latestRemovedXid);
+       appendStringInfo(buf, "latestRemovedXid %u nredirected %u ndead %u",
+                        xlrec->latestRemovedXid,
+                        xlrec->nredirected,
+                        xlrec->ndead);
+   }
+   else if (info == XLOG_HEAP2_VACUUM)
+   {
+       xl_heap_vacuum *xlrec = (xl_heap_vacuum *) rec;
+
+       appendStringInfo(buf, "nunused %u", xlrec->nunused);
    }
    else if (info == XLOG_HEAP2_FREEZE_PAGE)
    {
@@ -134,12 +143,6 @@ heap2_desc(StringInfo buf, XLogReaderState *record)
        appendStringInfo(buf, "cutoff xid %u ntuples %u",
                         xlrec->cutoff_xid, xlrec->ntuples);
    }
-   else if (info == XLOG_HEAP2_CLEANUP_INFO)
-   {
-       xl_heap_cleanup_info *xlrec = (xl_heap_cleanup_info *) rec;
-
-       appendStringInfo(buf, "latestRemovedXid %u", xlrec->latestRemovedXid);
-   }
    else if (info == XLOG_HEAP2_VISIBLE)
    {
        xl_heap_visible *xlrec = (xl_heap_visible *) rec;
@@ -229,15 +232,15 @@ heap2_identify(uint8 info)
 
    switch (info & ~XLR_INFO_MASK)
    {
-       case XLOG_HEAP2_CLEAN:
-           id = "CLEAN";
+       case XLOG_HEAP2_PRUNE:
+           id = "PRUNE";
+           break;
+       case XLOG_HEAP2_VACUUM:
+           id = "VACUUM";
            break;
        case XLOG_HEAP2_FREEZE_PAGE:
            id = "FREEZE_PAGE";
            break;
-       case XLOG_HEAP2_CLEANUP_INFO:
-           id = "CLEANUP_INFO";
-           break;
        case XLOG_HEAP2_VISIBLE:
            id = "VISIBLE";
            break;
index 97be4b0f23f4bfa56922d857d15545d5274c6983..9aab7136843f557f555c79db4ac1de7c468f0418 100644 (file)
@@ -484,8 +484,8 @@ DecodeHeap2Op(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
             * interested in.
             */
        case XLOG_HEAP2_FREEZE_PAGE:
-       case XLOG_HEAP2_CLEAN:
-       case XLOG_HEAP2_CLEANUP_INFO:
+       case XLOG_HEAP2_PRUNE:
+       case XLOG_HEAP2_VACUUM:
        case XLOG_HEAP2_VISIBLE:
        case XLOG_HEAP2_LOCK_UPDATED:
            break;
index ceb625e13ac2e51e42cdbe200222147730b542f1..e63b49fc385cecb3a60f730f09832265107d453d 100644 (file)
@@ -186,7 +186,7 @@ extern int  heap_page_prune(Relation relation, Buffer buffer,
                            struct GlobalVisState *vistest,
                            TransactionId old_snap_xmin,
                            TimestampTz old_snap_ts_ts,
-                           bool report_stats, TransactionId *latestRemovedXid,
+                           bool report_stats,
                            OffsetNumber *off_loc);
 extern void heap_page_prune_execute(Buffer buffer,
                                    OffsetNumber *redirected, int nredirected,
index 178d49710a553fe37608d3b09c7a5985c9b469f4..27db48184e6f15e4ba6b838e51bf8664b2e5b750 100644 (file)
@@ -51,9 +51,9 @@
  * these, too.
  */
 #define XLOG_HEAP2_REWRITE     0x00
-#define XLOG_HEAP2_CLEAN       0x10
-#define XLOG_HEAP2_FREEZE_PAGE 0x20
-#define XLOG_HEAP2_CLEANUP_INFO 0x30
+#define XLOG_HEAP2_PRUNE       0x10
+#define XLOG_HEAP2_VACUUM      0x20
+#define XLOG_HEAP2_FREEZE_PAGE 0x30
 #define XLOG_HEAP2_VISIBLE     0x40
 #define XLOG_HEAP2_MULTI_INSERT 0x50
 #define XLOG_HEAP2_LOCK_UPDATED 0x60
@@ -227,7 +227,8 @@ typedef struct xl_heap_update
 #define SizeOfHeapUpdate   (offsetof(xl_heap_update, new_offnum) + sizeof(OffsetNumber))
 
 /*
- * This is what we need to know about vacuum page cleanup/redirect
+ * This is what we need to know about page pruning (both during VACUUM and
+ * during opportunistic pruning)
  *
  * The array of OffsetNumbers following the fixed part of the record contains:
  * * for each redirected item: the item offset, then the offset redirected to
@@ -236,29 +237,32 @@ typedef struct xl_heap_update
  * The total number of OffsetNumbers is therefore 2*nredirected+ndead+nunused.
  * Note that nunused is not explicitly stored, but may be found by reference
  * to the total record length.
+ *
+ * Requires a super-exclusive lock.
  */
-typedef struct xl_heap_clean
+typedef struct xl_heap_prune
 {
    TransactionId latestRemovedXid;
    uint16      nredirected;
    uint16      ndead;
    /* OFFSET NUMBERS are in the block reference 0 */
-} xl_heap_clean;
+} xl_heap_prune;
 
-#define SizeOfHeapClean (offsetof(xl_heap_clean, ndead) + sizeof(uint16))
+#define SizeOfHeapPrune (offsetof(xl_heap_prune, ndead) + sizeof(uint16))
 
 /*
- * Cleanup_info is required in some cases during a lazy VACUUM.
- * Used for reporting the results of HeapTupleHeaderAdvanceLatestRemovedXid()
- * see vacuumlazy.c for full explanation
+ * The vacuum page record is similar to the prune record, but can only mark
+ * already dead items as unused
+ *
+ * Used by heap vacuuming only.  Does not require a super-exclusive lock.
  */
-typedef struct xl_heap_cleanup_info
+typedef struct xl_heap_vacuum
 {
-   RelFileNode node;
-   TransactionId latestRemovedXid;
-} xl_heap_cleanup_info;
+   uint16      nunused;
+   /* OFFSET NUMBERS are in the block reference 0 */
+} xl_heap_vacuum;
 
-#define SizeOfHeapCleanupInfo (sizeof(xl_heap_cleanup_info))
+#define SizeOfHeapVacuum (offsetof(xl_heap_vacuum, nunused) + sizeof(uint16))
 
 /* flags for infobits_set */
 #define XLHL_XMAX_IS_MULTI     0x01
@@ -397,13 +401,6 @@ extern void heap2_desc(StringInfo buf, XLogReaderState *record);
 extern const char *heap2_identify(uint8 info);
 extern void heap_xlog_logical_rewrite(XLogReaderState *r);
 
-extern XLogRecPtr log_heap_cleanup_info(RelFileNode rnode,
-                                       TransactionId latestRemovedXid);
-extern XLogRecPtr log_heap_clean(Relation reln, Buffer buffer,
-                                OffsetNumber *redirected, int nredirected,
-                                OffsetNumber *nowdead, int ndead,
-                                OffsetNumber *nowunused, int nunused,
-                                TransactionId latestRemovedXid);
 extern XLogRecPtr log_heap_freeze(Relation reln, Buffer buffer,
                                  TransactionId cutoff_xid, xl_heap_freeze_tuple *tuples,
                                  int ntuples);
index b23e286406c6946f8ac2f194e1092006b5d044d4..da94f2d472a6a1694466e2c0999536d88bfc3148 100644 (file)
@@ -31,7 +31,7 @@
 /*
  * Each page of XLOG file has a header like this:
  */
-#define XLOG_PAGE_MAGIC 0xD10B /* can be used as WAL version indicator */
+#define XLOG_PAGE_MAGIC 0xD10C /* can be used as WAL version indicator */
 
 typedef struct XLogPageHeaderData
 {
index 6a98064b2bdae372ebf512778580f64c94813f92..b26e81dcbf7a3240b20e09ea16b0224a94c695a6 100644 (file)
@@ -3554,8 +3554,6 @@ xl_hash_split_complete
 xl_hash_squeeze_page
 xl_hash_update_meta_page
 xl_hash_vacuum_one_page
-xl_heap_clean
-xl_heap_cleanup_info
 xl_heap_confirm
 xl_heap_delete
 xl_heap_freeze_page
@@ -3567,9 +3565,11 @@ xl_heap_lock
 xl_heap_lock_updated
 xl_heap_multi_insert
 xl_heap_new_cid
+xl_heap_prune
 xl_heap_rewrite_mapping
 xl_heap_truncate
 xl_heap_update
+xl_heap_vacuum
 xl_heap_visible
 xl_invalid_page
 xl_invalid_page_key