postgresql/src/backend/access/transam/multixact.c

1558 lines
46 KiB
C
Raw Normal View History

/*-------------------------------------------------------------------------
*
* multixact.c
* PostgreSQL multi-transaction-log manager
*
* The pg_multixact manager is a pg_clog-like manager that stores an array
* of TransactionIds for each MultiXactId. It is a fundamental part of the
* shared-row-lock implementation. A share-locked tuple stores a
* MultiXactId in its Xmax, and a transaction that needs to wait for the
* tuple to be unlocked can sleep on the potentially-several TransactionIds
* that compose the MultiXactId.
*
* We use two SLRU areas, one for storing the offsets on which the data
* starts for each MultiXactId in the other one. This trick allows us to
* store variable length arrays of TransactionIds. (We could alternatively
* use one area containing counts and TransactionIds, with valid MultiXactId
* values pointing at slots containing counts; but that way seems less robust
* since it would get completely confused if someone inquired about a bogus
* MultiXactId that pointed to an intermediate slot containing an XID.)
*
* This code is based on subtrans.c; see it for additional discussion.
* Like the subtransaction manager, we only need to remember multixact
* information for currently-open transactions. Thus, there is
* no need to preserve data over a crash and restart.
*
* The only XLOG interaction we need to take care of is that generated
* MultiXactId values must continue to increase across a system crash.
* Thus we log groups of MultiXactIds acquisition in the same fashion we do
* for Oids (see XLogPutNextMultiXactId).
*
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/backend/access/transam/multixact.c,v 1.1 2005/04/28 21:47:10 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/multixact.h"
#include "access/slru.h"
#include "access/xact.h"
#include "miscadmin.h"
#include "utils/memutils.h"
#include "storage/backendid.h"
#include "storage/lmgr.h"
#include "storage/sinval.h"
/*
* Defines for MultiXactOffset page sizes. A page is the same BLCKSZ as is
* used everywhere else in Postgres.
*
* Note: because both uint32 and TransactionIds are 32 bits and wrap around at
* 0xFFFFFFFF, MultiXact page numbering also wraps around at
* 0xFFFFFFFF/MULTIXACT_*_PER_PAGE, and segment numbering at
* 0xFFFFFFFF/MULTIXACT_*_PER_PAGE/SLRU_SEGMENTS_PER_PAGE. We need take no
* explicit notice of that fact in this module, except when comparing segment
* and page numbers in TruncateMultiXact
* (see MultiXact{Offset,Member}PagePrecedes).
*/
/* We need four bytes per offset and also four bytes per member */
#define MULTIXACT_OFFSETS_PER_PAGE (BLCKSZ / sizeof(uint32))
#define MULTIXACT_MEMBERS_PER_PAGE (BLCKSZ / sizeof(TransactionId))
#define MultiXactIdToOffsetPage(xid) \
((xid) / (uint32) MULTIXACT_OFFSETS_PER_PAGE)
#define MultiXactIdToOffsetEntry(xid) \
((xid) % (uint32) MULTIXACT_OFFSETS_PER_PAGE)
#define MXOffsetToMemberPage(xid) \
((xid) / (TransactionId) MULTIXACT_MEMBERS_PER_PAGE)
#define MXOffsetToMemberEntry(xid) \
((xid) % (TransactionId) MULTIXACT_MEMBERS_PER_PAGE)
/* Arbitrary number of MultiXactIds to allocate at each XLog call */
#define MXACT_PREFETCH 8192
/*
* Links to shared-memory data structures for MultiXact control
*/
static SlruCtlData MultiXactOffsetCtlData;
static SlruCtlData MultiXactMemberCtlData;
#define MultiXactOffsetCtl (&MultiXactOffsetCtlData)
#define MultiXactMemberCtl (&MultiXactMemberCtlData)
/*
* MultiXact state shared across all backends. All this state is protected
* by MultiXactGenLock. (We also use MultiXactOffsetControlLock and
* MultiXactMemberControlLock to guard accesses to the two sets of SLRU
* buffers. For concurrency's sake, we avoid holding more than one of these
* locks at a time.)
*/
typedef struct MultiXactStateData
{
/* next-to-be-assigned MultiXactId */
MultiXactId nextMXact;
/* MultiXactIds we have left before logging more */
uint32 mXactCount;
/* next-to-be-assigned offset */
uint32 nextOffset;
/* the Offset SLRU area was last truncated at this MultiXactId */
MultiXactId lastTruncationPoint;
/*
* Per-backend data starts here. We have two arrays stored in
* the area immediately following the MultiXactStateData struct.
* Each is indexed by BackendId. (Note: valid BackendIds run from 1 to
* MaxBackends; element zero of each array is never used.)
*
* OldestMemberMXactId[k] is the oldest MultiXactId each backend's
* current transaction(s) could possibly be a member of, or
* InvalidMultiXactId when the backend has no live transaction that
* could possibly be a member of a MultiXact. Each backend sets its
* entry to the current nextMXact counter just before first acquiring a
* shared lock in a given transaction, and clears it at transaction end.
* (This works because only during or after acquiring a shared lock
* could an XID possibly become a member of a MultiXact, and that
* MultiXact would have to be created during or after the lock
* acquisition.)
*
* OldestVisibleMXactId[k] is the oldest MultiXactId each backend's
* current transaction(s) think is potentially live, or InvalidMultiXactId
* when not in a transaction or not in a transaction that's paid any
* attention to MultiXacts yet. This is computed when first needed in
* a given transaction, and cleared at transaction end. We can compute
* it as the minimum of the valid OldestMemberMXactId[] entries at the
* time we compute it (using nextMXact if none are valid). Each backend
* is required not to attempt to access any SLRU data for MultiXactIds
* older than its own OldestVisibleMXactId[] setting; this is necessary
* because the checkpointer could truncate away such data at any instant.
*
* The checkpointer can compute the safe truncation point as the oldest
* valid value among all the OldestMemberMXactId[] and
* OldestVisibleMXactId[] entries, or nextMXact if none are valid.
* Clearly, it is not possible for any later-computed OldestVisibleMXactId
* value to be older than this, and so there is no risk of truncating
* data that is still needed.
*/
MultiXactId perBackendXactIds[1]; /* VARIABLE LENGTH ARRAY */
} MultiXactStateData;
/* Pointers to the state data in shared memory */
static MultiXactStateData *MultiXactState;
static MultiXactId *OldestMemberMXactId;
static MultiXactId *OldestVisibleMXactId;
/*
* Definitions for the backend-local MultiXactId cache.
*
* We use this cache to store known MultiXacts, so we don't need to go to
* SLRU areas everytime.
*
* The cache lasts for the duration of a single transaction, the rationale
* for this being that most entries will contain our own TransactionId and
* so they will be uninteresting by the time our next transaction starts.
* (XXX not clear that this is correct --- other members of the MultiXact
* could hang around longer than we did.)
*
* We allocate the cache entries in a memory context that is deleted at
* transaction end, so we don't need to do retail freeing of entries.
*/
typedef struct mXactCacheEnt
{
struct mXactCacheEnt *next;
MultiXactId multi;
int nxids;
TransactionId xids[1]; /* VARIABLE LENGTH ARRAY */
} mXactCacheEnt;
static mXactCacheEnt *MXactCache = NULL;
static MemoryContext MXactContext = NULL;
#ifdef MULTIXACT_DEBUG
#define debug_elog2(a,b) elog(a,b)
#define debug_elog3(a,b,c) elog(a,b,c)
#define debug_elog4(a,b,c,d) elog(a,b,c,d)
#define debug_elog5(a,b,c,d,e) elog(a,b,c,d,e)
#else
#define debug_elog2(a,b)
#define debug_elog3(a,b,c)
#define debug_elog4(a,b,c,d)
#define debug_elog5(a,b,c,d,e)
#endif
/* internal MultiXactId management */
static void MultiXactIdSetOldestVisible(void);
static MultiXactId CreateMultiXactId(int nxids, TransactionId *xids);
static int GetMultiXactIdMembers(MultiXactId multi, TransactionId **xids);
static MultiXactId GetNewMultiXactId(int nxids, uint32 *offset);
/* MultiXact cache management */
static MultiXactId mXactCacheGetBySet(int nxids, TransactionId *xids);
static int mXactCacheGetById(MultiXactId multi, TransactionId **xids);
static void mXactCachePut(MultiXactId multi, int nxids, TransactionId *xids);
static int xidComparator(const void *arg1, const void *arg2);
#ifdef MULTIXACT_DEBUG
static char *mxid_to_string(MultiXactId multi, int nxids, TransactionId *xids);
#endif
/* management of SLRU infrastructure */
static int ZeroMultiXactOffsetPage(int pageno);
static int ZeroMultiXactMemberPage(int pageno);
static bool MultiXactOffsetPagePrecedes(int page1, int page2);
static bool MultiXactMemberPagePrecedes(int page1, int page2);
static bool MultiXactIdPrecedes(MultiXactId multi1, MultiXactId multi2);
static bool MultiXactOffsetPrecedes(uint32 offset1, uint32 offset2);
static void ExtendMultiXactOffset(MultiXactId multi);
static void ExtendMultiXactMember(uint32 offset);
static void TruncateMultiXact(void);
/*
* MultiXactIdExpand
* Add a TransactionId to a possibly-already-existing MultiXactId.
*
* We abuse the notation for the first argument: if "isMulti" is true, then
* it's really a MultiXactId; else it's a TransactionId. We are already
* storing MultiXactId in HeapTupleHeader's xmax so assuming the datatypes
* are equivalent is necessary anyway.
*
* If isMulti is true, then get the members of the passed MultiXactId, add
* the passed TransactionId, and create a new MultiXactId. If isMulti is
* false, then take the two TransactionIds and create a new MultiXactId with
* them. The caller must ensure that the multi and xid are different
* in the latter case.
*
* If the TransactionId is already a member of the passed MultiXactId,
* just return it as-is.
*
* Note that we do NOT actually modify the membership of a pre-existing
* MultiXactId; instead we create a new one. This is necessary to avoid
* a race condition against MultiXactIdWait (see notes there).
*
* NB - we don't worry about our local MultiXactId cache here, because that
* is handled by the lower-level routines.
*/
MultiXactId
MultiXactIdExpand(MultiXactId multi, bool isMulti, TransactionId xid)
{
MultiXactId newMulti;
TransactionId *members;
TransactionId *newMembers;
int nmembers;
int i;
int j;
AssertArg(MultiXactIdIsValid(multi));
AssertArg(TransactionIdIsValid(xid));
debug_elog5(DEBUG2, "Expand: received %s %u, xid %u",
isMulti ? "MultiXactId" : "TransactionId",
multi, xid);
if (!isMulti)
{
/*
* The first argument is a TransactionId, not a MultiXactId.
*/
TransactionId xids[2];
Assert(!TransactionIdEquals(multi, xid));
xids[0] = multi;
xids[1] = xid;
newMulti = CreateMultiXactId(2, xids);
debug_elog5(DEBUG2, "Expand: returning %u two-elem %u/%u",
newMulti, multi, xid);
return newMulti;
}
nmembers = GetMultiXactIdMembers(multi, &members);
if (nmembers < 0)
{
/*
* The MultiXactId is obsolete. This can only happen if all the
* MultiXactId members stop running between the caller checking and
* passing it to us. It would be better to return that fact to the
* caller, but it would complicate the API and it's unlikely to happen
* too often, so just deal with it by creating a singleton MultiXact.
*/
newMulti = CreateMultiXactId(1, &xid);
debug_elog4(DEBUG2, "Expand: %u has no members, create singleton %u",
multi, newMulti);
return newMulti;
}
/*
* If the TransactionId is already a member of the MultiXactId,
* just return the existing MultiXactId.
*/
for (i = 0; i < nmembers; i++)
{
if (TransactionIdEquals(members[i], xid))
{
pfree(members);
debug_elog4(DEBUG2, "Expand: %u is already a member of %u",
xid, multi);
return multi;
}
}
/*
* Determine which of the members of the MultiXactId are still running,
* and use them to create a new one. (Removing dead members is just
* an optimization, but a useful one. Note we have the same race
* condition here as above: j could be 0 at the end of the loop.)
*/
newMembers = (TransactionId *)
palloc(sizeof(TransactionId) * (nmembers + 1));
for (i = 0, j = 0; i < nmembers; i++)
{
if (TransactionIdIsInProgress(members[i]))
newMembers[j++] = members[i];
}
newMembers[j++] = xid;
newMulti = CreateMultiXactId(j, newMembers);
pfree(members);
pfree(newMembers);
debug_elog3(DEBUG2, "Expand: returning new multi %u", newMulti);
return newMulti;
}
/*
* MultiXactIdIsRunning
* Returns whether a MultiXactId is "running".
*
* We return true if at least one member of the given MultiXactId is still
* running. Note that a "false" result is certain not to change,
* because it is not legal to add members to an existing MultiXactId.
*/
bool
MultiXactIdIsRunning(MultiXactId multi)
{
TransactionId *members;
TransactionId myXid;
int nmembers;
int i;
debug_elog3(DEBUG2, "IsRunning %u?", multi);
nmembers = GetMultiXactIdMembers(multi, &members);
if (nmembers < 0)
{
debug_elog2(DEBUG2, "IsRunning: no members");
return false;
}
/* checking for myself is cheap */
myXid = GetTopTransactionId();
for (i = 0; i < nmembers; i++)
{
if (TransactionIdEquals(members[i], myXid))
{
pfree(members);
debug_elog3(DEBUG2, "IsRunning: I (%d) am running!", i);
return true;
}
}
/*
* This could be made better by having a special entry point in sinval.c,
* walking the PGPROC array only once for the whole array. But in most
* cases nmembers should be small enough that it doesn't much matter.
*/
for (i = 0; i < nmembers; i++)
{
if (TransactionIdIsInProgress(members[i]))
{
pfree(members);
debug_elog4(DEBUG2, "IsRunning: member %d (%u) is running",
i, members[i]);
return true;
}
}
pfree(members);
debug_elog3(DEBUG2, "IsRunning: %u is not running", multi);
return false;
}
/*
* MultiXactIdSetOldestMember
* Save the oldest MultiXactId this transaction could be a member of.
*
* We set the OldestMemberMXactId for a given transaction the first time
* it's going to acquire a shared lock. We need to do this even if we end
* up using a TransactionId instead of a MultiXactId, because there is a
* chance that another transaction would add our XID to a MultiXactId.
*
* The value to set is the next-to-be-assigned MultiXactId, so this is meant
* to be called just before acquiring a shared lock.
*/
void
MultiXactIdSetOldestMember(void)
{
if (!MultiXactIdIsValid(OldestMemberMXactId[MyBackendId]))
{
MultiXactId nextMXact;
/*
* You might think we don't need to acquire a lock here, since
* fetching and storing of TransactionIds is probably atomic,
* but in fact we do: suppose we pick up nextMXact and then
* lose the CPU for a long time. Someone else could advance
* nextMXact, and then another someone else could compute an
* OldestVisibleMXactId that would be after the value we are
* going to store when we get control back. Which would be wrong.
*/
LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
/*
* We have to beware of the possibility that nextMXact is in the
* wrapped-around state. We don't fix the counter itself here,
* but we must be sure to store a valid value in our array entry.
*/
nextMXact = MultiXactState->nextMXact;
if (nextMXact < FirstMultiXactId)
nextMXact = FirstMultiXactId;
OldestMemberMXactId[MyBackendId] = nextMXact;
LWLockRelease(MultiXactGenLock);
debug_elog4(DEBUG2, "MultiXact: setting OldestMember[%d] = %u",
MyBackendId, nextMXact);
}
}
/*
* MultiXactIdSetOldestVisible
* Save the oldest MultiXactId this transaction considers possibly live.
*
* We set the OldestVisibleMXactId for a given transaction the first time
* it's going to inspect any MultiXactId. Once we have set this, we are
* guaranteed that the checkpointer won't truncate off SLRU data for
* MultiXactIds at or after our OldestVisibleMXactId.
*
* The value to set is the oldest of nextMXact and all the valid per-backend
* OldestMemberMXactId[] entries. Because of the locking we do, we can be
* certain that no subsequent call to MultiXactIdSetOldestMember can set
* an OldestMemberMXactId[] entry older than what we compute here. Therefore
* there is no live transaction, now or later, that can be a member of any
* MultiXactId older than the OldestVisibleMXactId we compute here.
*/
static void
MultiXactIdSetOldestVisible(void)
{
if (!MultiXactIdIsValid(OldestVisibleMXactId[MyBackendId]))
{
MultiXactId oldestMXact;
int i;
LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
/*
* We have to beware of the possibility that nextMXact is in the
* wrapped-around state. We don't fix the counter itself here,
* but we must be sure to store a valid value in our array entry.
*/
oldestMXact = MultiXactState->nextMXact;
if (oldestMXact < FirstMultiXactId)
oldestMXact = FirstMultiXactId;
for (i = 1; i <= MaxBackends; i++)
{
MultiXactId thisoldest = OldestMemberMXactId[i];
if (MultiXactIdIsValid(thisoldest) &&
MultiXactIdPrecedes(thisoldest, oldestMXact))
oldestMXact = thisoldest;
}
OldestVisibleMXactId[MyBackendId] = oldestMXact;
LWLockRelease(MultiXactGenLock);
debug_elog4(DEBUG2, "MultiXact: setting OldestVisible[%d] = %u",
MyBackendId, oldestMXact);
}
}
/*
* MultiXactIdWait
* Sleep on a MultiXactId.
*
* We do this by sleeping on each member using XactLockTableWait. Any
* members that belong to the current backend are *not* waited for, however;
* this would not merely be useless but would lead to Assert failure inside
* XactLockTableWait. By the time this returns, it is certain that all
* transactions *of other backends* that were members of the MultiXactId
* are dead (and no new ones can have been added, since it is not legal
* to add members to an existing MultiXactId).
*
* But by the time we finish sleeping, someone else may have changed the Xmax
* of the containing tuple, so the caller needs to iterate on us somehow.
*/
void
MultiXactIdWait(MultiXactId multi)
{
TransactionId *members;
int nmembers;
nmembers = GetMultiXactIdMembers(multi, &members);
if (nmembers >= 0)
{
int i;
for (i = 0; i < nmembers; i++)
{
TransactionId member = members[i];
debug_elog4(DEBUG2, "MultiXactIdWait: waiting for %d (%u)",
i, member);
if (!TransactionIdIsCurrentTransactionId(member))
XactLockTableWait(member);
}
pfree(members);
}
}
/*
* CreateMultiXactId
* Make a new MultiXactId
*
* Make SLRU and cache entries for a new MultiXactId, recording the given
* TransactionIds as members. Returns the newly created MultiXactId.
*
* NB: the passed xids[] array will be sorted in-place.
*/
static MultiXactId
CreateMultiXactId(int nxids, TransactionId *xids)
{
MultiXactId multi;
int pageno;
int prev_pageno;
int entryno;
int slotno;
uint32 *offptr;
uint32 offset;
int i;
debug_elog3(DEBUG2, "Create: %s",
mxid_to_string(InvalidMultiXactId, nxids, xids));
/*
* See if the same set of XIDs already exists in our cache; if so, just
* re-use that MultiXactId. (Note: it might seem that looking in our
* cache is insufficient, and we ought to search disk to see if a
* duplicate definition already exists. But since we only ever create
* MultiXacts containing our own XID, in most cases any such MultiXacts
* were in fact created by us, and so will be in our cache. There are
* corner cases where someone else added us to a MultiXact without our
* knowledge, but it's not worth checking for.)
*/
multi = mXactCacheGetBySet(nxids, xids);
if (MultiXactIdIsValid(multi))
{
debug_elog2(DEBUG2, "Create: in cache!");
return multi;
}
multi = GetNewMultiXactId(nxids, &offset);
LWLockAcquire(MultiXactOffsetControlLock, LW_EXCLUSIVE);
ExtendMultiXactOffset(multi);
pageno = MultiXactIdToOffsetPage(multi);
entryno = MultiXactIdToOffsetEntry(multi);
/*
* Note: we pass the MultiXactId to SimpleLruReadPage as the "transaction"
* to complain about if there's any I/O error. This is kinda bogus, but
* since the errors will always give the full pathname, it should be
* clear enough that a MultiXactId is really involved. Perhaps someday
* we'll take the trouble to generalize the slru.c error reporting code.
*/
slotno = SimpleLruReadPage(MultiXactOffsetCtl, pageno, multi);
offptr = (uint32 *) MultiXactOffsetCtl->shared->page_buffer[slotno];
offptr += entryno;
*offptr = offset;
MultiXactOffsetCtl->shared->page_status[slotno] = SLRU_PAGE_DIRTY;
/* Exchange our lock */
LWLockRelease(MultiXactOffsetControlLock);
debug_elog3(DEBUG2, "Create: got offset %u", offset);
LWLockAcquire(MultiXactMemberControlLock, LW_EXCLUSIVE);
prev_pageno = -1;
for (i = 0; i < nxids; i++, offset++)
{
TransactionId *memberptr;
ExtendMultiXactMember(offset);
pageno = MXOffsetToMemberPage(offset);
entryno = MXOffsetToMemberEntry(offset);
if (pageno != prev_pageno)
{
slotno = SimpleLruReadPage(MultiXactMemberCtl, pageno, multi);
prev_pageno = pageno;
}
memberptr = (TransactionId *)
MultiXactMemberCtl->shared->page_buffer[slotno];
memberptr += entryno;
*memberptr = xids[i];
MultiXactMemberCtl->shared->page_status[slotno] = SLRU_PAGE_DIRTY;
}
LWLockRelease(MultiXactMemberControlLock);
/* Store the new MultiXactId in the local cache, too */
mXactCachePut(multi, nxids, xids);
debug_elog2(DEBUG2, "Create: all done");
return multi;
}
/*
* GetNewMultiXactId
* Get the next MultiXactId.
*
* Get the next MultiXactId, XLogging if needed. Also, reserve the needed
* amount of space in the "members" area. The starting offset of the
* reserved space is returned in *offset.
*/
static MultiXactId
GetNewMultiXactId(int nxids, uint32 *offset)
{
MultiXactId result;
debug_elog3(DEBUG2, "GetNew: for %d xids", nxids);
/* MultiXactIdSetOldestMember() must have been called already */
Assert(MultiXactIdIsValid(OldestMemberMXactId[MyBackendId]));
LWLockAcquire(MultiXactGenLock, LW_EXCLUSIVE);
/* Handle wraparound of the nextMXact counter */
if (MultiXactState->nextMXact < FirstMultiXactId)
{
MultiXactState->nextMXact = FirstMultiXactId;
MultiXactState->mXactCount = 0;
}
/* If we run out of logged for use multixacts then we must log more */
if (MultiXactState->mXactCount == 0)
{
XLogPutNextMultiXactId(MultiXactState->nextMXact + MXACT_PREFETCH);
MultiXactState->mXactCount = MXACT_PREFETCH;
}
result = MultiXactState->nextMXact;
/*
* We don't care about MultiXactId wraparound here; it will be handled by
* the next iteration. But note that nextMXact may be InvalidMultiXactId
* after this routine exits, so anyone else looking at the variable must
* be prepared to deal with that.
*/
(MultiXactState->nextMXact)++;
(MultiXactState->mXactCount)--;
/*
* Reserve the members space.
*/
*offset = MultiXactState->nextOffset;
MultiXactState->nextOffset += nxids;
LWLockRelease(MultiXactGenLock);
debug_elog4(DEBUG2, "GetNew: returning %u offset %u", result, *offset);
return result;
}
/*
* GetMultiXactIdMembers
* Returns the set of TransactionIds that make up a MultiXactId
*
* We return -1 if the MultiXactId is too old to possibly have any members
* still running; in that case we have not actually looked them up, and
* *xids is not set.
*/
static int
GetMultiXactIdMembers(MultiXactId multi, TransactionId **xids)
{
int pageno;
int prev_pageno;
int entryno;
int slotno;
uint32 *offptr;
uint32 offset;
int length;
int i;
MultiXactId nextMXact;
MultiXactId tmpMXact;
uint32 nextOffset;
TransactionId *ptr;
debug_elog3(DEBUG2, "GetMembers: asked for %u", multi);
Assert(MultiXactIdIsValid(multi));
/* See if the MultiXactId is in the local cache */
length = mXactCacheGetById(multi, xids);
if (length >= 0)
{
debug_elog3(DEBUG2, "GetMembers: found %s in the cache",
mxid_to_string(multi, length, *xids));
return length;
}
/* Set our OldestVisibleMXactId[] entry if we didn't already */
MultiXactIdSetOldestVisible();
/*
* We check known limits on MultiXact before resorting to the SLRU area.
*
* An ID older than our OldestVisibleMXactId[] entry can't possibly still
* be running, and we'd run the risk of trying to read already-truncated
* SLRU data if we did try to examine it.
*
* Conversely, an ID >= nextMXact shouldn't ever be seen here; if it is
* seen, it implies undetected ID wraparound has occurred. We just
* silently assume that such an ID is no longer running.
*
* Shared lock is enough here since we aren't modifying any global state.
* Also, we can examine our own OldestVisibleMXactId without the lock,
* since no one else is allowed to change it.
*/
if (MultiXactIdPrecedes(multi, OldestVisibleMXactId[MyBackendId]))
{
debug_elog2(DEBUG2, "GetMembers: it's too old");
*xids = NULL;
return -1;
}
LWLockAcquire(MultiXactGenLock, LW_SHARED);
if (!MultiXactIdPrecedes(multi, MultiXactState->nextMXact))
{
LWLockRelease(MultiXactGenLock);
debug_elog2(DEBUG2, "GetMembers: it's too new!");
*xids = NULL;
return -1;
}
/*
* Before releasing the lock, save the current counter values, because
* the target MultiXactId may be just one less than nextMXact. We will
* need to use nextOffset as the endpoint if so.
*/
nextMXact = MultiXactState->nextMXact;
nextOffset = MultiXactState->nextOffset;
LWLockRelease(MultiXactGenLock);
/* Get the offset at which we need to start reading MultiXactMembers */
LWLockAcquire(MultiXactOffsetControlLock, LW_EXCLUSIVE);
pageno = MultiXactIdToOffsetPage(multi);
entryno = MultiXactIdToOffsetEntry(multi);
slotno = SimpleLruReadPage(MultiXactOffsetCtl, pageno, multi);
offptr = (uint32 *) MultiXactOffsetCtl->shared->page_buffer[slotno];
offptr += entryno;
offset = *offptr;
/*
* How many members do we need to read? If we are at the end of the
* assigned MultiXactIds, use the offset just saved above. Else we
* need to check the MultiXactId following ours.
*
* Use the same increment rule as GetNewMultiXactId(), that is, don't
* handle wraparound explicitly until needed.
*/
tmpMXact = multi + 1;
if (nextMXact == tmpMXact)
length = nextOffset - offset;
else
{
/* handle wraparound if needed */
if (tmpMXact < FirstMultiXactId)
tmpMXact = FirstMultiXactId;
prev_pageno = pageno;
pageno = MultiXactIdToOffsetPage(tmpMXact);
entryno = MultiXactIdToOffsetEntry(tmpMXact);
if (pageno != prev_pageno)
slotno = SimpleLruReadPage(MultiXactOffsetCtl, pageno, tmpMXact);
offptr = (uint32 *) MultiXactOffsetCtl->shared->page_buffer[slotno];
offptr += entryno;
length = *offptr - offset;
}
LWLockRelease(MultiXactOffsetControlLock);
ptr = (TransactionId *) palloc(length * sizeof(TransactionId));
*xids = ptr;
/* Now get the members themselves. */
LWLockAcquire(MultiXactMemberControlLock, LW_EXCLUSIVE);
prev_pageno = -1;
for (i = 0; i < length; i++, offset++)
{
TransactionId *xactptr;
pageno = MXOffsetToMemberPage(offset);
entryno = MXOffsetToMemberEntry(offset);
if (pageno != prev_pageno)
{
slotno = SimpleLruReadPage(MultiXactMemberCtl, pageno, multi);
prev_pageno = pageno;
}
xactptr = (TransactionId *)
MultiXactMemberCtl->shared->page_buffer[slotno];
xactptr += entryno;
ptr[i] = *xactptr;
}
LWLockRelease(MultiXactMemberControlLock);
/*
* Copy the result into the local cache.
*/
mXactCachePut(multi, length, ptr);
debug_elog3(DEBUG2, "GetMembers: no cache for %s",
mxid_to_string(multi, length, ptr));
return length;
}
/*
* mXactCacheGetBySet
* returns a MultiXactId from the cache based on the set of
* TransactionIds that compose it, or InvalidMultiXactId if
* none matches.
*
* This is helpful, for example, if two transactions want to lock a huge
* table. By using the cache, the second will use the same MultiXactId
* for the majority of tuples, thus keeping MultiXactId usage low (saving
* both I/O and wraparound issues).
*
* NB: the passed xids[] array will be sorted in-place.
*/
static MultiXactId
mXactCacheGetBySet(int nxids, TransactionId *xids)
{
mXactCacheEnt *entry;
debug_elog3(DEBUG2, "CacheGet: looking for %s",
mxid_to_string(InvalidMultiXactId, nxids, xids));
/* sort the array so comparison is easy */
qsort(xids, nxids, sizeof(TransactionId), xidComparator);
for (entry = MXactCache; entry != NULL; entry = entry->next)
{
if (entry->nxids != nxids)
continue;
/* We assume the cache entries are sorted */
if (memcmp(xids, entry->xids, nxids * sizeof(TransactionId)) == 0)
{
debug_elog3(DEBUG2, "CacheGet: found %u", entry->multi);
return entry->multi;
}
}
debug_elog2(DEBUG2, "CacheGet: not found :-(");
return InvalidMultiXactId;
}
/*
* mXactCacheGetById
* returns the composing TransactionId set from the cache for a
* given MultiXactId, if present.
*
* If successful, *xids is set to the address of a palloc'd copy of the
* TransactionId set. Return value is number of members, or -1 on failure.
*/
static int
mXactCacheGetById(MultiXactId multi, TransactionId **xids)
{
mXactCacheEnt *entry;
debug_elog3(DEBUG2, "CacheGet: looking for %u", multi);
for (entry = MXactCache; entry != NULL; entry = entry->next)
{
if (entry->multi == multi)
{
TransactionId *ptr;
Size size;
size = sizeof(TransactionId) * entry->nxids;
ptr = (TransactionId *) palloc(size);
*xids = ptr;
memcpy(ptr, entry->xids, size);
debug_elog3(DEBUG2, "CacheGet: found %s",
mxid_to_string(multi, entry->nxids, entry->xids));
return entry->nxids;
}
}
debug_elog2(DEBUG2, "CacheGet: not found");
return -1;
}
/*
* mXactCachePut
* Add a new MultiXactId and its composing set into the local cache.
*/
static void
mXactCachePut(MultiXactId multi, int nxids, TransactionId *xids)
{
mXactCacheEnt *entry;
debug_elog3(DEBUG2, "CachePut: storing %s",
mxid_to_string(multi, nxids, xids));
if (MXactContext == NULL)
{
/* The cache only lives as long as the current transaction */
debug_elog2(DEBUG2, "CachePut: initializing memory context");
MXactContext = AllocSetContextCreate(TopTransactionContext,
"MultiXact Cache Context",
ALLOCSET_SMALL_MINSIZE,
ALLOCSET_SMALL_INITSIZE,
ALLOCSET_SMALL_MAXSIZE);
}
entry = (mXactCacheEnt *)
MemoryContextAlloc(MXactContext,
offsetof(mXactCacheEnt, xids) +
nxids * sizeof(TransactionId));
entry->multi = multi;
entry->nxids = nxids;
memcpy(entry->xids, xids, nxids * sizeof(TransactionId));
/* mXactCacheGetBySet assumes the entries are sorted, so sort them */
qsort(entry->xids, nxids, sizeof(TransactionId), xidComparator);
entry->next = MXactCache;
MXactCache = entry;
}
/*
* xidComparator
* qsort comparison function for XIDs
*
* We don't need to use wraparound comparison for XIDs, and indeed must
* not do so since that does not respect the triangle inequality! Any
* old sort order will do.
*/
static int
xidComparator(const void *arg1, const void *arg2)
{
TransactionId xid1 = * (const TransactionId *) arg1;
TransactionId xid2 = * (const TransactionId *) arg2;
if (xid1 > xid2)
return 1;
if (xid1 < xid2)
return -1;
return 0;
}
#ifdef MULTIXACT_DEBUG
static char *
mxid_to_string(MultiXactId multi, int nxids, TransactionId *xids)
{
char *str = palloc(15 * (nxids + 1) + 4);
int i;
snprintf(str, 47, "%u %d[%u", multi, nxids, xids[0]);
for (i = 1; i < nxids; i++)
snprintf(str + strlen(str), 17, ", %u", xids[i]);
strcat(str, "]");
return str;
}
#endif
/*
* AtEOXact_MultiXact
* Handle transaction end for MultiXact
*
* This is called at top transaction commit or abort (we don't care which).
*/
void
AtEOXact_MultiXact(void)
{
/*
* Reset our OldestMemberMXactId and OldestVisibleMXactId values,
* both of which should only be valid while within a transaction.
*
* We assume that storing a MultiXactId is atomic and so we need
* not take MultiXactGenLock to do this.
*/
OldestMemberMXactId[MyBackendId] = InvalidMultiXactId;
OldestVisibleMXactId[MyBackendId] = InvalidMultiXactId;
/*
* Discard the local MultiXactId cache. Since MXactContext was created
* as a child of TopTransactionContext, we needn't delete it explicitly.
*/
MXactContext = NULL;
MXactCache = NULL;
}
/*
* Initialization of shared memory for MultiXact. We use two SLRU areas,
* thus double memory. Also, reserve space for the shared MultiXactState
* struct and the per-backend MultiXactId arrays (two of those, too).
*/
int
MultiXactShmemSize(void)
{
#define SHARED_MULTIXACT_STATE_SIZE \
(sizeof(MultiXactStateData) + sizeof(MultiXactId) * 2 * MaxBackends)
return (SimpleLruShmemSize() * 2 + SHARED_MULTIXACT_STATE_SIZE);
}
void
MultiXactShmemInit(void)
{
bool found;
debug_elog2(DEBUG2, "Shared Memory Init for MultiXact");
MultiXactOffsetCtl->PagePrecedes = MultiXactOffsetPagePrecedes;
MultiXactMemberCtl->PagePrecedes = MultiXactMemberPagePrecedes;
SimpleLruInit(MultiXactOffsetCtl, "MultiXactOffset Ctl",
MultiXactOffsetControlLock, "pg_multixact/offsets");
SimpleLruInit(MultiXactMemberCtl, "MultiXactMember Ctl",
MultiXactMemberControlLock, "pg_multixact/members");
/* Override default assumption that writes should be fsync'd */
MultiXactOffsetCtl->do_fsync = false;
MultiXactMemberCtl->do_fsync = false;
/* Initialize our shared state struct */
MultiXactState = ShmemInitStruct("Shared MultiXact State",
SHARED_MULTIXACT_STATE_SIZE,
&found);
if (!IsUnderPostmaster)
{
Assert(!found);
/* Make sure we zero out the per-backend state */
MemSet(MultiXactState, 0, SHARED_MULTIXACT_STATE_SIZE);
}
else
Assert(found);
/*
* Set up array pointers. Note that perBackendXactIds[0] is wasted
* space since we only use indexes 1..MaxBackends in each array.
*/
OldestMemberMXactId = MultiXactState->perBackendXactIds;
OldestVisibleMXactId = OldestMemberMXactId + MaxBackends;
}
/*
* This func must be called ONCE on system install. It creates the initial
* MultiXact segments. (The MultiXacts directories are assumed to have been
* created by initdb, and MultiXactShmemInit must have been called already.)
*
* Note: it's not really necessary to create the initial segments now,
* since slru.c would create 'em on first write anyway. But we may as well
* do it to be sure the directories are set up correctly.
*/
void
BootStrapMultiXact(void)
{
int slotno;
LWLockAcquire(MultiXactOffsetControlLock, LW_EXCLUSIVE);
/* Offsets first page */
slotno = ZeroMultiXactOffsetPage(0);
SimpleLruWritePage(MultiXactOffsetCtl, slotno, NULL);
Assert(MultiXactOffsetCtl->shared->page_status[slotno] == SLRU_PAGE_CLEAN);
LWLockRelease(MultiXactOffsetControlLock);
LWLockAcquire(MultiXactMemberControlLock, LW_EXCLUSIVE);
/* Members first page */
slotno = ZeroMultiXactMemberPage(0);
SimpleLruWritePage(MultiXactMemberCtl, slotno, NULL);
Assert(MultiXactMemberCtl->shared->page_status[slotno] == SLRU_PAGE_CLEAN);
LWLockRelease(MultiXactMemberControlLock);
}
/*
* Initialize (or reinitialize) a page of MultiXactOffset to zeroes.
*
* The page is not actually written, just set up in shared memory.
* The slot number of the new page is returned.
*
* Control lock must be held at entry, and will be held at exit.
*/
static int
ZeroMultiXactOffsetPage(int pageno)
{
return SimpleLruZeroPage(MultiXactOffsetCtl, pageno);
}
/*
* Ditto, for MultiXactMember
*/
static int
ZeroMultiXactMemberPage(int pageno)
{
return SimpleLruZeroPage(MultiXactMemberCtl, pageno);
}
/*
* This must be called ONCE during postmaster or standalone-backend startup.
*
* StartupXLOG has already established nextMXact by calling
* MultiXactSetNextMXact and/or MultiXactAdvanceNextMXact.
*
* 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
StartupMultiXact(void)
{
int startPage;
int cutoffPage;
uint32 offset;
/*
* We start nextOffset at zero after every reboot; there is no need to
* avoid offset values that were used in the previous system lifecycle.
*/
MultiXactState->nextOffset = 0;
/*
* Because of the above, a shutdown and restart is likely to leave
* high-numbered MultiXactMember page files that would not get recycled
* for a long time (about as long as the system had been up in the
* previous cycle of life). To clean out such page files, we issue an
* artificial truncation call that will zap any page files in the first
* half of the offset cycle. Should there be any page files in the last
* half, they will get cleaned out by the first checkpoint.
*
* XXX it might be a good idea to disable this when debugging, since it
* will tend to destroy evidence after a crash. To not be *too* ruthless,
* we arbitrarily spare the first 64 pages. (Note this will get
* rounded off to a multiple of SLRU_PAGES_PER_SEGMENT ...)
*/
offset = ((~ (uint32) 0) >> 1) + 1;
cutoffPage = MXOffsetToMemberPage(offset) + 64;
/*
* Defeat safety interlock in SimpleLruTruncate; this hack will be
* cleaned up by ZeroMultiXactMemberPage call below.
*/
MultiXactMemberCtl->shared->latest_page_number = cutoffPage;
SimpleLruTruncate(MultiXactMemberCtl, cutoffPage);
/*
* Initialize lastTruncationPoint to invalid, ensuring that the first
* checkpoint will try to do truncation.
*/
MultiXactState->lastTruncationPoint = InvalidMultiXactId;
/*
* Since we don't expect MultiXact to be valid across crashes, we
* initialize the currently-active pages to zeroes during startup.
* Whenever we advance into a new page, both ExtendMultiXact routines
* will likewise zero the new page without regard to whatever was
* previously on disk.
*/
LWLockAcquire(MultiXactOffsetControlLock, LW_EXCLUSIVE);
startPage = MultiXactIdToOffsetPage(MultiXactState->nextMXact);
(void) ZeroMultiXactOffsetPage(startPage);
LWLockRelease(MultiXactOffsetControlLock);
LWLockAcquire(MultiXactMemberControlLock, LW_EXCLUSIVE);
startPage = MXOffsetToMemberPage(MultiXactState->nextOffset);
(void) ZeroMultiXactMemberPage(startPage);
LWLockRelease(MultiXactMemberControlLock);
}
/*
* This must be called ONCE during postmaster or standalone-backend shutdown
*/
void
ShutdownMultiXact(void)
{
/*
* Flush dirty MultiXact pages to disk
*
* This is not actually necessary from a correctness point of view. We do
* it merely as a debugging aid.
*/
SimpleLruFlush(MultiXactOffsetCtl, false);
SimpleLruFlush(MultiXactMemberCtl, false);
}
/*
* Get the next MultiXactId to save in a checkpoint record
*/
MultiXactId
MultiXactGetCheckptMulti(bool is_shutdown)
{
MultiXactId retval;
LWLockAcquire(MultiXactGenLock, LW_SHARED);
retval = MultiXactState->nextMXact;
if (!is_shutdown)
retval += MultiXactState->mXactCount;
LWLockRelease(MultiXactGenLock);
debug_elog3(DEBUG2, "MultiXact: MultiXact for checkpoint record is %u",
retval);
return retval;
}
/*
* Perform a checkpoint --- either during shutdown, or on-the-fly
*/
void
CheckPointMultiXact(void)
{
/*
* Flush dirty MultiXact pages to disk
*
* This is not actually necessary from a correctness point of view. We do
* it merely to improve the odds that writing of dirty pages is done
* by the checkpoint process and not by backends.
*/
SimpleLruFlush(MultiXactOffsetCtl, true);
SimpleLruFlush(MultiXactMemberCtl, true);
/*
* Truncate the SLRU files
*/
TruncateMultiXact();
}
/*
* Set the next-to-be-assigned MultiXactId
*
* This is used when we can determine the correct next Id exactly
* from an XLog record. We need no locking since it is only called
* during bootstrap and XLog replay.
*/
void
MultiXactSetNextMXact(MultiXactId nextMulti)
{
debug_elog3(DEBUG2, "MultiXact: setting next multi to %u", nextMulti);
MultiXactState->nextMXact = nextMulti;
MultiXactState->mXactCount = 0;
}
/*
* Ensure the next-to-be-assigned MultiXactId is at least minMulti
*
* This is used when we can determine a minimum safe value
* from an XLog record. We need no locking since it is only called
* during XLog replay.
*/
void
MultiXactAdvanceNextMXact(MultiXactId minMulti)
{
if (MultiXactIdPrecedes(MultiXactState->nextMXact, minMulti))
{
debug_elog3(DEBUG2, "MultiXact: setting next multi to %u", minMulti);
MultiXactState->nextMXact = minMulti;
MultiXactState->mXactCount = 0;
}
}
/*
* Make sure that MultiXactOffset has room for a newly-allocated MultiXactId.
*
* The MultiXactOffsetControlLock should be held at entry, and will
* be held at exit.
*/
void
ExtendMultiXactOffset(MultiXactId multi)
{
int pageno;
/*
* No work except at first MultiXactId of a page. But beware: just after
* wraparound, the first MultiXactId of page zero is FirstMultiXactId.
*/
if (MultiXactIdToOffsetEntry(multi) != 0 &&
multi != FirstMultiXactId)
return;
pageno = MultiXactIdToOffsetPage(multi);
/* Zero the page */
ZeroMultiXactOffsetPage(pageno);
}
/*
* Make sure that MultiXactMember has room for the members of a newly-
* allocated MultiXactId.
*
* The MultiXactMemberControlLock should be held at entry, and will be held
* at exit.
*/
void
ExtendMultiXactMember(uint32 offset)
{
int pageno;
/*
* No work except at first entry of a page.
*/
if (MXOffsetToMemberEntry(offset) != 0)
return;
pageno = MXOffsetToMemberPage(offset);
/* Zero the page */
ZeroMultiXactMemberPage(pageno);
}
/*
* Remove all MultiXactOffset and MultiXactMember segments before the oldest
* ones still of interest.
*
* This is called only during checkpoints. We assume no more than one
* backend does this at a time.
*/
static void
TruncateMultiXact(void)
{
MultiXactId nextMXact;
uint32 nextOffset;
MultiXactId oldestMXact;
uint32 oldestOffset;
int cutoffPage;
int i;
/*
* First, compute where we can safely truncate. Per notes above,
* this is the oldest valid value among all the OldestMemberMXactId[] and
* OldestVisibleMXactId[] entries, or nextMXact if none are valid.
*/
LWLockAcquire(MultiXactGenLock, LW_SHARED);
/*
* We have to beware of the possibility that nextMXact is in the
* wrapped-around state. We don't fix the counter itself here,
* but we must be sure to use a valid value in our calculation.
*/
nextMXact = MultiXactState->nextMXact;
if (nextMXact < FirstMultiXactId)
nextMXact = FirstMultiXactId;
oldestMXact = nextMXact;
for (i = 1; i <= MaxBackends; i++)
{
MultiXactId thisoldest;
thisoldest = OldestMemberMXactId[i];
if (MultiXactIdIsValid(thisoldest) &&
MultiXactIdPrecedes(thisoldest, oldestMXact))
oldestMXact = thisoldest;
thisoldest = OldestVisibleMXactId[i];
if (MultiXactIdIsValid(thisoldest) &&
MultiXactIdPrecedes(thisoldest, oldestMXact))
oldestMXact = thisoldest;
}
/* Save the current nextOffset too */
nextOffset = MultiXactState->nextOffset;
LWLockRelease(MultiXactGenLock);
debug_elog3(DEBUG2, "MultiXact: truncation point = %u", oldestMXact);
/*
* If we already truncated at this point, do nothing. This saves time
* when no MultiXacts are getting used, which is probably not uncommon.
*/
if (MultiXactState->lastTruncationPoint == oldestMXact)
return;
/*
* We need to determine where to truncate MultiXactMember. If we
* found a valid oldest MultiXactId, read its starting offset;
* otherwise we use the nextOffset value we saved above.
*/
if (oldestMXact == nextMXact)
oldestOffset = nextOffset;
else
{
int pageno;
int slotno;
int entryno;
uint32 *offptr;
LWLockAcquire(MultiXactOffsetControlLock, LW_EXCLUSIVE);
pageno = MultiXactIdToOffsetPage(oldestMXact);
entryno = MultiXactIdToOffsetEntry(oldestMXact);
slotno = SimpleLruReadPage(MultiXactOffsetCtl, pageno, oldestMXact);
offptr = (uint32 *) MultiXactOffsetCtl->shared->page_buffer[slotno];
offptr += entryno;
oldestOffset = *offptr;
LWLockRelease(MultiXactOffsetControlLock);
}
/*
* The cutoff point is the start of the segment containing oldestMXact.
* We pass the *page* containing oldestMXact to SimpleLruTruncate.
*/
cutoffPage = MultiXactIdToOffsetPage(oldestMXact);
SimpleLruTruncate(MultiXactOffsetCtl, cutoffPage);
/*
* Also truncate MultiXactMember at the previously determined offset.
*/
cutoffPage = MXOffsetToMemberPage(oldestOffset);
SimpleLruTruncate(MultiXactMemberCtl, cutoffPage);
/*
* Set the last known truncation point. We don't need a lock for this
* since only one backend does checkpoints at a time.
*/
MultiXactState->lastTruncationPoint = oldestMXact;
}
/*
* Decide which of two MultiXactOffset page numbers is "older" for truncation
* purposes.
*
* We need to use comparison of MultiXactId here in order to do the right
* thing with wraparound. However, if we are asked about page number zero, we
* don't want to hand InvalidMultiXactId to MultiXactIdPrecedes: it'll get
* weird. So, offset both multis by FirstMultiXactId to avoid that.
* (Actually, the current implementation doesn't do anything weird with
* InvalidMultiXactId, but there's no harm in leaving this code like this.)
*/
static bool
MultiXactOffsetPagePrecedes(int page1, int page2)
{
MultiXactId multi1;
MultiXactId multi2;
multi1 = ((MultiXactId) page1) * MULTIXACT_OFFSETS_PER_PAGE;
multi1 += FirstMultiXactId;
multi2 = ((MultiXactId) page2) * MULTIXACT_OFFSETS_PER_PAGE;
multi2 += FirstMultiXactId;
return MultiXactIdPrecedes(multi1, multi2);
}
/*
* Decide which of two MultiXactMember page numbers is "older" for truncation
* purposes. There is no "invalid offset number" so use the numbers verbatim.
*/
static bool
MultiXactMemberPagePrecedes(int page1, int page2)
{
uint32 offset1;
uint32 offset2;
offset1 = ((uint32) page1) * MULTIXACT_MEMBERS_PER_PAGE;
offset2 = ((uint32) page2) * MULTIXACT_MEMBERS_PER_PAGE;
return MultiXactOffsetPrecedes(offset1, offset2);
}
/*
* Decide which of two MultiXactIds is earlier.
*
* XXX do we need to do something special for InvalidMultiXactId?
* (Doesn't look like it.)
*/
static bool
MultiXactIdPrecedes(MultiXactId multi1, MultiXactId multi2)
{
int32 diff = (int32) (multi1 - multi2);
return (diff < 0);
}
/*
* Decide which of two offsets is earlier.
*/
static bool
MultiXactOffsetPrecedes(uint32 offset1, uint32 offset2)
{
int32 diff = (int32) (offset1 - offset2);
return (diff < 0);
}