*
* 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)
{
* 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)
{
* 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.
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.
}
/*
- * 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;
/*
* 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);
/*
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.)
}
}
+/*
+ * 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.
*
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;
*/
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 */
* 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.
*
GlobalVisState *vistest,
TransactionId old_snap_xmin,
TimestampTz old_snap_ts,
- bool report_stats, TransactionId *latestRemovedXid,
+ bool report_stats,
OffsetNumber *off_loc)
{
int ndeleted = 0;
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));
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);
}
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 ?
*
/*
* 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,
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++)
/* rel's initial relfrozenxid and relminmxid */
TransactionId relfrozenxid;
MultiXactId relminmxid;
- TransactionId latestRemovedXid;
/* VACUUM operation's cutoff for pruning */
TransactionId OldestXmin;
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);
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;
}
}
-/*
- * 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
*
* 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)
* 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,
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,
maxoff = PageGetMaxOffsetNumber(page);
+retry:
+
/* Initialize (or reset) page-level counters */
tuples_deleted = 0;
lpdead_items = 0;
*/
tuples_deleted = heap_page_prune(rel, buf, vistest,
InvalidTransactionId, 0, false,
- &vacrel->latestRemovedXid,
&vacrel->offnum);
/*
offnum = OffsetNumberNext(offnum))
{
bool tuple_totally_frozen;
- bool tupgone = false;
/*
* Set the offset number so that we can display it along with any
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
* 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:
/*
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;
}
/*
*
* 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;
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);
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,
/*
* 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
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);
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),
}
/*
- * 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,
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.
/* 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);
}
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))
* 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
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)
{
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;
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;
* 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;
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,
* 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
#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
* 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
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);
/*
* 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
{
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
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