* value is removed; the cutoff value is stored in pg_class. The minimum value
* across all tables in each database is stored in pg_database, and the global
* minimum across all databases is part of pg_control and is kept in shared
- * memory. At checkpoint time, after the value is known flushed in WAL, any
- * files that correspond to multixacts older than that value are removed.
- * (These files are also removed when a restartpoint is executed.)
+ * memory. Whenever that minimum is advanced, the SLRUs are truncated.
*
* When new multixactid values are to be created, care is taken that the
* counter does not fall within the wraparound horizon considering the global
#include "postmaster/autovacuum.h"
#include "storage/lmgr.h"
#include "storage/pmsignal.h"
+#include "storage/proc.h"
#include "storage/procarray.h"
#include "utils/builtins.h"
#include "utils/memutils.h"
((xid) / (MultiXactOffset) MULTIXACT_OFFSETS_PER_PAGE)
#define MultiXactIdToOffsetEntry(xid) \
((xid) % (MultiXactOffset) MULTIXACT_OFFSETS_PER_PAGE)
+#define MultiXactIdToOffsetSegment(xid) (MultiXactIdToOffsetPage(xid) / SLRU_PAGES_PER_SEGMENT)
/*
* The situation for members is a bit more complex: we store one byte of
/* page in which a member is to be found */
#define MXOffsetToMemberPage(xid) ((xid) / (TransactionId) MULTIXACT_MEMBERS_PER_PAGE)
+#define MXOffsetToMemberSegment(xid) (MXOffsetToMemberPage(xid) / SLRU_PAGES_PER_SEGMENT)
/* Location (byte offset within page) of flag word for a given member */
#define MXOffsetToFlagsOffset(xid) \
Oid oldestMultiXactDB;
/*
- * Oldest multixact offset that is potentially referenced by a
- * multixact referenced by a relation. We don't always know this value,
- * so there's a flag here to indicate whether or not we currently do.
+ * Oldest multixact offset that is potentially referenced by a multixact
+ * referenced by a relation. We don't always know this value, so there's
+ * a flag here to indicate whether or not we currently do.
*/
MultiXactOffset oldestOffset;
bool oldestOffsetKnown;
/*
- * This is what the previous checkpoint stored as the truncate position.
- * This value is the oldestMultiXactId that was valid when a checkpoint
- * was last executed.
+ * True if a multixact truncation WAL record was replayed since the last
+ * checkpoint. This is used to trigger 'legacy truncations', i.e. truncate
+ * by looking at the data directory during WAL replay, when the primary is
+ * too old to generate truncation records.
*/
- MultiXactId lastCheckpointedOldest;
+ bool sawTruncationInCkptCycle;
/* support for anti-wraparound measures */
MultiXactId multiVacLimit;
MultiXactId multiWrapLimit;
/* support for members anti-wraparound measures */
- MultiXactOffset offsetStopLimit;
- bool offsetStopLimitKnown;
+ MultiXactOffset offsetStopLimit; /* known if oldestOffsetKnown */
/*
* Per-backend data starts here. We have two arrays stored in the area
MultiXactOffset offset2);
static void ExtendMultiXactOffset(MultiXactId multi);
static void ExtendMultiXactMember(MultiXactOffset offset, int nmembers);
-static void DetermineSafeOldestOffset(MultiXactId oldestMXact);
static bool MultiXactOffsetWouldWrap(MultiXactOffset boundary,
MultiXactOffset start, uint32 distance);
-static bool SetOffsetVacuumLimit(bool finish_setup);
+static bool SetOffsetVacuumLimit(void);
static bool find_multixact_start(MultiXactId multi, MultiXactOffset *result);
static void WriteMZeroPageXlogRec(int pageno, uint8 info);
+static void WriteMTruncateXlogRec(Oid oldestMultiDB,
+ MultiXactId startOff, MultiXactId endOff,
+ MultiXactOffset startMemb, MultiXactOffset endMemb);
/*
*----------
*/
#define OFFSET_WARN_SEGMENTS 20
- if (MultiXactState->offsetStopLimitKnown &&
+ if (MultiXactState->oldestOffsetKnown &&
MultiXactOffsetWouldWrap(MultiXactState->offsetStopLimit, nextOffset,
nmembers))
{
SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER);
}
- if (MultiXactState->offsetStopLimitKnown &&
+ if (MultiXactState->oldestOffsetKnown &&
MultiXactOffsetWouldWrap(MultiXactState->offsetStopLimit,
nextOffset,
nmembers + MULTIXACT_MEMBERS_PER_PAGE * SLRU_PAGES_PER_SEGMENT * OFFSET_WARN_SEGMENTS))
/*
* This must be called ONCE at the end of startup/recovery.
- *
- * We don't need any locks here, really; the SLRU locks are taken only because
- * slru.c expects to be called with locks held.
*/
void
TrimMultiXact(void)
{
- MultiXactId multi = MultiXactState->nextMXact;
- MultiXactOffset offset = MultiXactState->nextOffset;
- MultiXactId oldestMXact;
+ MultiXactId nextMXact;
+ MultiXactOffset offset;
+ MultiXactId oldestMXact;
+ Oid oldestMXactDB;
int pageno;
int entryno;
int flagsoff;
+ LWLockAcquire(MultiXactGenLock, LW_SHARED);
+ nextMXact = MultiXactState->nextMXact;
+ offset = MultiXactState->nextOffset;
+ oldestMXact = MultiXactState->oldestMultiXactId;
+ oldestMXactDB = MultiXactState->oldestMultiXactDB;
+ LWLockRelease(MultiXactGenLock);
/* Clean up offsets state */
LWLockAcquire(MultiXactOffsetControlLock, LW_EXCLUSIVE);
/*
* (Re-)Initialize our idea of the latest page number for offsets.
*/
- pageno = MultiXactIdToOffsetPage(multi);
+ pageno = MultiXactIdToOffsetPage(nextMXact);
MultiXactOffsetCtl->shared->latest_page_number = pageno;
/*
* Zero out the remainder of the current offsets page. See notes in
* TrimCLOG() for motivation.
*/
- entryno = MultiXactIdToOffsetEntry(multi);
+ entryno = MultiXactIdToOffsetEntry(nextMXact);
if (entryno != 0)
{
int slotno;
MultiXactOffset *offptr;
- slotno = SimpleLruReadPage(MultiXactOffsetCtl, pageno, true, multi);
+ slotno = SimpleLruReadPage(MultiXactOffsetCtl, pageno, true, nextMXact);
offptr = (MultiXactOffset *) MultiXactOffsetCtl->shared->page_buffer[slotno];
offptr += entryno;
LWLockRelease(MultiXactMemberControlLock);
- if (SetOffsetVacuumLimit(true) && IsUnderPostmaster)
- SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER);
- LWLockAcquire(MultiXactGenLock, LW_SHARED);
- oldestMXact = MultiXactState->lastCheckpointedOldest;
+ /* signal that we're officially up */
+ LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
+ MultiXactState->finishedStartup = true;
LWLockRelease(MultiXactGenLock);
- DetermineSafeOldestOffset(oldestMXact);
+
+ /* Now compute how far away the next members wraparound is. */
+ SetMultiXactIdLimit(oldestMXact, oldestMXactDB);
}
/*
(errmsg("MultiXactId wrap limit is %u, limited by database with OID %u",
multiWrapLimit, oldest_datoid)));
+ /*
+ * Computing the actual limits is only possible once the data directory is
+ * in a consistent state. There's no need to compute the limits while
+ * still replaying WAL - no decisions about new multis are made even
+ * though multixact creations might be replayed. So we'll only do further
+ * checks after TrimMultiXact() has been called.
+ */
+ if (!MultiXactState->finishedStartup)
+ return;
+
+ Assert(!InRecovery);
+
/* Set limits for offset vacuum. */
- needs_offset_vacuum = SetOffsetVacuumLimit(false);
+ needs_offset_vacuum = SetOffsetVacuumLimit();
/*
* If past the autovacuum force point, immediately signal an autovac
* another iteration immediately if there are still any old databases.
*/
if ((MultiXactIdPrecedes(multiVacLimit, curMulti) ||
- needs_offset_vacuum) && IsUnderPostmaster && !InRecovery)
+ needs_offset_vacuum) && IsUnderPostmaster)
SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER);
/* Give an immediate warning if past the wrap warn point */
- if (MultiXactIdPrecedes(multiWarnLimit, curMulti) && !InRecovery)
+ if (MultiXactIdPrecedes(multiWarnLimit, curMulti))
{
char *oldest_datname;
}
/*
- * Update our oldestMultiXactId value, but only if it's more recent than
- * what we had. However, even if not, always update the oldest multixact
- * offset limit.
+ * Update our oldestMultiXactId value, but only if it's more recent than what
+ * we had.
+ *
+ * This may only be called during WAL replay.
*/
void
MultiXactAdvanceOldest(MultiXactId oldestMulti, Oid oldestMultiDB)
{
+ Assert(InRecovery);
+
if (MultiXactIdPrecedes(MultiXactState->oldestMultiXactId, oldestMulti))
+ {
+ /*
+ * If there has been a truncation on the master, detected by seeing a
+ * moving oldestMulti, without a corresponding truncation record, we
+ * know that the primary is still running an older version of postgres
+ * that doesn't yet log multixact truncations. So perform the
+ * truncation ourselves.
+ */
+ if (!MultiXactState->sawTruncationInCkptCycle)
+ {
+ ereport(LOG,
+ (errmsg("performing legacy multixact truncation"),
+ errdetail("Legacy truncations are sometimes performed when replaying WAL from an older primary."),
+ errhint("Upgrade the primary, it is susceptible to data corruption.")));
+ TruncateMultiXact(oldestMulti, oldestMultiDB, true);
+ }
+
SetMultiXactIdLimit(oldestMulti, oldestMultiDB);
-}
+ }
-/*
- * Update the "safe truncation point". This is the newest value of oldestMulti
- * that is known to be flushed as part of a checkpoint record.
- */
-void
-MultiXactSetSafeTruncate(MultiXactId safeTruncateMulti)
-{
- LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
- MultiXactState->lastCheckpointedOldest = safeTruncateMulti;
- LWLockRelease(MultiXactGenLock);
+ /* only looked at in the startup process, no lock necessary */
+ MultiXactState->sawTruncationInCkptCycle = false;
}
/*
return oldestMXact;
}
-/*
- * Based on the given oldest MultiXactId, determine what's the oldest member
- * offset and install the limit info in MultiXactState, where it can be used to
- * prevent overrun of old data in the members SLRU area.
- */
-static void
-DetermineSafeOldestOffset(MultiXactId oldestMXact)
-{
- MultiXactOffset oldestOffset;
- MultiXactOffset nextOffset;
- MultiXactOffset offsetStopLimit;
- MultiXactOffset prevOffsetStopLimit;
- MultiXactId nextMXact;
- bool finishedStartup;
- bool prevOffsetStopLimitKnown;
-
- /* Fetch values from shared memory. */
- LWLockAcquire(MultiXactGenLock, LW_SHARED);
- finishedStartup = MultiXactState->finishedStartup;
- nextMXact = MultiXactState->nextMXact;
- nextOffset = MultiXactState->nextOffset;
- prevOffsetStopLimit = MultiXactState->offsetStopLimit;
- prevOffsetStopLimitKnown = MultiXactState->offsetStopLimitKnown;
- LWLockRelease(MultiXactGenLock);
-
- /* Don't worry about this until after we've started up. */
- if (!finishedStartup)
- return;
-
- /*
- * Determine the offset of the oldest multixact. Normally, we can read
- * the offset from the multixact itself, but there's an important special
- * case: if there are no multixacts in existence at all, oldestMXact
- * obviously can't point to one. It will instead point to the multixact
- * ID that will be assigned the next time one is needed.
- *
- * NB: oldestMXact should be the oldest multixact that still exists in the
- * SLRU, unlike in SetOffsetVacuumLimit, where we do this same computation
- * based on the oldest value that might be referenced in a table.
- */
- if (nextMXact == oldestMXact)
- oldestOffset = nextOffset;
- else
- {
- bool oldestOffsetKnown;
-
- oldestOffsetKnown = find_multixact_start(oldestMXact, &oldestOffset);
- if (!oldestOffsetKnown)
- {
- ereport(LOG,
- (errmsg("MultiXact member wraparound protections are disabled because oldest checkpointed MultiXact %u does not exist on disk",
- oldestMXact)));
- return;
- }
- }
-
- /* move back to start of the corresponding segment */
- offsetStopLimit = oldestOffset - (oldestOffset %
- (MULTIXACT_MEMBERS_PER_PAGE * SLRU_PAGES_PER_SEGMENT));
- /* always leave one segment before the wraparound point */
- offsetStopLimit -= (MULTIXACT_MEMBERS_PER_PAGE * SLRU_PAGES_PER_SEGMENT);
-
- /* if nothing has changed, we're done */
- if (prevOffsetStopLimitKnown && offsetStopLimit == prevOffsetStopLimit)
- return;
-
- LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
- MultiXactState->offsetStopLimit = offsetStopLimit;
- MultiXactState->offsetStopLimitKnown = true;
- LWLockRelease(MultiXactGenLock);
-
- if (!prevOffsetStopLimitKnown && IsUnderPostmaster)
- ereport(LOG,
- (errmsg("MultiXact member wraparound protections are now enabled")));
- ereport(DEBUG1,
- (errmsg("MultiXact member stop limit is now %u based on MultiXact %u",
- offsetStopLimit, oldestMXact)));
-}
-
/*
* Determine how aggressively we need to vacuum in order to prevent member
* wraparound.
*
- * To determine the oldest multixact ID, we look at oldestMultiXactId, not
- * lastCheckpointedOldest. That's because vacuuming can't help with anything
- * older than oldestMultiXactId; anything older than that isn't referenced
- * by any table. Offsets older than oldestMultiXactId but not as old as
- * lastCheckpointedOldest will go away after the next checkpoint.
+ * To do so determine what's the oldest member offset and install the limit
+ * info in MultiXactState, where it can be used to prevent overrun of old data
+ * in the members SLRU area.
*
* The return value is true if emergency autovacuum is required and false
* otherwise.
*/
static bool
-SetOffsetVacuumLimit(bool finish_setup)
+SetOffsetVacuumLimit(void)
{
- MultiXactId oldestMultiXactId;
+ MultiXactId oldestMultiXactId;
MultiXactId nextMXact;
- bool finishedStartup;
- MultiXactOffset oldestOffset = 0; /* placate compiler */
+ MultiXactOffset oldestOffset = 0; /* placate compiler */
+ MultiXactOffset prevOldestOffset;
MultiXactOffset nextOffset;
bool oldestOffsetKnown = false;
- MultiXactOffset prevOldestOffset;
bool prevOldestOffsetKnown;
+ MultiXactOffset offsetStopLimit = 0;
+
+ /*
+ * NB: Have to prevent concurrent truncation, we might otherwise try to
+ * lookup a oldestMulti that's concurrently getting truncated away.
+ */
+ LWLockAcquire(MultiXactTruncationLock, LW_SHARED);
/* Read relevant fields from shared memory. */
LWLockAcquire(MultiXactGenLock, LW_SHARED);
oldestMultiXactId = MultiXactState->oldestMultiXactId;
nextMXact = MultiXactState->nextMXact;
nextOffset = MultiXactState->nextOffset;
- finishedStartup = MultiXactState->finishedStartup;
- prevOldestOffset = MultiXactState->oldestOffset;
prevOldestOffsetKnown = MultiXactState->oldestOffsetKnown;
+ prevOldestOffset = MultiXactState->oldestOffset;
+ Assert(MultiXactState->finishedStartup);
LWLockRelease(MultiXactGenLock);
- /* Don't do this until after any recovery is complete. */
- if (!finishedStartup && !finish_setup)
- return false;
-
/*
- * If no multixacts exist, then oldestMultiXactId will be the next
- * multixact that will be created, rather than an existing multixact.
+ * Determine the offset of the oldest multixact. Normally, we can read
+ * the offset from the multixact itself, but there's an important special
+ * case: if there are no multixacts in existence at all, oldestMXact
+ * obviously can't point to one. It will instead point to the multixact
+ * ID that will be assigned the next time one is needed.
*/
if (oldestMultiXactId == nextMXact)
{
/*
- * When the next multixact gets created, it will be stored at the
- * next offset.
+ * When the next multixact gets created, it will be stored at the next
+ * offset.
*/
oldestOffset = nextOffset;
oldestOffsetKnown = true;
else
{
/*
- * Figure out where the oldest existing multixact's offsets are stored.
- * Due to bugs in early release of PostgreSQL 9.3.X and 9.4.X, the
- * supposedly-earliest multixact might not really exist. We are
+ * Figure out where the oldest existing multixact's offsets are
+ * stored. Due to bugs in early release of PostgreSQL 9.3.X and 9.4.X,
+ * the supposedly-earliest multixact might not really exist. We are
* careful not to fail in that case.
*/
oldestOffsetKnown =
find_multixact_start(oldestMultiXactId, &oldestOffset);
- }
- /*
- * Except when initializing the system for the first time, there's no
- * need to update anything if we don't know the oldest offset or if it
- * hasn't changed.
- */
- if (finish_setup ||
- (oldestOffsetKnown && !prevOldestOffsetKnown) ||
- (oldestOffsetKnown && prevOldestOffset != oldestOffset))
- {
- /* Install the new limits. */
- LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
- MultiXactState->oldestOffset = oldestOffset;
- MultiXactState->oldestOffsetKnown = oldestOffsetKnown;
- MultiXactState->finishedStartup = true;
- LWLockRelease(MultiXactGenLock);
-
- /* Log the info */
if (oldestOffsetKnown)
ereport(DEBUG1,
(errmsg("oldest MultiXactId member is at offset %u",
- oldestOffset)));
+ oldestOffset)));
else
- ereport(DEBUG1,
- (errmsg("oldest MultiXactId member offset unknown")));
+ ereport(LOG,
+ (errmsg("MultiXact member wraparound protections are disabled because oldest checkpointed MultiXact %u does not exist on disk",
+ oldestMultiXactId)));
}
+ LWLockRelease(MultiXactTruncationLock);
+
/*
- * If we failed to get the oldest offset this time, but we have a value
- * from a previous pass through this function, assess the need for
- * autovacuum based on that old value rather than automatically forcing
- * it.
+ * If we can, compute limits (and install them MultiXactState) to prevent
+ * overrun of old data in the members SLRU area. We can only do so if the
+ * oldest offset is known though.
*/
- if (prevOldestOffsetKnown && !oldestOffsetKnown)
+ if (oldestOffsetKnown)
+ {
+ /* move back to start of the corresponding segment */
+ offsetStopLimit = oldestOffset - (oldestOffset %
+ (MULTIXACT_MEMBERS_PER_PAGE * SLRU_PAGES_PER_SEGMENT));
+
+ /* always leave one segment before the wraparound point */
+ offsetStopLimit -= (MULTIXACT_MEMBERS_PER_PAGE * SLRU_PAGES_PER_SEGMENT);
+
+ if (!prevOldestOffsetKnown && IsUnderPostmaster)
+ ereport(LOG,
+ (errmsg("MultiXact member wraparound protections are now enabled")));
+ ereport(DEBUG1,
+ (errmsg("MultiXact member stop limit is now %u based on MultiXact %u",
+ offsetStopLimit, oldestMultiXactId)));
+ }
+ else if (prevOldestOffsetKnown)
{
+ /*
+ * If we failed to get the oldest offset this time, but we have a
+ * value from a previous pass through this function, use the old value
+ * rather than automatically forcing it.
+ */
oldestOffset = prevOldestOffset;
oldestOffsetKnown = true;
}
+ /* Install the computed values */
+ LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
+ MultiXactState->oldestOffset = oldestOffset;
+ MultiXactState->oldestOffsetKnown = oldestOffsetKnown;
+ MultiXactState->offsetStopLimit = offsetStopLimit;
+ LWLockRelease(MultiXactGenLock);
+
/*
- * Do we need an emergency autovacuum? If we're not sure, assume yes.
+ * Do we need an emergency autovacuum? If we're not sure, assume yes.
*/
return !oldestOffsetKnown ||
(nextOffset - oldestOffset > MULTIXACT_MEMBER_SAFE_THRESHOLD);
* boundary point, hence the name. The reason we don't want to use the regular
* 2^31-modulo arithmetic here is that we want to be able to use the whole of
* the 2^32-1 space here, allowing for more multixacts that would fit
- * otherwise. See also SlruScanDirCbRemoveMembers.
+ * otherwise.
*/
static bool
MultiXactOffsetWouldWrap(MultiXactOffset boundary, MultiXactOffset start,
*
* Returns false if the file containing the multi does not exist on disk.
* Otherwise, returns true and sets *result to the starting member offset.
+ *
+ * This function does not prevent concurrent truncation, so if that's
+ * required, the caller has to protect against that.
*/
static bool
find_multixact_start(MultiXactId multi, MultiXactOffset *result)
int slotno;
MultiXactOffset *offptr;
+ /* XXX: Remove || AmStartupProcess() after WAL page magic bump */
+ Assert(MultiXactState->finishedStartup || AmStartupProcess());
+
pageno = MultiXactIdToOffsetPage(multi);
entryno = MultiXactIdToOffsetEntry(multi);
+ /*
+ * Flush out dirty data, so PhysicalPageExists can work correctly.
+ * SimpleLruFlush() is a pretty big hammer for that. Alternatively we
+ * could add a in-memory version of page exists, but find_multixact_start
+ * is called infrequently, and it doesn't seem bad to flush buffers to
+ * disk before truncation.
+ */
+ SimpleLruFlush(MultiXactOffsetCtl, true);
+ SimpleLruFlush(MultiXactMemberCtl, true);
+
if (!SimpleLruDoesPhysicalPageExist(MultiXactOffsetCtl, pageno))
return false;
return multixacts - victim_multixacts;
}
-/*
- * SlruScanDirectory callback.
- * This callback deletes segments that are outside the range determined by
- * the given page numbers.
- *
- * Both range endpoints are exclusive (that is, segments containing any of
- * those pages are kept.)
- */
-typedef struct MembersLiveRange
-{
- int rangeStart;
- int rangeEnd;
-} MembersLiveRange;
-
-static bool
-SlruScanDirCbRemoveMembers(SlruCtl ctl, char *filename, int segpage,
- void *data)
-{
- MembersLiveRange *range = (MembersLiveRange *) data;
- MultiXactOffset nextOffset;
-
- if ((segpage == range->rangeStart) ||
- (segpage == range->rangeEnd))
- return false; /* easy case out */
-
- /*
- * To ensure that no segment is spuriously removed, we must keep track of
- * new segments added since the start of the directory scan; to do this,
- * we update our end-of-range point as we run.
- *
- * As an optimization, we can skip looking at shared memory if we know for
- * certain that the current segment must be kept. This is so because
- * nextOffset never decreases, and we never increase rangeStart during any
- * one run.
- */
- if (!((range->rangeStart > range->rangeEnd &&
- segpage > range->rangeEnd && segpage < range->rangeStart) ||
- (range->rangeStart < range->rangeEnd &&
- (segpage < range->rangeStart || segpage > range->rangeEnd))))
- return false;
-
- /*
- * Update our idea of the end of the live range.
- */
- LWLockAcquire(MultiXactGenLock, LW_SHARED);
- nextOffset = MultiXactState->nextOffset;
- LWLockRelease(MultiXactGenLock);
- range->rangeEnd = MXOffsetToMemberPage(nextOffset);
-
- /* Recheck the deletion condition. If it still holds, perform deletion */
- if ((range->rangeStart > range->rangeEnd &&
- segpage > range->rangeEnd && segpage < range->rangeStart) ||
- (range->rangeStart < range->rangeEnd &&
- (segpage < range->rangeStart || segpage > range->rangeEnd)))
- SlruDeleteSegment(ctl, filename);
-
- return false; /* keep going */
-}
-
typedef struct mxtruncinfo
{
int earliestExistingPage;
return false; /* keep going */
}
+
+/*
+ * Delete members segments [oldest, newOldest)
+ *
+ * The members SLRU can, in contrast to the offsets one, be filled to almost
+ * the full range at once. This means SimpleLruTruncate() can't trivially be
+ * used - instead the to-be-deleted range is computed using the offsets
+ * SLRU. C.f. TruncateMultiXact().
+ */
+static void
+PerformMembersTruncation(MultiXactOffset oldestOffset, MultiXactOffset newOldestOffset)
+{
+ const int maxsegment = MXOffsetToMemberSegment(MaxMultiXactOffset);
+ int startsegment = MXOffsetToMemberSegment(oldestOffset);
+ int endsegment = MXOffsetToMemberSegment(newOldestOffset);
+ int segment = startsegment;
+
+ /*
+ * Delete all the segments but the last one. The last segment can still
+ * contain, possibly partially, valid data.
+ */
+ while (segment != endsegment)
+ {
+ elog(DEBUG2, "truncating multixact members segment %x", segment);
+ SlruDeleteSegment(MultiXactMemberCtl, segment);
+
+ /* move to next segment, handling wraparound correctly */
+ if (segment == maxsegment)
+ segment = 0;
+ else
+ segment += 1;
+ }
+}
+
+/*
+ * Delete offsets segments [oldest, newOldest)
+ */
+static void
+PerformOffsetsTruncation(MultiXactId oldestMulti, MultiXactId newOldestMulti)
+{
+ /*
+ * We step back one multixact to avoid passing a cutoff page that hasn't
+ * been created yet in the rare case that oldestMulti would be the first
+ * item on a page and oldestMulti == nextMulti. In that case, if we
+ * didn't subtract one, we'd trigger SimpleLruTruncate's wraparound
+ * detection.
+ */
+ SimpleLruTruncate(MultiXactOffsetCtl,
+ MultiXactIdToOffsetPage(PreviousMultiXactId(newOldestMulti)));
+}
+
/*
* Remove all MultiXactOffset and MultiXactMember segments before the oldest
* ones still of interest.
*
- * On a primary, this is called by the checkpointer process after a checkpoint
- * has been flushed; during crash recovery, it's called from
- * CreateRestartPoint(). In the latter case, we rely on the fact that
- * xlog_redo() will already have called MultiXactAdvanceOldest(). Our
- * latest_page_number will already have been initialized by StartupMultiXact()
- * and kept up to date as new pages are zeroed.
+ * On a primary this is called as part of vacuum (via
+ * vac_truncate_clog()). During recovery truncation is normally done by
+ * replaying truncation WAL records instead of this routine; the exception is
+ * when replaying records from an older primary that doesn't yet generate
+ * truncation WAL records. In that case truncation is triggered by
+ * MultiXactAdvanceOldest().
+ *
+ * newOldestMulti is the oldest currently required multixact, newOldestMultiDB
+ * is one of the databases preventing newOldestMulti from increasing.
*/
void
-TruncateMultiXact(void)
+TruncateMultiXact(MultiXactId newOldestMulti, Oid newOldestMultiDB, bool in_recovery)
{
- MultiXactId oldestMXact;
+ MultiXactId oldestMulti;
+ MultiXactId nextMulti;
+ MultiXactOffset newOldestOffset;
MultiXactOffset oldestOffset;
- MultiXactId nextMXact;
- MultiXactOffset nextOffset;
+ MultiXactOffset nextOffset;
mxtruncinfo trunc;
MultiXactId earliest;
- MembersLiveRange range;
- Assert(AmCheckpointerProcess() || AmStartupProcess() ||
- !IsPostmasterEnvironment);
+ /*
+ * Need to allow being called in recovery for backwards compatibility,
+ * when an updated standby replays WAL generated by a non-updated primary.
+ */
+ Assert(in_recovery || !RecoveryInProgress());
+ Assert(!in_recovery || AmStartupProcess());
+ Assert(in_recovery || MultiXactState->finishedStartup);
+
+ /*
+ * We can only allow one truncation to happen at once. Otherwise parts of
+ * members might vanish while we're doing lookups or similar. There's no
+ * need to have an interlock with creating new multis or such, since those
+ * are constrained by the limits (which only grow, never shrink).
+ */
+ LWLockAcquire(MultiXactTruncationLock, LW_EXCLUSIVE);
LWLockAcquire(MultiXactGenLock, LW_SHARED);
- oldestMXact = MultiXactState->lastCheckpointedOldest;
- nextMXact = MultiXactState->nextMXact;
+ nextMulti = MultiXactState->nextMXact;
nextOffset = MultiXactState->nextOffset;
+ oldestMulti = MultiXactState->oldestMultiXactId;
LWLockRelease(MultiXactGenLock);
- Assert(MultiXactIdIsValid(oldestMXact));
+ Assert(MultiXactIdIsValid(oldestMulti));
+
+ /*
+ * Make sure to only attempt truncation if there's values to truncate
+ * away. In normal processing values shouldn't go backwards, but there's
+ * some corner cases (due to bugs) where that's possible.
+ */
+ if (MultiXactIdPrecedesOrEquals(newOldestMulti, oldestMulti))
+ {
+ LWLockRelease(MultiXactTruncationLock);
+ return;
+ }
/*
* Note we can't just plow ahead with the truncation; it's possible that
* going to attempt to read the offsets page to determine where to
* truncate the members SLRU. So we first scan the directory to determine
* the earliest offsets page number that we can read without error.
+ *
+ * NB: It's also possible that the page that oldestMulti is on has already
+ * been truncated away, and we crashed before updating oldestMulti.
*/
trunc.earliestExistingPage = -1;
SlruScanDirectory(MultiXactOffsetCtl, SlruScanDirCbFindEarliest, &trunc);
if (earliest < FirstMultiXactId)
earliest = FirstMultiXactId;
- /*
- * If there's nothing to remove, we can bail out early.
- *
- * Due to bugs in early releases of PostgreSQL 9.3.X and 9.4.X,
- * oldestMXact might point to a multixact that does not exist.
- * Autovacuum will eventually advance it to a value that does exist,
- * and we want to set a proper offsetStopLimit when that happens,
- * so call DetermineSafeOldestOffset here even if we're not actually
- * truncating.
- */
- if (MultiXactIdPrecedes(oldestMXact, earliest))
+ /* If there's nothing to remove, we can bail out early. */
+ if (MultiXactIdPrecedes(oldestMulti, earliest))
{
- DetermineSafeOldestOffset(oldestMXact);
+ LWLockRelease(MultiXactTruncationLock);
return;
}
* the starting offset of the oldest multixact.
*
* Hopefully, find_multixact_start will always work here, because we've
- * already checked that it doesn't precede the earliest MultiXact on
- * disk. But if it fails, don't truncate anything, and log a message.
+ * already checked that it doesn't precede the earliest MultiXact on disk.
+ * But if it fails, don't truncate anything, and log a message.
*/
- if (oldestMXact == nextMXact)
- oldestOffset = nextOffset; /* there are NO MultiXacts */
- else if (!find_multixact_start(oldestMXact, &oldestOffset))
+ if (oldestMulti == nextMulti)
+ {
+ /* there are NO MultiXacts */
+ oldestOffset = nextOffset;
+ }
+ else if (!find_multixact_start(oldestMulti, &oldestOffset))
{
ereport(LOG,
(errmsg("oldest MultiXact %u not found, earliest MultiXact %u, skipping truncation",
- oldestMXact, earliest)));
+ oldestMulti, earliest)));
+ LWLockRelease(MultiXactTruncationLock);
return;
}
/*
- * To truncate MultiXactMembers, we need to figure out the active page
- * range and delete all files outside that range. The start point is the
- * start of the segment containing the oldest offset; an end point of the
- * segment containing the next offset to use is enough. The end point is
- * updated as MultiXactMember gets extended concurrently, elsewhere.
+ * Secondly compute up to where to truncate. Lookup the corresponding
+ * member offset for newOldestMulti for that.
*/
- range.rangeStart = MXOffsetToMemberPage(oldestOffset);
- range.rangeStart -= range.rangeStart % SLRU_PAGES_PER_SEGMENT;
-
- range.rangeEnd = MXOffsetToMemberPage(nextOffset);
+ if (newOldestMulti == nextMulti)
+ {
+ /* there are NO MultiXacts */
+ newOldestOffset = nextOffset;
+ }
+ else if (!find_multixact_start(newOldestMulti, &newOldestOffset))
+ {
+ ereport(LOG,
+ (errmsg("cannot truncate up to MultiXact %u because it does not exist on disk, skipping truncation",
+ newOldestMulti)));
+ LWLockRelease(MultiXactTruncationLock);
+ return;
+ }
- SlruScanDirectory(MultiXactMemberCtl, SlruScanDirCbRemoveMembers, &range);
+ elog(DEBUG1, "performing multixact truncation: "
+ "offsets [%u, %u), offsets segments [%x, %x), "
+ "members [%u, %u), members segments [%x, %x)",
+ oldestMulti, newOldestMulti,
+ MultiXactIdToOffsetSegment(oldestMulti),
+ MultiXactIdToOffsetSegment(newOldestMulti),
+ oldestOffset, newOldestOffset,
+ MXOffsetToMemberSegment(oldestOffset),
+ MXOffsetToMemberSegment(newOldestOffset));
/*
- * Now we can truncate MultiXactOffset. We step back one multixact to
- * avoid passing a cutoff page that hasn't been created yet in the rare
- * case that oldestMXact would be the first item on a page and oldestMXact
- * == nextMXact. In that case, if we didn't subtract one, we'd trigger
- * SimpleLruTruncate's wraparound detection.
+ * Do truncation, and the WAL logging of the truncation, in a critical
+ * section. That way offsets/members cannot get out of sync anymore, i.e.
+ * once consistent the newOldestMulti will always exist in members, even
+ * if we crashed in the wrong moment.
*/
- SimpleLruTruncate(MultiXactOffsetCtl,
- MultiXactIdToOffsetPage(PreviousMultiXactId(oldestMXact)));
+ START_CRIT_SECTION();
/*
- * Now, and only now, we can advance the stop point for multixact members.
- * If we did it any sooner, the segments we deleted above might already
- * have been overwritten with new members. That would be bad.
+ * Prevent checkpoints from being scheduled concurrently. This is critical
+ * because otherwise a truncation record might not be replayed after a
+ * crash/basebackup, even though the state of the data directory would
+ * require it. It's not possible (startup process doesn't have a PGXACT
+ * entry), and not needed, to do this during recovery, when performing an
+ * old-style truncation, though. There the entire scheduling depends on
+ * the replayed WAL records which be the same after a possible crash.
+ */
+ if (!in_recovery)
+ {
+ Assert(!MyPgXact->delayChkpt);
+ MyPgXact->delayChkpt = true;
+ }
+
+ /* WAL log truncation */
+ if (!in_recovery)
+ WriteMTruncateXlogRec(newOldestMultiDB,
+ oldestMulti, newOldestMulti,
+ oldestOffset, newOldestOffset);
+
+ /*
+ * Update in-memory limits before performing the truncation, while inside
+ * the critical section: Have to do it before truncation, to prevent
+ * concurrent lookups of those values. Has to be inside the critical
+ * section as otherwise a future call to this function would error out,
+ * while looking up the oldest member in offsets, if our caller crashes
+ * before updating the limits.
*/
- DetermineSafeOldestOffset(oldestMXact);
+ LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
+ MultiXactState->oldestMultiXactId = newOldestMulti;
+ MultiXactState->oldestMultiXactDB = newOldestMultiDB;
+ LWLockRelease(MultiXactGenLock);
+
+ /* First truncate members */
+ PerformMembersTruncation(oldestOffset, newOldestOffset);
+
+ /* Then offsets */
+ PerformOffsetsTruncation(oldestMulti, newOldestMulti);
+
+ if (!in_recovery)
+ MyPgXact->delayChkpt = false;
+
+ END_CRIT_SECTION();
+ LWLockRelease(MultiXactTruncationLock);
}
/*
(void) XLogInsert(RM_MULTIXACT_ID, info);
}
+/*
+ * Write a TRUNCATE xlog record
+ *
+ * We must flush the xlog record to disk before returning --- see notes in
+ * TruncateCLOG().
+ */
+static void
+WriteMTruncateXlogRec(Oid oldestMultiDB,
+ MultiXactId startTruncOff, MultiXactId endTruncOff,
+ MultiXactOffset startTruncMemb, MultiXactOffset endTruncMemb)
+{
+ XLogRecPtr recptr;
+ xl_multixact_truncate xlrec;
+
+ xlrec.oldestMultiDB = oldestMultiDB;
+
+ xlrec.startTruncOff = startTruncOff;
+ xlrec.endTruncOff = endTruncOff;
+
+ xlrec.startTruncMemb = startTruncMemb;
+ xlrec.endTruncMemb = endTruncMemb;
+
+ XLogBeginInsert();
+ XLogRegisterData((char *) (&xlrec), SizeOfMultiXactTruncate);
+ recptr = XLogInsert(RM_MULTIXACT_ID, XLOG_MULTIXACT_TRUNCATE_ID);
+ XLogFlush(recptr);
+}
+
/*
* MULTIXACT resource manager's routines
*/
LWLockRelease(XidGenLock);
}
}
+ else if (info == XLOG_MULTIXACT_TRUNCATE_ID)
+ {
+ xl_multixact_truncate xlrec;
+ int pageno;
+
+ memcpy(&xlrec, XLogRecGetData(record),
+ SizeOfMultiXactTruncate);
+
+ elog(DEBUG1, "replaying multixact truncation: "
+ "offsets [%u, %u), offsets segments [%x, %x), "
+ "members [%u, %u), members segments [%x, %x)",
+ xlrec.startTruncOff, xlrec.endTruncOff,
+ MultiXactIdToOffsetSegment(xlrec.startTruncOff),
+ MultiXactIdToOffsetSegment(xlrec.endTruncOff),
+ xlrec.startTruncMemb, xlrec.endTruncMemb,
+ MXOffsetToMemberSegment(xlrec.startTruncMemb),
+ MXOffsetToMemberSegment(xlrec.endTruncMemb));
+
+ /* should not be required, but more than cheap enough */
+ LWLockAcquire(MultiXactTruncationLock, LW_EXCLUSIVE);
+
+ /*
+ * Advance the horizon values, so they're current at the end of
+ * recovery.
+ */
+ SetMultiXactIdLimit(xlrec.endTruncOff, xlrec.oldestMultiDB);
+
+ PerformMembersTruncation(xlrec.startTruncMemb, xlrec.endTruncMemb);
+
+ /*
+ * During XLOG replay, latest_page_number isn't necessarily set up
+ * yet; insert a suitable value to bypass the sanity test in
+ * SimpleLruTruncate.
+ */
+ pageno = MultiXactIdToOffsetPage(xlrec.endTruncOff);
+ MultiXactOffsetCtl->shared->latest_page_number = pageno;
+ PerformOffsetsTruncation(xlrec.startTruncOff, xlrec.endTruncOff);
+
+ LWLockRelease(MultiXactTruncationLock);
+
+ /* only looked at in the startup process, no lock necessary */
+ MultiXactState->sawTruncationInCkptCycle = true;
+ }
else
elog(PANIC, "multixact_redo: unknown op code %u", info);
}