aboutsummaryrefslogtreecommitdiff
path: root/src/backend/access/heap/vacuumlazy.c
diff options
context:
space:
mode:
Diffstat (limited to 'src/backend/access/heap/vacuumlazy.c')
-rw-r--r--src/backend/access/heap/vacuumlazy.c248
1 files changed, 106 insertions, 142 deletions
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 84700979c35..446e3bc4523 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -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;
@@ -805,40 +802,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
*
* This routine prunes each page in the heap, which will among other
@@ -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
@@ -1714,6 +1691,17 @@ lazy_scan_prune(LVRelState *vacrel,
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
* and ANALYZE may produce wildly different reltuples values, e.g.
@@ -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))