*
* 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_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.
+ * already. XLOG_HEAP2_PRUNE_VACUUM_SCAN 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_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.
+ * that already. XLOG_HEAP2_PRUNE_VACUUM_SCAN 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)
{
static TM_Result heap_lock_updated_tuple(Relation rel, HeapTuple tuple,
ItemPointer ctid, TransactionId xid,
LockTupleMode mode);
-static int heap_log_freeze_plan(HeapTupleFreeze *tuples, int ntuples,
- xl_heap_freeze_plan *plans_out,
- OffsetNumber *offsets_out);
static void GetMultiXactIdHintBits(MultiXactId multi, uint16 *new_infomask,
uint16 *new_infomask2);
static TransactionId MultiXactIdGetUpdateXid(TransactionId xmax,
/* Now WAL-log freezing if necessary */
if (RelationNeedsWAL(rel))
{
- xl_heap_freeze_plan plans[MaxHeapTuplesPerPage];
- OffsetNumber offsets[MaxHeapTuplesPerPage];
- int nplans;
- xl_heap_freeze_page xlrec;
- XLogRecPtr recptr;
-
- /* Prepare deduplicated representation for use in WAL record */
- nplans = heap_log_freeze_plan(tuples, ntuples, plans, offsets);
-
- xlrec.snapshotConflictHorizon = snapshotConflictHorizon;
- xlrec.isCatalogRel = RelationIsAccessibleInLogicalDecoding(rel);
- xlrec.nplans = nplans;
-
- XLogBeginInsert();
- XLogRegisterData((char *) &xlrec, SizeOfHeapFreezePage);
-
- /*
- * The freeze plan array and offset array are not actually in the
- * buffer, but pretend that they are. When XLogInsert stores the
- * whole buffer, the arrays need not be stored too.
- */
- XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
- XLogRegisterBufData(0, (char *) plans,
- nplans * sizeof(xl_heap_freeze_plan));
- XLogRegisterBufData(0, (char *) offsets,
- ntuples * sizeof(OffsetNumber));
-
- recptr = XLogInsert(RM_HEAP2_ID, XLOG_HEAP2_FREEZE_PAGE);
-
- PageSetLSN(page, recptr);
+ log_heap_prune_and_freeze(rel, buffer, snapshotConflictHorizon,
+ false, /* no cleanup lock required */
+ PRUNE_VACUUM_SCAN,
+ tuples, ntuples,
+ NULL, 0, /* redirected */
+ NULL, 0, /* dead */
+ NULL, 0); /* unused */
}
END_CRIT_SECTION();
}
-/*
- * Comparator used to deduplicate XLOG_HEAP2_FREEZE_PAGE freeze plans
- */
-static int
-heap_log_freeze_cmp(const void *arg1, const void *arg2)
-{
- HeapTupleFreeze *frz1 = (HeapTupleFreeze *) arg1;
- HeapTupleFreeze *frz2 = (HeapTupleFreeze *) arg2;
-
- if (frz1->xmax < frz2->xmax)
- return -1;
- else if (frz1->xmax > frz2->xmax)
- return 1;
-
- if (frz1->t_infomask2 < frz2->t_infomask2)
- return -1;
- else if (frz1->t_infomask2 > frz2->t_infomask2)
- return 1;
-
- if (frz1->t_infomask < frz2->t_infomask)
- return -1;
- else if (frz1->t_infomask > frz2->t_infomask)
- return 1;
-
- if (frz1->frzflags < frz2->frzflags)
- return -1;
- else if (frz1->frzflags > frz2->frzflags)
- return 1;
-
- /*
- * heap_log_freeze_eq would consider these tuple-wise plans to be equal.
- * (So the tuples will share a single canonical freeze plan.)
- *
- * We tiebreak on page offset number to keep each freeze plan's page
- * offset number array individually sorted. (Unnecessary, but be tidy.)
- */
- if (frz1->offset < frz2->offset)
- return -1;
- else if (frz1->offset > frz2->offset)
- return 1;
-
- Assert(false);
- return 0;
-}
-
-/*
- * Compare fields that describe actions required to freeze tuple with caller's
- * open plan. If everything matches then the frz tuple plan is equivalent to
- * caller's plan.
- */
-static inline bool
-heap_log_freeze_eq(xl_heap_freeze_plan *plan, HeapTupleFreeze *frz)
-{
- if (plan->xmax == frz->xmax &&
- plan->t_infomask2 == frz->t_infomask2 &&
- plan->t_infomask == frz->t_infomask &&
- plan->frzflags == frz->frzflags)
- return true;
-
- /* Caller must call heap_log_freeze_new_plan again for frz */
- return false;
-}
-
-/*
- * Start new plan initialized using tuple-level actions. At least one tuple
- * will have steps required to freeze described by caller's plan during REDO.
- */
-static inline void
-heap_log_freeze_new_plan(xl_heap_freeze_plan *plan, HeapTupleFreeze *frz)
-{
- plan->xmax = frz->xmax;
- plan->t_infomask2 = frz->t_infomask2;
- plan->t_infomask = frz->t_infomask;
- plan->frzflags = frz->frzflags;
- plan->ntuples = 1; /* for now */
-}
-
-/*
- * Deduplicate tuple-based freeze plans so that each distinct set of
- * processing steps is only stored once in XLOG_HEAP2_FREEZE_PAGE records.
- * Called during original execution of freezing (for logged relations).
- *
- * Return value is number of plans set in *plans_out for caller. Also writes
- * an array of offset numbers into *offsets_out output argument for caller
- * (actually there is one array per freeze plan, but that's not of immediate
- * concern to our caller).
- */
-static int
-heap_log_freeze_plan(HeapTupleFreeze *tuples, int ntuples,
- xl_heap_freeze_plan *plans_out,
- OffsetNumber *offsets_out)
-{
- int nplans = 0;
-
- /* Sort tuple-based freeze plans in the order required to deduplicate */
- qsort(tuples, ntuples, sizeof(HeapTupleFreeze), heap_log_freeze_cmp);
-
- for (int i = 0; i < ntuples; i++)
- {
- HeapTupleFreeze *frz = tuples + i;
-
- if (i == 0)
- {
- /* New canonical freeze plan starting with first tup */
- heap_log_freeze_new_plan(plans_out, frz);
- nplans++;
- }
- else if (heap_log_freeze_eq(plans_out, frz))
- {
- /* tup matches open canonical plan -- include tup in it */
- Assert(offsets_out[i - 1] < frz->offset);
- plans_out->ntuples++;
- }
- else
- {
- /* Tup doesn't match current plan -- done with it now */
- plans_out++;
-
- /* New canonical freeze plan starting with this tup */
- heap_log_freeze_new_plan(plans_out, frz);
- nplans++;
- }
-
- /*
- * Save page offset number in dedicated buffer in passing.
- *
- * REDO routine relies on the record's offset numbers array grouping
- * offset numbers by freeze plan. The sort order within each grouping
- * is ascending offset number order, just to keep things tidy.
- */
- offsets_out[i] = frz->offset;
- }
-
- Assert(nplans > 0 && nplans <= ntuples);
-
- return nplans;
-}
-
/*
* heap_freeze_tuple
* Freeze tuple in place, without WAL logging.
* must have considered the original tuple header as part of
* generating its own snapshotConflictHorizon value.
*
- * 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 snapshotConflictHorizon field of
- * their own for this reason.
+ * Relying on XLOG_HEAP2_PRUNE_VACUUM_SCAN records like this is
+ * the same strategy that index vacuuming uses in all cases. Index
+ * VACUUM WAL records don't even have a snapshotConflictHorizon
+ * field of their own for this reason.
*/
if (!ItemIdIsNormal(lp))
break;
}
/*
- * Handles XLOG_HEAP2_PRUNE record type.
- *
- * Acquires a full cleanup lock.
+ * Replay XLOG_HEAP2_PRUNE_* records.
*/
static void
-heap_xlog_prune(XLogReaderState *record)
+heap_xlog_prune_freeze(XLogReaderState *record)
{
XLogRecPtr lsn = record->EndRecPtr;
- xl_heap_prune *xlrec = (xl_heap_prune *) XLogRecGetData(record);
+ char *maindataptr = XLogRecGetData(record);
+ xl_heap_prune xlrec;
Buffer buffer;
RelFileLocator rlocator;
BlockNumber blkno;
XLogRedoAction action;
XLogRecGetBlockTag(record, 0, &rlocator, NULL, &blkno);
+ memcpy(&xlrec, maindataptr, SizeOfHeapPrune);
+ maindataptr += SizeOfHeapPrune;
/*
- * 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.
+ * We will take an ordinary exclusive lock or a cleanup lock depending on
+ * whether the XLHP_CLEANUP_LOCK flag is set. With an ordinary exclusive
+ * lock, we better not be doing anything that requires moving existing
+ * tuple data.
*/
- if (InHotStandby)
- ResolveRecoveryConflictWithSnapshot(xlrec->snapshotConflictHorizon,
- xlrec->isCatalogRel,
- rlocator);
+ Assert((xlrec.flags & XLHP_CLEANUP_LOCK) != 0 ||
+ (xlrec.flags & (XLHP_HAS_REDIRECTIONS | XLHP_HAS_DEAD_ITEMS)) == 0);
/*
- * If we have a full-page image, restore it (using a cleanup lock) and
- * we're done.
+ * We are about to remove and/or freeze tuples. In Hot Standby mode,
+ * ensure that there are no queries running for which the removed tuples
+ * are still visible or which still consider the frozen xids as running.
+ * The conflict horizon XID comes after xl_heap_prune.
*/
- action = XLogReadBufferForRedoExtended(record, 0, RBM_NORMAL, true,
+ if ((xlrec.flags & XLHP_HAS_CONFLICT_HORIZON) != 0)
+ {
+ TransactionId snapshot_conflict_horizon;
+
+ /* memcpy() because snapshot_conflict_horizon is stored unaligned */
+ memcpy(&snapshot_conflict_horizon, maindataptr, sizeof(TransactionId));
+ maindataptr += sizeof(TransactionId);
+
+ if (InHotStandby)
+ ResolveRecoveryConflictWithSnapshot(snapshot_conflict_horizon,
+ (xlrec.flags & XLHP_IS_CATALOG_REL) != 0,
+ rlocator);
+ }
+
+ /*
+ * If we have a full-page image, restore it and we're done.
+ */
+ action = XLogReadBufferForRedoExtended(record, 0, RBM_NORMAL,
+ (xlrec.flags & XLHP_CLEANUP_LOCK) != 0,
&buffer);
if (action == BLK_NEEDS_REDO)
{
Page page = (Page) BufferGetPage(buffer);
- OffsetNumber *end;
OffsetNumber *redirected;
OffsetNumber *nowdead;
OffsetNumber *nowunused;
int nredirected;
int ndead;
int nunused;
+ int nplans;
Size datalen;
+ xlhp_freeze_plan *plans;
+ OffsetNumber *frz_offsets;
+ char *dataptr = XLogRecGetBlockData(record, 0, &datalen);
- redirected = (OffsetNumber *) XLogRecGetBlockData(record, 0, &datalen);
-
- nredirected = xlrec->nredirected;
- ndead = xlrec->ndead;
- end = (OffsetNumber *) ((char *) redirected + datalen);
- nowdead = redirected + (nredirected * 2);
- nowunused = nowdead + ndead;
- nunused = (end - nowunused);
- Assert(nunused >= 0);
-
- /* Update all line pointers per the record, and repair fragmentation */
- heap_page_prune_execute(buffer,
- redirected, nredirected,
- nowdead, ndead,
- nowunused, nunused);
+ heap_xlog_deserialize_prune_and_freeze(dataptr, xlrec.flags,
+ &nplans, &plans, &frz_offsets,
+ &nredirected, &redirected,
+ &ndead, &nowdead,
+ &nunused, &nowunused);
/*
- * Note: we don't worry about updating the page's prunability hints.
- * At worst this will cause an extra prune cycle to occur soon.
+ * Update all line pointers per the record, and repair fragmentation
+ * if needed.
*/
+ if (nredirected > 0 || ndead > 0 || nunused > 0)
+ heap_page_prune_execute(buffer,
+ (xlrec.flags & XLHP_CLEANUP_LOCK) == 0,
+ redirected, nredirected,
+ nowdead, ndead,
+ nowunused, nunused);
+
+ /* Freeze tuples */
+ for (int p = 0; p < nplans; p++)
+ {
+ HeapTupleFreeze frz;
- PageSetLSN(page, lsn);
- MarkBufferDirty(buffer);
- }
+ /*
+ * Convert freeze plan representation from WAL record into
+ * per-tuple format used by heap_execute_freeze_tuple
+ */
+ frz.xmax = plans[p].xmax;
+ frz.t_infomask2 = plans[p].t_infomask2;
+ frz.t_infomask = plans[p].t_infomask;
+ frz.frzflags = plans[p].frzflags;
+ frz.offset = InvalidOffsetNumber; /* unused, but be tidy */
- if (BufferIsValid(buffer))
- {
- Size freespace = PageGetHeapFreeSpace(BufferGetPage(buffer));
+ for (int i = 0; i < plans[p].ntuples; i++)
+ {
+ OffsetNumber offset = *(frz_offsets++);
+ ItemId lp;
+ HeapTupleHeader tuple;
- UnlockReleaseBuffer(buffer);
+ lp = PageGetItemId(page, offset);
+ tuple = (HeapTupleHeader) PageGetItem(page, lp);
+ heap_execute_freeze_tuple(tuple, &frz);
+ }
+ }
+
+ /* There should be no more data */
+ Assert((char *) frz_offsets == dataptr + datalen);
/*
- * 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.)
- *
- * Do this regardless of a full-page image being applied, since the
- * FSM data is not in the page anyway.
+ * Note: we don't worry about updating the page's prunability hints.
+ * At worst this will cause an extra prune cycle to occur soon.
*/
- XLogRecordPageWithFreeSpace(rlocator, blkno, freespace);
- }
-}
-
-/*
- * Handles XLOG_HEAP2_VACUUM record type.
- *
- * Acquires an ordinary 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);
- }
-
- /* Attempt to truncate line pointer array now */
- PageTruncateLinePointerArray(page);
PageSetLSN(page, lsn);
MarkBufferDirty(buffer);
}
+ /*
+ * If we released any space or line pointers, update the free space map.
+ *
+ * Do this regardless of a full-page image being applied, since the FSM
+ * data is not in the page anyway.
+ */
if (BufferIsValid(buffer))
{
- Size freespace = PageGetHeapFreeSpace(BufferGetPage(buffer));
- RelFileLocator rlocator;
-
- XLogRecGetBlockTag(record, 0, &rlocator, NULL, &blkno);
+ if (xlrec.flags & (XLHP_HAS_REDIRECTIONS |
+ XLHP_HAS_DEAD_ITEMS |
+ XLHP_HAS_NOW_UNUSED_ITEMS))
+ {
+ Size freespace = PageGetHeapFreeSpace(BufferGetPage(buffer));
- UnlockReleaseBuffer(buffer);
+ 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(rlocator, blkno, freespace);
+ XLogRecordPageWithFreeSpace(rlocator, blkno, freespace);
+ }
+ else
+ UnlockReleaseBuffer(buffer);
}
}
UnlockReleaseBuffer(vmbuffer);
}
-/*
- * Replay XLOG_HEAP2_FREEZE_PAGE records
- */
-static void
-heap_xlog_freeze_page(XLogReaderState *record)
-{
- XLogRecPtr lsn = record->EndRecPtr;
- xl_heap_freeze_page *xlrec = (xl_heap_freeze_page *) XLogRecGetData(record);
- Buffer buffer;
-
- /*
- * In Hot Standby mode, ensure that there's no queries running which still
- * consider the frozen xids as running.
- */
- if (InHotStandby)
- {
- RelFileLocator rlocator;
-
- XLogRecGetBlockTag(record, 0, &rlocator, NULL, NULL);
- ResolveRecoveryConflictWithSnapshot(xlrec->snapshotConflictHorizon,
- xlrec->isCatalogRel,
- rlocator);
- }
-
- if (XLogReadBufferForRedo(record, 0, &buffer) == BLK_NEEDS_REDO)
- {
- Page page = BufferGetPage(buffer);
- xl_heap_freeze_plan *plans;
- OffsetNumber *offsets;
- int curoff = 0;
-
- plans = (xl_heap_freeze_plan *) XLogRecGetBlockData(record, 0, NULL);
- offsets = (OffsetNumber *) ((char *) plans +
- (xlrec->nplans *
- sizeof(xl_heap_freeze_plan)));
- for (int p = 0; p < xlrec->nplans; p++)
- {
- HeapTupleFreeze frz;
-
- /*
- * Convert freeze plan representation from WAL record into
- * per-tuple format used by heap_execute_freeze_tuple
- */
- frz.xmax = plans[p].xmax;
- frz.t_infomask2 = plans[p].t_infomask2;
- frz.t_infomask = plans[p].t_infomask;
- frz.frzflags = plans[p].frzflags;
- frz.offset = InvalidOffsetNumber; /* unused, but be tidy */
-
- for (int i = 0; i < plans[p].ntuples; i++)
- {
- OffsetNumber offset = offsets[curoff++];
- ItemId lp;
- HeapTupleHeader tuple;
-
- lp = PageGetItemId(page, offset);
- tuple = (HeapTupleHeader) PageGetItem(page, lp);
- heap_execute_freeze_tuple(tuple, &frz);
- }
- }
-
- PageSetLSN(page, lsn);
- MarkBufferDirty(buffer);
- }
- if (BufferIsValid(buffer))
- UnlockReleaseBuffer(buffer);
-}
-
/*
* Given an "infobits" field from an XLog record, set the correct bits in the
* given infomask and infomask2 for the tuple touched by the record.
switch (info & XLOG_HEAP_OPMASK)
{
- 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);
+ case XLOG_HEAP2_PRUNE_ON_ACCESS:
+ case XLOG_HEAP2_PRUNE_VACUUM_SCAN:
+ case XLOG_HEAP2_PRUNE_VACUUM_CLEANUP:
+ heap_xlog_prune_freeze(record);
break;
case XLOG_HEAP2_VISIBLE:
heap_xlog_visible(record);
* that during on-access pruning with the current implementation.
*/
heap_page_prune(relation, buffer, vistest, false,
- &presult, NULL);
+ &presult, PRUNE_ON_ACCESS, NULL);
/*
* Report the number of tuples reclaimed to pgstats. This is
* tuples removed and the number of line pointers newly marked LP_DEAD.
* heap_page_prune() is responsible for initializing it.
*
+ * reason indicates why the pruning is performed. It is included in the WAL
+ * record for debugging and analysis purposes, but otherwise has no effect.
+ *
* off_loc is the offset location required by the caller to use in error
* callback.
*/
GlobalVisState *vistest,
bool mark_unused_now,
PruneResult *presult,
+ PruneReason reason,
OffsetNumber *off_loc)
{
Page page = BufferGetPage(buffer);
* Apply the planned item changes, then repair page fragmentation, and
* update the page's hint bit about whether it has free line pointers.
*/
- heap_page_prune_execute(buffer,
+ heap_page_prune_execute(buffer, false,
prstate.redirected, prstate.nredirected,
prstate.nowdead, prstate.ndead,
prstate.nowunused, prstate.nunused);
MarkBufferDirty(buffer);
/*
- * Emit a WAL XLOG_HEAP2_PRUNE record showing what we did
+ * Emit a WAL XLOG_HEAP2_PRUNE_FREEZE record showing what we did
*/
if (RelationNeedsWAL(relation))
{
- xl_heap_prune xlrec;
- XLogRecPtr recptr;
-
- xlrec.isCatalogRel = RelationIsAccessibleInLogicalDecoding(relation);
- xlrec.snapshotConflictHorizon = prstate.snapshotConflictHorizon;
- 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);
+ log_heap_prune_and_freeze(relation, buffer,
+ prstate.snapshotConflictHorizon,
+ true, reason,
+ NULL, 0,
+ prstate.redirected, prstate.nredirected,
+ prstate.nowdead, prstate.ndead,
+ prstate.nowunused, prstate.nunused);
}
}
else
/*
* Perform the actual page changes needed by heap_page_prune.
- * It is expected that the caller has a full cleanup lock on the
- * buffer.
+ *
+ * If 'lp_truncate_only' is set, we are merely marking LP_DEAD line pointers
+ * as unused, not redirecting or removing anything else. The
+ * PageRepairFragmentation() call is skipped in that case.
+ *
+ * If 'lp_truncate_only' is not set, the caller must hold a cleanup lock on
+ * the buffer. If it is set, an ordinary exclusive lock suffices.
*/
void
-heap_page_prune_execute(Buffer buffer,
+heap_page_prune_execute(Buffer buffer, bool lp_truncate_only,
OffsetNumber *redirected, int nredirected,
OffsetNumber *nowdead, int ndead,
OffsetNumber *nowunused, int nunused)
/* Shouldn't be called unless there's something to do */
Assert(nredirected > 0 || ndead > 0 || nunused > 0);
+ /* If 'lp_truncate_only', we can only remove already-dead line pointers */
+ Assert(!lp_truncate_only || (nredirected == 0 && ndead == 0));
+
/* Update all redirected line pointers */
offnum = redirected;
for (int i = 0; i < nredirected; i++)
#ifdef USE_ASSERT_CHECKING
- /*
- * When heap_page_prune() was called, mark_unused_now may have been
- * passed as true, which allows would-be LP_DEAD items to be made
- * LP_UNUSED instead. This is only possible if the relation has no
- * indexes. If there are any dead items, then mark_unused_now was not
- * true and every item being marked LP_UNUSED must refer to a
- * heap-only tuple.
- */
- if (ndead > 0)
+ if (lp_truncate_only)
{
- Assert(ItemIdHasStorage(lp) && ItemIdIsNormal(lp));
- htup = (HeapTupleHeader) PageGetItem(page, lp);
- Assert(HeapTupleHeaderIsHeapOnly(htup));
+ /* Setting LP_DEAD to LP_UNUSED in vacuum's second pass */
+ Assert(ItemIdIsDead(lp) && !ItemIdHasStorage(lp));
}
else
{
- Assert(ItemIdIsUsed(lp));
+ /*
+ * When heap_page_prune() was called, mark_unused_now may have
+ * been passed as true, which allows would-be LP_DEAD items to be
+ * made LP_UNUSED instead. This is only possible if the relation
+ * has no indexes. If there are any dead items, then
+ * mark_unused_now was not true and every item being marked
+ * LP_UNUSED must refer to a heap-only tuple.
+ */
+ if (ndead > 0)
+ {
+ Assert(ItemIdHasStorage(lp) && ItemIdIsNormal(lp));
+ htup = (HeapTupleHeader) PageGetItem(page, lp);
+ Assert(HeapTupleHeaderIsHeapOnly(htup));
+ }
+ else
+ Assert(ItemIdIsUsed(lp));
}
#endif
ItemIdSetUnused(lp);
}
- /*
- * Finally, repair any fragmentation, and update the page's hint bit about
- * whether it has free pointers.
- */
- PageRepairFragmentation(page);
+ if (lp_truncate_only)
+ PageTruncateLinePointerArray(page);
+ else
+ {
+ /*
+ * Finally, repair any fragmentation, and update the page's hint bit
+ * about whether it has free pointers.
+ */
+ PageRepairFragmentation(page);
- /*
- * Now that the page has been modified, assert that redirect items still
- * point to valid targets.
- */
- page_verify_redirects(page);
+ /*
+ * Now that the page has been modified, assert that redirect items
+ * still point to valid targets.
+ */
+ page_verify_redirects(page);
+ }
}
}
}
}
+
+
+/*
+ * Compare fields that describe actions required to freeze tuple with caller's
+ * open plan. If everything matches then the frz tuple plan is equivalent to
+ * caller's plan.
+ */
+static inline bool
+heap_log_freeze_eq(xlhp_freeze_plan *plan, HeapTupleFreeze *frz)
+{
+ if (plan->xmax == frz->xmax &&
+ plan->t_infomask2 == frz->t_infomask2 &&
+ plan->t_infomask == frz->t_infomask &&
+ plan->frzflags == frz->frzflags)
+ return true;
+
+ /* Caller must call heap_log_freeze_new_plan again for frz */
+ return false;
+}
+
+/*
+ * Comparator used to deduplicate XLOG_HEAP2_FREEZE_PAGE freeze plans
+ */
+static int
+heap_log_freeze_cmp(const void *arg1, const void *arg2)
+{
+ HeapTupleFreeze *frz1 = (HeapTupleFreeze *) arg1;
+ HeapTupleFreeze *frz2 = (HeapTupleFreeze *) arg2;
+
+ if (frz1->xmax < frz2->xmax)
+ return -1;
+ else if (frz1->xmax > frz2->xmax)
+ return 1;
+
+ if (frz1->t_infomask2 < frz2->t_infomask2)
+ return -1;
+ else if (frz1->t_infomask2 > frz2->t_infomask2)
+ return 1;
+
+ if (frz1->t_infomask < frz2->t_infomask)
+ return -1;
+ else if (frz1->t_infomask > frz2->t_infomask)
+ return 1;
+
+ if (frz1->frzflags < frz2->frzflags)
+ return -1;
+ else if (frz1->frzflags > frz2->frzflags)
+ return 1;
+
+ /*
+ * heap_log_freeze_eq would consider these tuple-wise plans to be equal.
+ * (So the tuples will share a single canonical freeze plan.)
+ *
+ * We tiebreak on page offset number to keep each freeze plan's page
+ * offset number array individually sorted. (Unnecessary, but be tidy.)
+ */
+ if (frz1->offset < frz2->offset)
+ return -1;
+ else if (frz1->offset > frz2->offset)
+ return 1;
+
+ Assert(false);
+ return 0;
+}
+
+/*
+ * Start new plan initialized using tuple-level actions. At least one tuple
+ * will have steps required to freeze described by caller's plan during REDO.
+ */
+static inline void
+heap_log_freeze_new_plan(xlhp_freeze_plan *plan, HeapTupleFreeze *frz)
+{
+ plan->xmax = frz->xmax;
+ plan->t_infomask2 = frz->t_infomask2;
+ plan->t_infomask = frz->t_infomask;
+ plan->frzflags = frz->frzflags;
+ plan->ntuples = 1; /* for now */
+}
+
+/*
+ * Deduplicate tuple-based freeze plans so that each distinct set of
+ * processing steps is only stored once in XLOG_HEAP2_FREEZE_PAGE records.
+ * Called during original execution of freezing (for logged relations).
+ *
+ * Return value is number of plans set in *plans_out for caller. Also writes
+ * an array of offset numbers into *offsets_out output argument for caller
+ * (actually there is one array per freeze plan, but that's not of immediate
+ * concern to our caller).
+ */
+static int
+heap_log_freeze_plan(HeapTupleFreeze *tuples, int ntuples,
+ xlhp_freeze_plan *plans_out,
+ OffsetNumber *offsets_out)
+{
+ int nplans = 0;
+
+ /* Sort tuple-based freeze plans in the order required to deduplicate */
+ qsort(tuples, ntuples, sizeof(HeapTupleFreeze), heap_log_freeze_cmp);
+
+ for (int i = 0; i < ntuples; i++)
+ {
+ HeapTupleFreeze *frz = tuples + i;
+
+ if (i == 0)
+ {
+ /* New canonical freeze plan starting with first tup */
+ heap_log_freeze_new_plan(plans_out, frz);
+ nplans++;
+ }
+ else if (heap_log_freeze_eq(plans_out, frz))
+ {
+ /* tup matches open canonical plan -- include tup in it */
+ Assert(offsets_out[i - 1] < frz->offset);
+ plans_out->ntuples++;
+ }
+ else
+ {
+ /* Tup doesn't match current plan -- done with it now */
+ plans_out++;
+
+ /* New canonical freeze plan starting with this tup */
+ heap_log_freeze_new_plan(plans_out, frz);
+ nplans++;
+ }
+
+ /*
+ * Save page offset number in dedicated buffer in passing.
+ *
+ * REDO routine relies on the record's offset numbers array grouping
+ * offset numbers by freeze plan. The sort order within each grouping
+ * is ascending offset number order, just to keep things tidy.
+ */
+ offsets_out[i] = frz->offset;
+ }
+
+ Assert(nplans > 0 && nplans <= ntuples);
+
+ return nplans;
+}
+
+/*
+ * Write an XLOG_HEAP2_PRUNE_FREEZE WAL record
+ *
+ * This is used for several different page maintenance operations:
+ *
+ * - Page pruning, in VACUUM's 1st pass or on access: Some items are
+ * redirected, some marked dead, and some removed altogether.
+ *
+ * - Freezing: Items are marked as 'frozen'.
+ *
+ * - Vacuum, 2nd pass: Items that are already LP_DEAD are marked as unused.
+ *
+ * They have enough commonalities that we use a single WAL record for them
+ * all.
+ *
+ * If replaying the record requires a cleanup lock, pass cleanup_lock = true.
+ * Replaying 'redirected' or 'dead' items always requires a cleanup lock, but
+ * replaying 'unused' items depends on whether they were all previously marked
+ * as dead.
+ *
+ * Note: This function scribbles on the 'frozen' array.
+ *
+ * Note: This is called in a critical section, so careful what you do here.
+ */
+void
+log_heap_prune_and_freeze(Relation relation, Buffer buffer,
+ TransactionId conflict_xid,
+ bool cleanup_lock,
+ PruneReason reason,
+ HeapTupleFreeze *frozen, int nfrozen,
+ OffsetNumber *redirected, int nredirected,
+ OffsetNumber *dead, int ndead,
+ OffsetNumber *unused, int nunused)
+{
+ xl_heap_prune xlrec;
+ XLogRecPtr recptr;
+ uint8 info;
+
+ /* The following local variables hold data registered in the WAL record: */
+ xlhp_freeze_plan plans[MaxHeapTuplesPerPage];
+ xlhp_freeze_plans freeze_plans;
+ xlhp_prune_items redirect_items;
+ xlhp_prune_items dead_items;
+ xlhp_prune_items unused_items;
+ OffsetNumber frz_offsets[MaxHeapTuplesPerPage];
+
+ xlrec.flags = 0;
+
+ /*
+ * Prepare data for the buffer. The arrays are not actually in the
+ * buffer, but we pretend that they are. When XLogInsert stores a full
+ * page image, the arrays can be omitted.
+ */
+ XLogBeginInsert();
+ XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
+ if (nfrozen > 0)
+ {
+ int nplans;
+
+ xlrec.flags |= XLHP_HAS_FREEZE_PLANS;
+
+ /*
+ * Prepare deduplicated representation for use in the WAL record. This
+ * destructively sorts frozen tuples array in-place.
+ */
+ nplans = heap_log_freeze_plan(frozen, nfrozen, plans, frz_offsets);
+
+ freeze_plans.nplans = nplans;
+ XLogRegisterBufData(0, (char *) &freeze_plans,
+ offsetof(xlhp_freeze_plans, plans));
+ XLogRegisterBufData(0, (char *) plans,
+ sizeof(xlhp_freeze_plan) * nplans);
+ }
+ if (nredirected > 0)
+ {
+ xlrec.flags |= XLHP_HAS_REDIRECTIONS;
+
+ redirect_items.ntargets = nredirected;
+ XLogRegisterBufData(0, (char *) &redirect_items,
+ offsetof(xlhp_prune_items, data));
+ XLogRegisterBufData(0, (char *) redirected,
+ sizeof(OffsetNumber[2]) * nredirected);
+ }
+ if (ndead > 0)
+ {
+ xlrec.flags |= XLHP_HAS_DEAD_ITEMS;
+
+ dead_items.ntargets = ndead;
+ XLogRegisterBufData(0, (char *) &dead_items,
+ offsetof(xlhp_prune_items, data));
+ XLogRegisterBufData(0, (char *) dead,
+ sizeof(OffsetNumber) * ndead);
+ }
+ if (nunused > 0)
+ {
+ xlrec.flags |= XLHP_HAS_NOW_UNUSED_ITEMS;
+
+ unused_items.ntargets = nunused;
+ XLogRegisterBufData(0, (char *) &unused_items,
+ offsetof(xlhp_prune_items, data));
+ XLogRegisterBufData(0, (char *) unused,
+ sizeof(OffsetNumber) * nunused);
+ }
+ if (nfrozen > 0)
+ XLogRegisterBufData(0, (char *) frz_offsets,
+ sizeof(OffsetNumber) * nfrozen);
+
+ /*
+ * Prepare the main xl_heap_prune record. We already set the XLPH_HAS_*
+ * flag above.
+ */
+ if (RelationIsAccessibleInLogicalDecoding(relation))
+ xlrec.flags |= XLHP_IS_CATALOG_REL;
+ if (TransactionIdIsValid(conflict_xid))
+ xlrec.flags |= XLHP_HAS_CONFLICT_HORIZON;
+ if (cleanup_lock)
+ xlrec.flags |= XLHP_CLEANUP_LOCK;
+ else
+ {
+ Assert(nredirected == 0 && ndead == 0);
+ /* also, any items in 'unused' must've been LP_DEAD previously */
+ }
+ XLogRegisterData((char *) &xlrec, SizeOfHeapPrune);
+ if (TransactionIdIsValid(conflict_xid))
+ XLogRegisterData((char *) &conflict_xid, sizeof(TransactionId));
+
+ switch (reason)
+ {
+ case PRUNE_ON_ACCESS:
+ info = XLOG_HEAP2_PRUNE_ON_ACCESS;
+ break;
+ case PRUNE_VACUUM_SCAN:
+ info = XLOG_HEAP2_PRUNE_VACUUM_SCAN;
+ break;
+ case PRUNE_VACUUM_CLEANUP:
+ info = XLOG_HEAP2_PRUNE_VACUUM_CLEANUP;
+ break;
+ default:
+ elog(ERROR, "unrecognized prune reason: %d", (int) reason);
+ break;
+ }
+ recptr = XLogInsert(RM_HEAP2_ID, info);
+
+ PageSetLSN(BufferGetPage(buffer), recptr);
+}
* false otherwise.
*/
heap_page_prune(rel, buf, vacrel->vistest, vacrel->nindexes == 0,
- &presult, &vacrel->offnum);
+ &presult, PRUNE_VACUUM_SCAN, &vacrel->offnum);
/*
* We will update the VM after collecting LP_DEAD items and freezing
/* XLOG stuff */
if (RelationNeedsWAL(vacrel->rel))
{
- xl_heap_vacuum xlrec;
- XLogRecPtr recptr;
-
- xlrec.nunused = nunused;
-
- XLogBeginInsert();
- XLogRegisterData((char *) &xlrec, SizeOfHeapVacuum);
-
- XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
- XLogRegisterBufData(0, (char *) unused, nunused * sizeof(OffsetNumber));
-
- recptr = XLogInsert(RM_HEAP2_ID, XLOG_HEAP2_VACUUM);
-
- PageSetLSN(page, recptr);
+ log_heap_prune_and_freeze(vacrel->rel, buffer,
+ InvalidTransactionId,
+ false, /* no cleanup lock required */
+ PRUNE_VACUUM_CLEANUP,
+ NULL, 0, /* frozen */
+ NULL, 0, /* redirected */
+ NULL, 0, /* dead */
+ unused, nunused);
}
/*
static void
plan_elem_desc(StringInfo buf, void *plan, void *data)
{
- xl_heap_freeze_plan *new_plan = (xl_heap_freeze_plan *) plan;
+ xlhp_freeze_plan *new_plan = (xlhp_freeze_plan *) plan;
OffsetNumber **offsets = data;
appendStringInfo(buf, "{ xmax: %u, infomask: %u, infomask2: %u, ntuples: %u",
appendStringInfoString(buf, " }");
}
+
+/*
+ * Given a MAXALIGNed buffer returned by XLogRecGetBlockData() and pointed to
+ * by cursor and any xl_heap_prune flags, deserialize the arrays of
+ * OffsetNumbers contained in an XLOG_HEAP2_PRUNE_* record.
+ *
+ * This is in heapdesc.c so it can be shared between heap2_redo and heap2_desc
+ * code, the latter of which is used in frontend (pg_waldump) code.
+ */
+void
+heap_xlog_deserialize_prune_and_freeze(char *cursor, uint8 flags,
+ int *nplans, xlhp_freeze_plan **plans,
+ OffsetNumber **frz_offsets,
+ int *nredirected, OffsetNumber **redirected,
+ int *ndead, OffsetNumber **nowdead,
+ int *nunused, OffsetNumber **nowunused)
+{
+ if (flags & XLHP_HAS_FREEZE_PLANS)
+ {
+ xlhp_freeze_plans *freeze_plans = (xlhp_freeze_plans *) cursor;
+
+ *nplans = freeze_plans->nplans;
+ Assert(*nplans > 0);
+ *plans = freeze_plans->plans;
+
+ cursor += offsetof(xlhp_freeze_plans, plans);
+ cursor += sizeof(xlhp_freeze_plan) * *nplans;
+ }
+ else
+ {
+ *nplans = 0;
+ *plans = NULL;
+ }
+
+ if (flags & XLHP_HAS_REDIRECTIONS)
+ {
+ xlhp_prune_items *subrecord = (xlhp_prune_items *) cursor;
+
+ *nredirected = subrecord->ntargets;
+ Assert(*nredirected > 0);
+ *redirected = &subrecord->data[0];
+
+ cursor += offsetof(xlhp_prune_items, data);
+ cursor += sizeof(OffsetNumber[2]) * *nredirected;
+ }
+ else
+ {
+ *nredirected = 0;
+ *redirected = NULL;
+ }
+
+ if (flags & XLHP_HAS_DEAD_ITEMS)
+ {
+ xlhp_prune_items *subrecord = (xlhp_prune_items *) cursor;
+
+ *ndead = subrecord->ntargets;
+ Assert(*ndead > 0);
+ *nowdead = subrecord->data;
+
+ cursor += offsetof(xlhp_prune_items, data);
+ cursor += sizeof(OffsetNumber) * *ndead;
+ }
+ else
+ {
+ *ndead = 0;
+ *nowdead = NULL;
+ }
+
+ if (flags & XLHP_HAS_NOW_UNUSED_ITEMS)
+ {
+ xlhp_prune_items *subrecord = (xlhp_prune_items *) cursor;
+
+ *nunused = subrecord->ntargets;
+ Assert(*nunused > 0);
+ *nowunused = subrecord->data;
+
+ cursor += offsetof(xlhp_prune_items, data);
+ cursor += sizeof(OffsetNumber) * *nunused;
+ }
+ else
+ {
+ *nunused = 0;
+ *nowunused = NULL;
+ }
+
+ *frz_offsets = (OffsetNumber *) cursor;
+}
+
void
heap_desc(StringInfo buf, XLogReaderState *record)
{
uint8 info = XLogRecGetInfo(record) & ~XLR_INFO_MASK;
info &= XLOG_HEAP_OPMASK;
- if (info == XLOG_HEAP2_PRUNE)
+ if (info == XLOG_HEAP2_PRUNE_ON_ACCESS ||
+ info == XLOG_HEAP2_PRUNE_VACUUM_SCAN ||
+ info == XLOG_HEAP2_PRUNE_VACUUM_CLEANUP)
{
xl_heap_prune *xlrec = (xl_heap_prune *) rec;
- appendStringInfo(buf, "snapshotConflictHorizon: %u, nredirected: %u, ndead: %u, isCatalogRel: %c",
- xlrec->snapshotConflictHorizon,
- xlrec->nredirected,
- xlrec->ndead,
- xlrec->isCatalogRel ? 'T' : 'F');
-
- if (XLogRecHasBlockData(record, 0))
- {
- OffsetNumber *end;
- OffsetNumber *redirected;
- OffsetNumber *nowdead;
- OffsetNumber *nowunused;
- int nredirected;
- int nunused;
- Size datalen;
-
- redirected = (OffsetNumber *) XLogRecGetBlockData(record, 0,
- &datalen);
-
- nredirected = xlrec->nredirected;
- end = (OffsetNumber *) ((char *) redirected + datalen);
- nowdead = redirected + (nredirected * 2);
- nowunused = nowdead + xlrec->ndead;
- nunused = (end - nowunused);
- Assert(nunused >= 0);
-
- appendStringInfo(buf, ", nunused: %d", nunused);
-
- appendStringInfoString(buf, ", redirected:");
- array_desc(buf, redirected, sizeof(OffsetNumber) * 2,
- nredirected, &redirect_elem_desc, NULL);
- appendStringInfoString(buf, ", dead:");
- array_desc(buf, nowdead, sizeof(OffsetNumber), xlrec->ndead,
- &offset_elem_desc, NULL);
- appendStringInfoString(buf, ", unused:");
- array_desc(buf, nowunused, sizeof(OffsetNumber), nunused,
- &offset_elem_desc, NULL);
- }
- }
- else if (info == XLOG_HEAP2_VACUUM)
- {
- xl_heap_vacuum *xlrec = (xl_heap_vacuum *) rec;
-
- appendStringInfo(buf, "nunused: %u", xlrec->nunused);
-
- if (XLogRecHasBlockData(record, 0))
+ if (xlrec->flags & XLHP_HAS_CONFLICT_HORIZON)
{
- OffsetNumber *nowunused;
+ TransactionId conflict_xid;
- nowunused = (OffsetNumber *) XLogRecGetBlockData(record, 0, NULL);
+ memcpy(&conflict_xid, rec + SizeOfHeapPrune, sizeof(TransactionId));
- appendStringInfoString(buf, ", unused:");
- array_desc(buf, nowunused, sizeof(OffsetNumber), xlrec->nunused,
- &offset_elem_desc, NULL);
+ appendStringInfo(buf, "snapshotConflictHorizon: %u",
+ conflict_xid);
}
- }
- else if (info == XLOG_HEAP2_FREEZE_PAGE)
- {
- xl_heap_freeze_page *xlrec = (xl_heap_freeze_page *) rec;
- appendStringInfo(buf, "snapshotConflictHorizon: %u, nplans: %u, isCatalogRel: %c",
- xlrec->snapshotConflictHorizon, xlrec->nplans,
- xlrec->isCatalogRel ? 'T' : 'F');
+ appendStringInfo(buf, ", isCatalogRel: %c",
+ xlrec->flags & XLHP_IS_CATALOG_REL ? 'T' : 'F');
if (XLogRecHasBlockData(record, 0))
{
- xl_heap_freeze_plan *plans;
- OffsetNumber *offsets;
-
- plans = (xl_heap_freeze_plan *) XLogRecGetBlockData(record, 0, NULL);
- offsets = (OffsetNumber *) ((char *) plans +
- (xlrec->nplans *
- sizeof(xl_heap_freeze_plan)));
- appendStringInfoString(buf, ", plans:");
- array_desc(buf, plans, sizeof(xl_heap_freeze_plan), xlrec->nplans,
- &plan_elem_desc, &offsets);
+ Size datalen;
+ OffsetNumber *redirected;
+ OffsetNumber *nowdead;
+ OffsetNumber *nowunused;
+ int nredirected;
+ int nunused;
+ int ndead;
+ int nplans;
+ xlhp_freeze_plan *plans;
+ OffsetNumber *frz_offsets;
+
+ char *cursor = XLogRecGetBlockData(record, 0, &datalen);
+
+ heap_xlog_deserialize_prune_and_freeze(cursor, xlrec->flags,
+ &nplans, &plans, &frz_offsets,
+ &nredirected, &redirected,
+ &ndead, &nowdead,
+ &nunused, &nowunused);
+
+ appendStringInfo(buf, ", nplans: %u, nredirected: %u, ndead: %u, nunused: %u",
+ nplans, nredirected, ndead, nunused);
+
+ if (nplans > 0)
+ {
+ appendStringInfoString(buf, ", plans:");
+ array_desc(buf, plans, sizeof(xlhp_freeze_plan), nplans,
+ &plan_elem_desc, &frz_offsets);
+ }
+
+ if (nredirected > 0)
+ {
+ appendStringInfoString(buf, ", redirected:");
+ array_desc(buf, redirected, sizeof(OffsetNumber) * 2,
+ nredirected, &redirect_elem_desc, NULL);
+ }
+
+ if (ndead > 0)
+ {
+ appendStringInfoString(buf, ", dead:");
+ array_desc(buf, nowdead, sizeof(OffsetNumber), ndead,
+ &offset_elem_desc, NULL);
+ }
+
+ if (nunused > 0)
+ {
+ appendStringInfoString(buf, ", unused:");
+ array_desc(buf, nowunused, sizeof(OffsetNumber), nunused,
+ &offset_elem_desc, NULL);
+ }
}
}
else if (info == XLOG_HEAP2_VISIBLE)
switch (info & ~XLR_INFO_MASK)
{
- case XLOG_HEAP2_PRUNE:
- id = "PRUNE";
+ case XLOG_HEAP2_PRUNE_ON_ACCESS:
+ id = "PRUNE_ON_ACCESS";
break;
- case XLOG_HEAP2_VACUUM:
- id = "VACUUM";
+ case XLOG_HEAP2_PRUNE_VACUUM_SCAN:
+ id = "PRUNE_VACUUM_SCAN";
break;
- case XLOG_HEAP2_FREEZE_PAGE:
- id = "FREEZE_PAGE";
+ case XLOG_HEAP2_PRUNE_VACUUM_CLEANUP:
+ id = "PRUNE_VACUUM_CLEANUP";
break;
case XLOG_HEAP2_VISIBLE:
id = "VISIBLE";
* Everything else here is just low level physical stuff we're not
* interested in.
*/
- case XLOG_HEAP2_FREEZE_PAGE:
- case XLOG_HEAP2_PRUNE:
- case XLOG_HEAP2_VACUUM:
+ case XLOG_HEAP2_PRUNE_ON_ACCESS:
+ case XLOG_HEAP2_PRUNE_VACUUM_SCAN:
+ case XLOG_HEAP2_PRUNE_VACUUM_CLEANUP:
case XLOG_HEAP2_VISIBLE:
case XLOG_HEAP2_LOCK_UPDATED:
break;
int8 htsv[MaxHeapTuplesPerPage + 1];
} PruneResult;
+/* 'reason' codes for heap_page_prune() */
+typedef enum
+{
+ PRUNE_ON_ACCESS, /* on-access pruning */
+ PRUNE_VACUUM_SCAN, /* VACUUM 1st heap pass */
+ PRUNE_VACUUM_CLEANUP, /* VACUUM 2nd heap pass */
+} PruneReason;
+
/*
* Pruning calculates tuple visibility once and saves the results in an array
* of int8. See PruneResult.htsv for details. This helper function is meant to
struct GlobalVisState *vistest,
bool mark_unused_now,
PruneResult *presult,
+ PruneReason reason,
OffsetNumber *off_loc);
-extern void heap_page_prune_execute(Buffer buffer,
+extern void heap_page_prune_execute(Buffer buffer, bool lp_truncate_only,
OffsetNumber *redirected, int nredirected,
OffsetNumber *nowdead, int ndead,
OffsetNumber *nowunused, int nunused);
extern void heap_get_root_tuples(Page page, OffsetNumber *root_offsets);
+extern void log_heap_prune_and_freeze(Relation relation, Buffer buffer,
+ TransactionId conflict_xid,
+ bool lp_truncate_only,
+ PruneReason reason,
+ HeapTupleFreeze *frozen, int nfrozen,
+ OffsetNumber *redirected, int nredirected,
+ OffsetNumber *dead, int ndead,
+ OffsetNumber *unused, int nunused);
/* in heap/vacuumlazy.c */
struct VacuumParams;
* are associated with RM_HEAP2_ID, but are not logically different from
* the ones above associated with RM_HEAP_ID. XLOG_HEAP_OPMASK applies to
* these, too.
+ *
+ * There's no difference between XLOG_HEAP2_PRUNE_ON_ACCESS,
+ * XLOG_HEAP2_PRUNE_VACUUM_SCAN and XLOG_HEAP2_PRUNE_VACUUM_CLEANUP records.
+ * They have separate opcodes just for debugging and analysis purposes, to
+ * indicate why the WAL record was emitted.
*/
#define XLOG_HEAP2_REWRITE 0x00
-#define XLOG_HEAP2_PRUNE 0x10
-#define XLOG_HEAP2_VACUUM 0x20
-#define XLOG_HEAP2_FREEZE_PAGE 0x30
+#define XLOG_HEAP2_PRUNE_ON_ACCESS 0x10
+#define XLOG_HEAP2_PRUNE_VACUUM_SCAN 0x20
+#define XLOG_HEAP2_PRUNE_VACUUM_CLEANUP 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 page pruning (both during VACUUM and
- * during opportunistic pruning)
+ * These structures and flags encode VACUUM pruning and freezing and on-access
+ * pruning page modifications.
+ *
+ * xl_heap_prune is the main record. The XLHP_HAS_* flags indicate which
+ * "sub-records" are included and the other XLHP_* flags provide additional
+ * information about the conditions for replay.
+ *
+ * The data for block reference 0 contains "sub-records" depending on which of
+ * the XLHP_HAS_* flags are set. See xlhp_* struct definitions below. The
+ * sub-records appear in the same order as the XLHP_* flags. An example
+ * record with every sub-record included:
+ *
+ *-----------------------------------------------------------------------------
+ * Main data section:
+ *
+ * xl_heap_prune
+ * uint8 flags
+ * TransactionId snapshot_conflict_horizon
+ *
+ * Block 0 data section:
+ *
+ * xlhp_freeze_plans
+ * uint16 nplans
+ * [2 bytes of padding]
+ * xlhp_freeze_plan plans[nplans]
+ *
+ * xlhp_prune_items
+ * uint16 nredirected
+ * OffsetNumber redirected[2 * nredirected]
+ *
+ * xlhp_prune_items
+ * uint16 ndead
+ * OffsetNumber nowdead[ndead]
+ *
+ * xlhp_prune_items
+ * uint16 nunused
+ * OffsetNumber nowunused[nunused]
*
- * The array of OffsetNumbers following the fixed part of the record contains:
- * * for each redirected item: the item offset, then the offset redirected to
- * * for each now-dead item: the item offset
- * * for each now-unused item: the item offset
- * 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.
+ * OffsetNumber frz_offsets[sum([plan.ntuples for plan in plans])]
+ *-----------------------------------------------------------------------------
*
- * Acquires a full cleanup lock.
+ * NOTE: because the record data is assembled from many optional parts, we
+ * have to pay close attention to alignment. In the main data section,
+ * 'snapshot_conflict_horizon' is stored unaligned after 'flags', to save
+ * space. In the block 0 data section, the freeze plans appear first, because
+ * they contain TransactionId fields that require 4-byte alignment. All the
+ * other fields require only 2-byte alignment. This is also the reason that
+ * 'frz_offsets' is stored separately from the xlhp_freeze_plan structs.
*/
typedef struct xl_heap_prune
{
- TransactionId snapshotConflictHorizon;
- uint16 nredirected;
- uint16 ndead;
- bool isCatalogRel; /* to handle recovery conflict during logical
- * decoding on standby */
- /* OFFSET NUMBERS are in the block reference 0 */
+ uint8 reason;
+ uint8 flags;
+
+ /*
+ * If XLHP_HAS_CONFLICT_HORIZON is set, the conflict horzion XID follows,
+ * unaligned
+ */
} xl_heap_prune;
-#define SizeOfHeapPrune (offsetof(xl_heap_prune, isCatalogRel) + sizeof(bool))
+#define SizeOfHeapPrune (offsetof(xl_heap_prune, flags) + sizeof(uint8))
+
+/* to handle recovery conflict during logical decoding on standby */
+#define XLHP_IS_CATALOG_REL (1 << 1)
/*
- * The vacuum page record is similar to the prune record, but can only mark
- * already LP_DEAD items LP_UNUSED (during VACUUM's second heap pass)
+ * Does replaying the record require a cleanup-lock?
*
- * Acquires an ordinary exclusive lock only.
+ * Pruning, in VACUUM's first pass or when otherwise accessing a page,
+ * requires a cleanup lock. For freezing, and VACUUM's second pass which
+ * marks LP_DEAD line pointers as unused without moving any tuple data, an
+ * ordinary exclusive lock is sufficient.
+ */
+#define XLHP_CLEANUP_LOCK (1 << 2)
+
+/*
+ * If we remove or freeze any entries that contain xids, we need to include a
+ * snapshot conflict horizon. It's used in Hot Standby mode to ensure that
+ * there are no queries running for which the removed tuples are still
+ * visible, or which still consider the frozen XIDs as running.
+ */
+#define XLHP_HAS_CONFLICT_HORIZON (1 << 3)
+
+/*
+ * Indicates that an xlhp_freeze_plans sub-record and one or more
+ * xlhp_freeze_plan sub-records are present.
+ */
+#define XLHP_HAS_FREEZE_PLANS (1 << 4)
+
+/*
+ * XLHP_HAS_REDIRECTIONS, XLHP_HAS_DEAD_ITEMS, and XLHP_HAS_NOW_UNUSED
+ * indicate that xlhp_prune_items sub-records with redirected, dead, and
+ * unused item offsets are present.
*/
-typedef struct xl_heap_vacuum
+#define XLHP_HAS_REDIRECTIONS (1 << 5)
+#define XLHP_HAS_DEAD_ITEMS (1 << 6)
+#define XLHP_HAS_NOW_UNUSED_ITEMS (1 << 7)
+
+/*
+ * xlhp_freeze_plan describes how to freeze a group of one or more heap tuples
+ * (appears in xl_heap_prune's xlhp_freeze_plans sub-record)
+ */
+/* 0x01 was XLH_FREEZE_XMIN */
+#define XLH_FREEZE_XVAC 0x02
+#define XLH_INVALID_XVAC 0x04
+
+typedef struct xlhp_freeze_plan
{
- uint16 nunused;
- /* OFFSET NUMBERS are in the block reference 0 */
-} xl_heap_vacuum;
+ TransactionId xmax;
+ uint16 t_infomask2;
+ uint16 t_infomask;
+ uint8 frzflags;
+
+ /* Length of individual page offset numbers array for this plan */
+ uint16 ntuples;
+} xlhp_freeze_plan;
+
+/*
+ * This is what we need to know about a block being frozen during vacuum
+ *
+ * The backup block's data contains an array of xlhp_freeze_plan structs (with
+ * nplans elements). The individual item offsets are located in an array at
+ * the end of the entire record with with nplans * (each plan's ntuples)
+ * members. Those offsets are in the same order as the plans. The REDO
+ * routine uses the offsets to freeze the corresponding heap tuples.
+ *
+ * (As of PostgreSQL 17, XLOG_HEAP2_PRUNE_VACUUM_SCAN records replace the
+ * separate XLOG_HEAP2_FREEZE_PAGE records.)
+ */
+typedef struct xlhp_freeze_plans
+{
+ uint16 nplans;
+ xlhp_freeze_plan plans[FLEXIBLE_ARRAY_MEMBER];
+} xlhp_freeze_plans;
+
+/*
+ * Generic sub-record type contained in block reference 0 of an xl_heap_prune
+ * record and used for redirect, dead, and unused items if any of
+ * XLHP_HAS_REDIRECTIONS/XLHP_HAS_DEAD_ITEMS/XLHP_HAS_NOW_UNUSED_ITEMS are
+ * set. Note that in the XLHP_HAS_REDIRECTIONS variant, there are actually 2
+ * * length number of OffsetNumbers in the data.
+ */
+typedef struct xlhp_prune_items
+{
+ uint16 ntargets;
+ OffsetNumber data[FLEXIBLE_ARRAY_MEMBER];
+} xlhp_prune_items;
-#define SizeOfHeapVacuum (offsetof(xl_heap_vacuum, nunused) + sizeof(uint16))
/* flags for infobits_set */
#define XLHL_XMAX_IS_MULTI 0x01
#define SizeOfHeapInplace (offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
-/*
- * This struct represents a 'freeze plan', which describes how to freeze a
- * group of one or more heap tuples (appears in xl_heap_freeze_page record)
- */
-/* 0x01 was XLH_FREEZE_XMIN */
-#define XLH_FREEZE_XVAC 0x02
-#define XLH_INVALID_XVAC 0x04
-
-typedef struct xl_heap_freeze_plan
-{
- TransactionId xmax;
- uint16 t_infomask2;
- uint16 t_infomask;
- uint8 frzflags;
-
- /* Length of individual page offset numbers array for this plan */
- uint16 ntuples;
-} xl_heap_freeze_plan;
-
-/*
- * This is what we need to know about a block being frozen during vacuum
- *
- * Backup block 0's data contains an array of xl_heap_freeze_plan structs
- * (with nplans elements), followed by one or more page offset number arrays.
- * Each such page offset number array corresponds to a single freeze plan
- * (REDO routine freezes corresponding heap tuples using freeze plan).
- */
-typedef struct xl_heap_freeze_page
-{
- TransactionId snapshotConflictHorizon;
- uint16 nplans;
- bool isCatalogRel; /* to handle recovery conflict during logical
- * decoding on standby */
-
- /*
- * In payload of blk 0 : FREEZE PLANS and OFFSET NUMBER ARRAY
- */
-} xl_heap_freeze_page;
-
-#define SizeOfHeapFreezePage (offsetof(xl_heap_freeze_page, isCatalogRel) + sizeof(bool))
-
/*
* This is what we need to know about setting a visibility map bit
*
TransactionId snapshotConflictHorizon,
uint8 vmflags);
+/* in heapdesc.c, so it can be shared between frontend/backend code */
+extern void heap_xlog_deserialize_prune_and_freeze(char *cursor, uint8 flags,
+ int *nplans, xlhp_freeze_plan **plans,
+ OffsetNumber **frz_offsets,
+ int *nredirected, OffsetNumber **redirected,
+ int *ndead, OffsetNumber **nowdead,
+ int *nunused, OffsetNumber **nowunused);
+
#endif /* HEAPAM_XLOG_H */
/*
* Each page of XLOG file has a header like this:
*/
-#define XLOG_PAGE_MAGIC 0xD114 /* can be used as WAL version indicator */
+#define XLOG_PAGE_MAGIC 0xD115 /* can be used as WAL version indicator */
typedef struct XLogPageHeaderData
{
PromptInterruptContext
ProtocolVersion
PrsStorage
+PruneReason
PruneResult
PruneState
PruneStepResult
xl_hash_vacuum_one_page
xl_heap_confirm
xl_heap_delete
-xl_heap_freeze_page
-xl_heap_freeze_plan
xl_heap_header
xl_heap_inplace
xl_heap_insert
xl_heap_rewrite_mapping
xl_heap_truncate
xl_heap_update
-xl_heap_vacuum
xl_heap_visible
xl_invalid_page
xl_invalid_page_key
xl_xact_subxacts
xl_xact_twophase
xl_xact_xinfo
+xlhp_freeze_plan
+xlhp_freeze_plans
+xlhp_prune_items
xmlBuffer
xmlBufferPtr
xmlChar