1
0
mirror of https://github.com/postgres/postgres.git synced 2025-11-10 17:42:29 +03:00

Invent ResourceOwner mechanism as per my recent proposal, and use it to

keep track of portal-related resources separately from transaction-related
resources.  This allows cursors to work in a somewhat sane fashion with
nested transactions.  For now, cursor behavior is non-subtransactional,
that is a cursor's state does not roll back if you abort a subtransaction
that fetched from the cursor.  We might want to change that later.
This commit is contained in:
Tom Lane
2004-07-17 03:32:14 +00:00
parent f4c069ca8f
commit fe548629c5
41 changed files with 2086 additions and 1192 deletions

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.172 2004/07/01 00:50:46 tgl Exp $
* $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.173 2004/07/17 03:28:49 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -45,8 +45,8 @@
#include "storage/bufpage.h"
#include "storage/proc.h"
#include "storage/smgr.h"
#include "utils/memutils.h"
#include "utils/relcache.h"
#include "utils/resowner.h"
#include "pgstat.h"
@@ -65,13 +65,9 @@ long NDirectFileRead; /* some I/O's are direct file access.
* bypass bufmgr */
long NDirectFileWrite; /* e.g., I/O in psort and hashjoin. */
/* List of upper-level-transaction buffer refcount arrays */
static List *upperRefCounts = NIL;
static void PinBuffer(BufferDesc *buf);
static void UnpinBuffer(BufferDesc *buf);
static void BufferFixLeak(Buffer bufnum, int32 shouldBe, bool emitWarning);
static void PinBuffer(BufferDesc *buf, bool fixOwner);
static void UnpinBuffer(BufferDesc *buf, bool fixOwner);
static void WaitIO(BufferDesc *buf);
static void StartBufferIO(BufferDesc *buf, bool forInput);
static void TerminateBufferIO(BufferDesc *buf, int err_flag);
@@ -103,6 +99,7 @@ static void write_buffer(Buffer buffer, bool unpin);
Buffer
ReadBuffer(Relation reln, BlockNumber blockNum)
{
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
return ReadBufferInternal(reln, blockNum, false);
}
@@ -111,6 +108,8 @@ ReadBuffer(Relation reln, BlockNumber blockNum)
*
* bufferLockHeld: if true, caller already acquired the bufmgr lock.
* (This is assumed never to be true if dealing with a local buffer!)
*
* The caller must have done ResourceOwnerEnlargeBuffers(CurrentResourceOwner)
*/
static Buffer
ReadBufferInternal(Relation reln, BlockNumber blockNum,
@@ -287,7 +286,7 @@ BufferAlloc(Relation reln,
*/
*foundPtr = TRUE;
PinBuffer(buf);
PinBuffer(buf, true);
if (!(buf->flags & BM_VALID))
{
@@ -337,6 +336,9 @@ BufferAlloc(Relation reln,
buf->refcount = 1;
PrivateRefCount[BufferDescriptorGetBuffer(buf) - 1] = 1;
ResourceOwnerRememberBuffer(CurrentResourceOwner,
BufferDescriptorGetBuffer(buf));
if ((buf->flags & BM_VALID) &&
(buf->flags & BM_DIRTY || buf->cntxDirty))
{
@@ -382,7 +384,7 @@ BufferAlloc(Relation reln,
* buffer we were planning to use.
*/
TerminateBufferIO(buf, 0);
UnpinBuffer(buf);
UnpinBuffer(buf, true);
buf = buf2;
@@ -390,7 +392,7 @@ BufferAlloc(Relation reln,
*foundPtr = TRUE;
PinBuffer(buf);
PinBuffer(buf, true);
if (!(buf->flags & BM_VALID))
{
@@ -425,7 +427,7 @@ BufferAlloc(Relation reln,
if (buf->refcount > 1 || buf->flags & BM_DIRTY || buf->cntxDirty)
{
TerminateBufferIO(buf, 0);
UnpinBuffer(buf);
UnpinBuffer(buf, true);
inProgress = FALSE;
buf = NULL;
}
@@ -497,7 +499,7 @@ write_buffer(Buffer buffer, bool release)
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
if (release)
UnpinBuffer(bufHdr);
UnpinBuffer(bufHdr, true);
LWLockRelease(BufMgrLock);
}
@@ -561,6 +563,8 @@ ReleaseAndReadBuffer(Buffer buffer,
if (bufHdr->tag.blockNum == blockNum &&
RelFileNodeEquals(bufHdr->tag.rnode, relation->rd_node))
return buffer;
ResourceOwnerForgetBuffer(CurrentResourceOwner, buffer);
/* owner now has a free slot, so no need for Enlarge() */
LocalRefCount[-buffer - 1]--;
}
else
@@ -570,16 +574,20 @@ ReleaseAndReadBuffer(Buffer buffer,
if (bufHdr->tag.blockNum == blockNum &&
RelFileNodeEquals(bufHdr->tag.rnode, relation->rd_node))
return buffer;
ResourceOwnerForgetBuffer(CurrentResourceOwner, buffer);
/* owner now has a free slot, so no need for Enlarge() */
if (PrivateRefCount[buffer - 1] > 1)
PrivateRefCount[buffer - 1]--;
else
{
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
UnpinBuffer(bufHdr);
UnpinBuffer(bufHdr, false);
return ReadBufferInternal(relation, blockNum, true);
}
}
}
else
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
return ReadBufferInternal(relation, blockNum, false);
}
@@ -589,9 +597,12 @@ ReleaseAndReadBuffer(Buffer buffer,
*
* This should be applied only to shared buffers, never local ones.
* Bufmgr lock must be held by caller.
*
* Most but not all callers want CurrentResourceOwner to be adjusted.
* Note that ResourceOwnerEnlargeBuffers must have been done already.
*/
static void
PinBuffer(BufferDesc *buf)
PinBuffer(BufferDesc *buf, bool fixOwner)
{
int b = BufferDescriptorGetBuffer(buf) - 1;
@@ -599,6 +610,9 @@ PinBuffer(BufferDesc *buf)
buf->refcount++;
PrivateRefCount[b]++;
Assert(PrivateRefCount[b] > 0);
if (fixOwner)
ResourceOwnerRememberBuffer(CurrentResourceOwner,
BufferDescriptorGetBuffer(buf));
}
/*
@@ -606,12 +620,18 @@ PinBuffer(BufferDesc *buf)
*
* This should be applied only to shared buffers, never local ones.
* Bufmgr lock must be held by caller.
*
* Most but not all callers want CurrentResourceOwner to be adjusted.
*/
static void
UnpinBuffer(BufferDesc *buf)
UnpinBuffer(BufferDesc *buf, bool fixOwner)
{
int b = BufferDescriptorGetBuffer(buf) - 1;
if (fixOwner)
ResourceOwnerForgetBuffer(CurrentResourceOwner,
BufferDescriptorGetBuffer(buf));
Assert(buf->refcount > 0);
Assert(PrivateRefCount[b] > 0);
PrivateRefCount[b]--;
@@ -677,6 +697,9 @@ BufferSync(int percent, int maxpages)
if (maxpages > 0 && num_buffer_dirty > maxpages)
num_buffer_dirty = maxpages;
/* Make sure we can handle the pin inside the loop */
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
/*
* Loop over buffers to be written. Note the BufMgrLock is held at
* loop top, but is released and reacquired within FlushBuffer,
@@ -724,13 +747,13 @@ BufferSync(int percent, int maxpages)
* buffer now and set IO state for it *before* acquiring shlock to
* avoid conflicts with FlushRelationBuffers.
*/
PinBuffer(bufHdr);
PinBuffer(bufHdr, true);
StartBufferIO(bufHdr, false);
FlushBuffer(bufHdr, NULL);
TerminateBufferIO(bufHdr, 0);
UnpinBuffer(bufHdr);
UnpinBuffer(bufHdr, true);
}
LWLockRelease(BufMgrLock);
@@ -831,104 +854,34 @@ AtEOXact_Buffers(bool isCommit)
for (i = 0; i < NBuffers; i++)
{
if (PrivateRefCount[i] != 0)
BufferFixLeak(i, 0, isCommit);
{
BufferDesc *buf = &(BufferDescriptors[i]);
if (isCommit)
elog(WARNING,
"buffer refcount leak: [%03d] "
"(rel=%u/%u/%u, blockNum=%u, flags=0x%x, refcount=%u %d)",
i,
buf->tag.rnode.spcNode, buf->tag.rnode.dbNode,
buf->tag.rnode.relNode,
buf->tag.blockNum, buf->flags,
buf->refcount, PrivateRefCount[i]);
/*
* We don't worry about updating the ResourceOwner structures;
* resowner.c will clear them for itself.
*/
PrivateRefCount[i] = 1; /* make sure we release shared pin */
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
UnpinBuffer(buf, false);
LWLockRelease(BufMgrLock);
Assert(PrivateRefCount[i] == 0);
}
}
AtEOXact_LocalBuffers(isCommit);
}
/*
* During subtransaction start, save buffer reference counts.
*/
void
AtSubStart_Buffers(void)
{
int32 *copyRefCounts;
Size rcSize;
MemoryContext old_cxt;
/* this is probably the active context already, but be safe */
old_cxt = MemoryContextSwitchTo(CurTransactionContext);
/*
* We need to copy the current state of PrivateRefCount[]. In the typical
* scenario, few if any of the entries will be nonzero, and we could save
* space by storing only the nonzero ones. However, copying the whole
* thing is lots simpler and faster both here and in AtEOSubXact_Buffers,
* so it seems best to waste the space.
*/
rcSize = NBuffers * sizeof(int32);
copyRefCounts = (int32 *) palloc(rcSize);
memcpy(copyRefCounts, PrivateRefCount, rcSize);
/* Attach to list */
upperRefCounts = lcons(copyRefCounts, upperRefCounts);
MemoryContextSwitchTo(old_cxt);
}
/*
* AtEOSubXact_Buffers
*
* At subtransaction end, we restore the saved counts. If committing, we
* complain if the refcounts don't match; if aborting, just restore silently.
*/
void
AtEOSubXact_Buffers(bool isCommit)
{
int32 *oldRefCounts;
int i;
oldRefCounts = (int32 *) linitial(upperRefCounts);
upperRefCounts = list_delete_first(upperRefCounts);
for (i = 0; i < NBuffers; i++)
{
if (PrivateRefCount[i] != oldRefCounts[i])
BufferFixLeak(i, oldRefCounts[i], isCommit);
}
pfree(oldRefCounts);
}
/*
* Fix a buffer refcount leak.
*
* The caller does not hold the BufMgrLock.
*/
static void
BufferFixLeak(Buffer bufnum, int32 shouldBe, bool emitWarning)
{
BufferDesc *buf = &(BufferDescriptors[bufnum]);
if (emitWarning)
elog(WARNING,
"buffer refcount leak: [%03d] (rel=%u/%u/%u, blockNum=%u, flags=0x%x, refcount=%u %d, should be=%d)",
bufnum,
buf->tag.rnode.spcNode, buf->tag.rnode.dbNode,
buf->tag.rnode.relNode,
buf->tag.blockNum, buf->flags,
buf->refcount, PrivateRefCount[bufnum], shouldBe);
/* If it's less, we're in a heap o' trouble */
if (PrivateRefCount[bufnum] <= shouldBe)
elog(FATAL, "buffer refcount was decreased by subtransaction");
if (shouldBe > 0)
{
/* We still keep the shared-memory pin */
PrivateRefCount[bufnum] = shouldBe;
}
else
{
PrivateRefCount[bufnum] = 1; /* make sure we release shared pin */
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
UnpinBuffer(buf);
LWLockRelease(BufMgrLock);
Assert(PrivateRefCount[bufnum] == 0);
}
}
/*
* FlushBufferPool
*
@@ -1172,9 +1125,15 @@ DropRelFileNodeBuffers(RelFileNode rnode, bool istemp,
if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
bufHdr->tag.blockNum >= firstDelBlock)
{
if (LocalRefCount[i] != 0)
elog(FATAL, "block %u of %u/%u/%u is still referenced (local %u)",
bufHdr->tag.blockNum,
bufHdr->tag.rnode.spcNode,
bufHdr->tag.rnode.dbNode,
bufHdr->tag.rnode.relNode,
LocalRefCount[i]);
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
bufHdr->cntxDirty = false;
LocalRefCount[i] = 0;
bufHdr->tag.rnode.relNode = InvalidOid;
}
}
@@ -1205,29 +1164,22 @@ recheck:
*/
goto recheck;
}
/*
* There should be no pin on the buffer.
*/
if (bufHdr->refcount != 0)
elog(FATAL, "block %u of %u/%u/%u is still referenced (private %d, global %u)",
bufHdr->tag.blockNum,
bufHdr->tag.rnode.spcNode,
bufHdr->tag.rnode.dbNode,
bufHdr->tag.rnode.relNode,
PrivateRefCount[i - 1], bufHdr->refcount);
/* Now we can do what we came for */
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
bufHdr->cntxDirty = false;
/*
* Release any refcount we may have. If someone else has a
* pin on the buffer, we got trouble.
*/
if (bufHdr->refcount != 0)
{
/* the sole pin should be ours */
if (bufHdr->refcount != 1 || PrivateRefCount[i - 1] == 0)
elog(FATAL, "block %u of %u/%u/%u is still referenced (private %d, global %u)",
bufHdr->tag.blockNum,
bufHdr->tag.rnode.spcNode,
bufHdr->tag.rnode.dbNode,
bufHdr->tag.rnode.relNode,
PrivateRefCount[i - 1], bufHdr->refcount);
/* Make sure it will be released */
PrivateRefCount[i - 1] = 1;
UnpinBuffer(bufHdr);
}
/*
* And mark the buffer as no longer occupied by this rel.
*/
@@ -1353,7 +1305,7 @@ PrintPinnedBufs(void)
for (i = 0; i < NBuffers; ++i, ++buf)
{
if (PrivateRefCount[i] > 0)
elog(WARNING,
elog(NOTICE,
"[%02d] (freeNext=%d, freePrev=%d, rel=%u/%u/%u, "
"blockNum=%u, flags=0x%x, refcount=%u %d)",
i, buf->freeNext, buf->freePrev,
@@ -1456,6 +1408,9 @@ FlushRelationBuffers(Relation rel, BlockNumber firstDelBlock)
return;
}
/* Make sure we can handle the pin inside the loop */
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
for (i = 0; i < NBuffers; i++)
@@ -1466,7 +1421,7 @@ FlushRelationBuffers(Relation rel, BlockNumber firstDelBlock)
if ((bufHdr->flags & BM_VALID) &&
(bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty))
{
PinBuffer(bufHdr);
PinBuffer(bufHdr, true);
/* Someone else might be flushing buffer */
if (bufHdr->flags & BM_IO_IN_PROGRESS)
WaitIO(bufHdr);
@@ -1479,7 +1434,7 @@ FlushRelationBuffers(Relation rel, BlockNumber firstDelBlock)
TerminateBufferIO(bufHdr, 0);
}
UnpinBuffer(bufHdr);
UnpinBuffer(bufHdr, true);
if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
elog(ERROR, "FlushRelationBuffers(\"%s\", %u): block %u was re-dirtied",
RelationGetRelationName(rel), firstDelBlock,
@@ -1507,6 +1462,8 @@ ReleaseBuffer(Buffer buffer)
{
BufferDesc *bufHdr;
ResourceOwnerForgetBuffer(CurrentResourceOwner, buffer);
if (BufferIsLocal(buffer))
{
Assert(LocalRefCount[-buffer - 1] > 0);
@@ -1526,11 +1483,39 @@ ReleaseBuffer(Buffer buffer)
else
{
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
UnpinBuffer(bufHdr);
UnpinBuffer(bufHdr, false);
LWLockRelease(BufMgrLock);
}
}
/*
* IncrBufferRefCount
* Increment the pin count on a buffer that we have *already* pinned
* at least once.
*
* This function cannot be used on a buffer we do not have pinned,
* because it doesn't change the shared buffer state. Therefore the
* Assert checks are for refcount > 0. Someone got this wrong once...
*/
void
IncrBufferRefCount(Buffer buffer)
{
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
ResourceOwnerRememberBuffer(CurrentResourceOwner, buffer);
if (BufferIsLocal(buffer))
{
Assert(buffer >= -NLocBuffer);
Assert(LocalRefCount[-buffer - 1] > 0);
LocalRefCount[-buffer - 1]++;
}
else
{
Assert(!BAD_BUFFER_ID(buffer));
Assert(PrivateRefCount[buffer - 1] > 0);
PrivateRefCount[buffer - 1]++;
}
}
#ifdef NOT_USED
void
IncrBufferRefCount_Debug(char *file, int line, Buffer buffer)

View File

@@ -9,7 +9,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/buffer/localbuf.c,v 1.56 2004/06/18 06:13:33 tgl Exp $
* $PostgreSQL: pgsql/src/backend/storage/buffer/localbuf.c,v 1.57 2004/07/17 03:28:49 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -19,6 +19,7 @@
#include "storage/bufmgr.h"
#include "storage/smgr.h"
#include "utils/relcache.h"
#include "utils/resowner.h"
/*#define LBDEBUG*/
@@ -62,6 +63,8 @@ LocalBufferAlloc(Relation reln, BlockNumber blockNum, bool *foundPtr)
#endif
LocalRefCount[i]++;
ResourceOwnerRememberBuffer(CurrentResourceOwner,
BufferDescriptorGetBuffer(bufHdr));
if (bufHdr->flags & BM_VALID)
*foundPtr = TRUE;
else
@@ -88,6 +91,8 @@ LocalBufferAlloc(Relation reln, BlockNumber blockNum, bool *foundPtr)
{
bufHdr = &LocalBufferDescriptors[b];
LocalRefCount[b]++;
ResourceOwnerRememberBuffer(CurrentResourceOwner,
BufferDescriptorGetBuffer(bufHdr));
nextFreeLocalBuf = (b + 1) % NLocBuffer;
break;
}
@@ -179,6 +184,7 @@ WriteLocalBuffer(Buffer buffer, bool release)
{
Assert(LocalRefCount[bufid] > 0);
LocalRefCount[bufid]--;
ResourceOwnerForgetBuffer(CurrentResourceOwner, buffer);
}
}

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/lmgr/lock.c,v 1.134 2004/07/01 00:50:59 tgl Exp $
* $PostgreSQL: pgsql/src/backend/storage/lmgr/lock.c,v 1.135 2004/07/17 03:28:51 tgl Exp $
*
* NOTES
* Outside modules can create a lock table and acquire/release
@@ -30,14 +30,15 @@
*/
#include "postgres.h"
#include <unistd.h>
#include <signal.h>
#include <unistd.h>
#include "access/xact.h"
#include "miscadmin.h"
#include "storage/proc.h"
#include "utils/memutils.h"
#include "utils/ps_status.h"
#include "utils/resowner.h"
/* This configuration variable is used to set the lock table size */
@@ -424,6 +425,9 @@ LockAcquire(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
/* ???????? This must be changed when short term locks will be used */
locktag->lockmethodid = lockmethodid;
/* Prepare to record the lock in the current resource owner */
ResourceOwnerEnlargeLocks(CurrentResourceOwner);
Assert(lockmethodid < NumLockMethods);
lockMethodTable = LockMethods[lockmethodid];
if (!lockMethodTable)
@@ -567,6 +571,8 @@ LockAcquire(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
if (proclock->holding[lockmode] > 0)
{
GrantLock(lock, proclock, lockmode);
ResourceOwnerRememberLock(CurrentResourceOwner, locktag, xid,
lockmode);
PROCLOCK_PRINT("LockAcquire: owning", proclock);
LWLockRelease(masterLock);
return TRUE;
@@ -580,6 +586,8 @@ LockAcquire(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
if (myHolding[lockmode] > 0)
{
GrantLock(lock, proclock, lockmode);
ResourceOwnerRememberLock(CurrentResourceOwner, locktag, xid,
lockmode);
PROCLOCK_PRINT("LockAcquire: my other XID owning", proclock);
LWLockRelease(masterLock);
return TRUE;
@@ -601,6 +609,8 @@ LockAcquire(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
{
/* No conflict with held or previously requested locks */
GrantLock(lock, proclock, lockmode);
ResourceOwnerRememberLock(CurrentResourceOwner, locktag, xid,
lockmode);
}
else
{
@@ -803,6 +813,9 @@ LockCountMyLocks(SHMEM_OFFSET lockOffset, PGPROC *proc, int *myHolding)
*
* NOTE: if proc was blocked, it also needs to be removed from the wait list
* and have its waitLock/waitHolder fields cleared. That's not done here.
*
* NOTE: the lock also has to be recorded in the current ResourceOwner;
* but since we may be awaking some other process, we can't do that here.
*/
void
GrantLock(LOCK *lock, PROCLOCK *proclock, LOCKMODE lockmode)
@@ -964,6 +977,9 @@ LockRelease(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
/* ???????? This must be changed when short term locks will be used */
locktag->lockmethodid = lockmethodid;
/* Record release of the lock in the current resource owner */
ResourceOwnerForgetLock(CurrentResourceOwner, locktag, xid, lockmode);
Assert(lockmethodid < NumLockMethods);
lockMethodTable = LockMethods[lockmethodid];
if (!lockMethodTable)
@@ -1134,20 +1150,15 @@ LockRelease(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
*
* Well, not necessarily *all* locks. The available behaviors are:
*
* which == ReleaseAll: release all locks regardless of transaction
* allxids == true: release all locks regardless of transaction
* affiliation.
*
* which == ReleaseAllExceptSession: release all locks with Xid != 0
* allxids == false: release all locks with Xid != 0
* (zero is the Xid used for "session" locks).
*
* which == ReleaseGivenXids: release only locks whose Xids appear in
* the xids[] array (of length nxids).
*
* xids/nxids are ignored when which != ReleaseGivenXids.
*/
bool
LockReleaseAll(LOCKMETHODID lockmethodid, PGPROC *proc,
LockReleaseWhich which, int nxids, TransactionId *xids)
bool allxids)
{
SHM_QUEUE *procHolders = &(proc->procHolders);
PROCLOCK *proclock;
@@ -1196,25 +1207,9 @@ LockReleaseAll(LOCKMETHODID lockmethodid, PGPROC *proc,
if (LOCK_LOCKMETHOD(*lock) != lockmethodid)
goto next_item;
if (which == ReleaseGivenXids)
{
/* Ignore locks with an Xid not in the list */
bool release = false;
for (i = 0; i < nxids; i++)
{
if (TransactionIdEquals(proclock->tag.xid, xids[i]))
{
release = true;
break;
}
}
if (!release)
goto next_item;
}
/* Ignore locks with Xid=0 unless we are asked to release All locks */
else if (TransactionIdEquals(proclock->tag.xid, InvalidTransactionId)
&& which != ReleaseAll)
/* Ignore locks with Xid=0 unless we are asked to release all locks */
if (TransactionIdEquals(proclock->tag.xid, InvalidTransactionId)
&& !allxids)
goto next_item;
PROCLOCK_PRINT("LockReleaseAll", proclock);

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.149 2004/07/01 00:50:59 tgl Exp $
* $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.150 2004/07/17 03:28:51 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -40,7 +40,6 @@
*/
#include "postgres.h"
#include <errno.h>
#include <signal.h>
#include <unistd.h>
#include <sys/time.h>
@@ -51,6 +50,8 @@
#include "storage/proc.h"
#include "storage/sinval.h"
#include "storage/spin.h"
#include "utils/resowner.h"
/* GUC variables */
int DeadlockTimeout = 1000;
@@ -75,6 +76,11 @@ static PGPROC *DummyProcs = NULL;
static bool waitingForLock = false;
static bool waitingForSignal = false;
/* Auxiliary state, valid when waitingForLock is true */
static LOCKTAG waitingForLockTag;
static TransactionId waitingForLockXid;
static LOCKMODE waitingForLockMode;
/* Mark these volatile because they can be changed by signal handler */
static volatile bool statement_timeout_active = false;
static volatile bool deadlock_timeout_active = false;
@@ -234,7 +240,7 @@ InitProcess(void)
* prepared for us by InitProcGlobal.
*/
SHMQueueElemInit(&(MyProc->links));
MyProc->errType = STATUS_OK;
MyProc->waitStatus = STATUS_OK;
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
MyProc->pid = MyProcPid;
@@ -308,7 +314,7 @@ InitDummyProcess(int proctype)
*/
MyProc->pid = MyProcPid; /* marks dummy proc as in use by me */
SHMQueueElemInit(&(MyProc->links));
MyProc->errType = STATUS_OK;
MyProc->waitStatus = STATUS_OK;
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
MyProc->databaseId = MyDatabaseId;
@@ -348,15 +354,40 @@ LockWaitCancel(void)
if (!waitingForLock)
return false;
waitingForLock = false;
/* Turn off the deadlock timer, if it's still running (see ProcSleep) */
disable_sig_alarm(false);
/* Unlink myself from the wait queue, if on it (might not be anymore!) */
LWLockAcquire(LockMgrLock, LW_EXCLUSIVE);
if (MyProc->links.next != INVALID_OFFSET)
{
/* We could not have been granted the lock yet */
Assert(MyProc->waitStatus == STATUS_ERROR);
RemoveFromWaitQueue(MyProc);
}
else
{
/*
* Somebody kicked us off the lock queue already. Perhaps they
* granted us the lock, or perhaps they detected a deadlock.
* If they did grant us the lock, we'd better remember it in
* CurrentResourceOwner.
*
* Exception: if CurrentResourceOwner is NULL then we can't do
* anything. This could only happen when we are invoked from ProcKill
* or some similar place, where all our locks are about to be released
* anyway.
*/
if (MyProc->waitStatus == STATUS_OK && CurrentResourceOwner != NULL)
ResourceOwnerRememberLock(CurrentResourceOwner,
&waitingForLockTag,
waitingForLockXid,
waitingForLockMode);
}
waitingForLock = false;
LWLockRelease(LockMgrLock);
/*
@@ -380,34 +411,29 @@ LockWaitCancel(void)
/*
* ProcReleaseLocks() -- release locks associated with current transaction
* at main transaction and subtransaction commit or abort
*
* The options for which locks to release are the same as for the underlying
* LockReleaseAll() function.
*
* Notes:
* at main transaction commit or abort
*
* At main transaction commit, we release all locks except session locks.
* At main transaction abort, we release all locks including session locks;
* this lets us clean up after a VACUUM FULL failure.
*
* At subtransaction commit, we don't release any locks (so this func is not
* called at all); we will defer the releasing to the parent transaction.
* needed at all); we will defer the releasing to the parent transaction.
* At subtransaction abort, we release all locks held by the subtransaction;
* this is implemented by passing in the Xids of the failed subxact and its
* children in the xids[] array.
* this is implemented by retail releasing of the locks under control of
* the ResourceOwner mechanism.
*
* Note that user locks are not released in any case.
*/
void
ProcReleaseLocks(LockReleaseWhich which, int nxids, TransactionId *xids)
ProcReleaseLocks(bool isCommit)
{
if (!MyProc)
return;
/* If waiting, get off wait queue (should only be needed after error) */
LockWaitCancel();
/* Release locks */
LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, which, nxids, xids);
LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, !isCommit);
}
@@ -440,11 +466,11 @@ ProcKill(int code, Datum arg)
LockWaitCancel();
/* Remove from the standard lock table */
LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, ReleaseAll, 0, NULL);
LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, true);
#ifdef USER_LOCKS
/* Remove from the user lock table */
LockReleaseAll(USER_LOCKMETHOD, MyProc, ReleaseAll, 0, NULL);
LockReleaseAll(USER_LOCKMETHOD, MyProc, true);
#endif
SpinLockAcquire(ProcStructLock);
@@ -618,6 +644,10 @@ ProcSleep(LockMethod lockMethodTable,
{
/* Skip the wait and just grant myself the lock. */
GrantLock(lock, proclock, lockmode);
ResourceOwnerRememberLock(CurrentResourceOwner,
&lock->tag,
proclock->tag.xid,
lockmode);
return STATUS_OK;
}
/* Break out of loop to put myself before him */
@@ -653,7 +683,7 @@ ProcSleep(LockMethod lockMethodTable,
MyProc->waitHolder = proclock;
MyProc->waitLockMode = lockmode;
MyProc->errType = STATUS_OK; /* initialize result for success */
MyProc->waitStatus = STATUS_ERROR; /* initialize result for error */
/*
* If we detected deadlock, give up without waiting. This must agree
@@ -663,11 +693,13 @@ ProcSleep(LockMethod lockMethodTable,
if (early_deadlock)
{
RemoveFromWaitQueue(MyProc);
MyProc->errType = STATUS_ERROR;
return STATUS_ERROR;
}
/* mark that we are waiting for a lock */
waitingForLockTag = lock->tag;
waitingForLockXid = proclock->tag.xid;
waitingForLockMode = lockmode;
waitingForLock = true;
/*
@@ -683,7 +715,7 @@ ProcSleep(LockMethod lockMethodTable,
/*
* Set timer so we can wake up after awhile and check for a deadlock.
* If a deadlock is detected, the handler releases the process's
* semaphore and sets MyProc->errType = STATUS_ERROR, allowing us to
* semaphore and sets MyProc->waitStatus = STATUS_ERROR, allowing us to
* know that we must report failure rather than success.
*
* By delaying the check until we've waited for a bit, we can avoid
@@ -703,8 +735,10 @@ ProcSleep(LockMethod lockMethodTable,
* We pass interruptOK = true, which eliminates a window in which
* cancel/die interrupts would be held off undesirably. This is a
* promise that we don't mind losing control to a cancel/die interrupt
* here. We don't, because we have no state-change work to do after
* being granted the lock (the grantor did it all).
* here. We don't, because we have no shared-state-change work to do
* after being granted the lock (the grantor did it all). We do have
* to worry about updating the local CurrentResourceOwner, but if we
* lose control to an error, LockWaitCancel will fix that up.
*/
PGSemaphoreLock(&MyProc->sem, true);
@@ -715,20 +749,32 @@ ProcSleep(LockMethod lockMethodTable,
elog(FATAL, "could not disable timer for process wakeup");
/*
* Now there is nothing for LockWaitCancel to do.
* Re-acquire the locktable's masterLock. We have to do this to hold
* off cancel/die interrupts before we can mess with waitingForLock
* (else we might have a missed or duplicated CurrentResourceOwner
* update).
*/
LWLockAcquire(masterLock, LW_EXCLUSIVE);
/*
* We no longer want LockWaitCancel to do anything.
*/
waitingForLock = false;
/*
* Re-acquire the locktable's masterLock.
* If we got the lock, be sure to remember it in CurrentResourceOwner.
*/
LWLockAcquire(masterLock, LW_EXCLUSIVE);
if (MyProc->waitStatus == STATUS_OK)
ResourceOwnerRememberLock(CurrentResourceOwner,
&lock->tag,
proclock->tag.xid,
lockmode);
/*
* We don't have to do anything else, because the awaker did all the
* necessary update of the lock table and MyProc.
*/
return MyProc->errType;
return MyProc->waitStatus;
}
@@ -743,7 +789,7 @@ ProcSleep(LockMethod lockMethodTable,
* to twiddle the lock's request counts too --- see RemoveFromWaitQueue.
*/
PGPROC *
ProcWakeup(PGPROC *proc, int errType)
ProcWakeup(PGPROC *proc, int waitStatus)
{
PGPROC *retProc;
@@ -764,7 +810,7 @@ ProcWakeup(PGPROC *proc, int errType)
/* Clean up process' state and pass it the ok/fail signal */
proc->waitLock = NULL;
proc->waitHolder = NULL;
proc->errType = errType;
proc->waitStatus = waitStatus;
/* And awaken it */
PGSemaphoreUnlock(&proc->sem);
@@ -891,10 +937,10 @@ CheckDeadLock(void)
RemoveFromWaitQueue(MyProc);
/*
* Set MyProc->errType to STATUS_ERROR so that ProcSleep will report
* Set MyProc->waitStatus to STATUS_ERROR so that ProcSleep will report
* an error after we return from the signal handler.
*/
MyProc->errType = STATUS_ERROR;
MyProc->waitStatus = STATUS_ERROR;
/*
* Unlock my semaphore so that the interrupted ProcSleep() call can

View File

@@ -11,12 +11,13 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/smgr/smgr.c,v 1.76 2004/07/11 19:52:51 tgl Exp $
* $PostgreSQL: pgsql/src/backend/storage/smgr/smgr.c,v 1.77 2004/07/17 03:28:55 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/xact.h"
#include "commands/tablespace.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
@@ -81,10 +82,15 @@ static HTAB *SMgrRelationHash = NULL;
* executed immediately, but is just entered in the list. When and if
* the transaction commits, we can delete the physical file.
*
* The list is kept in CurTransactionContext. In subtransactions, each
* subtransaction has its own list in its own CurTransactionContext, but
* successful subtransactions attach their lists to their parent's list.
* Failed subtransactions can immediately execute the abort-time actions.
* To handle subtransactions, every entry is marked with its transaction
* nesting level. At subtransaction commit, we reassign the subtransaction's
* entries to the parent nesting level. At subtransaction abort, we can
* immediately execute the abort-time actions for all entries of the current
* nesting level.
*
* NOTE: the list is kept in TopMemoryContext to be sure it won't disappear
* unbetimes. It'd probably be OK to keep it in TopTransactionContext,
* but I'm being paranoid.
*/
typedef struct PendingRelDelete
@@ -93,11 +99,11 @@ typedef struct PendingRelDelete
int which; /* which storage manager? */
bool isTemp; /* is it a temporary relation? */
bool atCommit; /* T=delete at commit; F=delete at abort */
int nestLevel; /* xact nesting level of request */
struct PendingRelDelete *next; /* linked-list link */
} PendingRelDelete;
static List *pendingDeletes = NIL; /* head of linked list */
static List *upperPendingDeletes = NIL; /* list of upper-xact lists */
static PendingRelDelete *pendingDeletes = NULL; /* head of linked list */
/*
@@ -308,7 +314,6 @@ smgrcreate(SMgrRelation reln, bool isTemp, bool isRedo)
XLogRecData rdata;
xl_smgr_create xlrec;
PendingRelDelete *pending;
MemoryContext old_cxt;
/*
* We may be using the target table space for the first time in this
@@ -349,17 +354,15 @@ smgrcreate(SMgrRelation reln, bool isTemp, bool isRedo)
lsn = XLogInsert(RM_SMGR_ID, XLOG_SMGR_CREATE | XLOG_NO_TRAN, &rdata);
/* Add the relation to the list of stuff to delete at abort */
old_cxt = MemoryContextSwitchTo(CurTransactionContext);
pending = (PendingRelDelete *) palloc(sizeof(PendingRelDelete));
pending = (PendingRelDelete *)
MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelDelete));
pending->relnode = reln->smgr_rnode;
pending->which = reln->smgr_which;
pending->isTemp = isTemp;
pending->atCommit = false; /* delete if abort */
pendingDeletes = lcons(pending, pendingDeletes);
MemoryContextSwitchTo(old_cxt);
pending->nestLevel = GetCurrentTransactionNestLevel();
pending->next = pendingDeletes;
pendingDeletes = pending;
}
/*
@@ -374,20 +377,17 @@ void
smgrscheduleunlink(SMgrRelation reln, bool isTemp)
{
PendingRelDelete *pending;
MemoryContext old_cxt;
/* Add the relation to the list of stuff to delete at commit */
old_cxt = MemoryContextSwitchTo(CurTransactionContext);
pending = (PendingRelDelete *) palloc(sizeof(PendingRelDelete));
pending = (PendingRelDelete *)
MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelDelete));
pending->relnode = reln->smgr_rnode;
pending->which = reln->smgr_which;
pending->isTemp = isTemp;
pending->atCommit = true; /* delete if commit */
pendingDeletes = lcons(pending, pendingDeletes);
MemoryContextSwitchTo(old_cxt);
pending->nestLevel = GetCurrentTransactionNestLevel();
pending->next = pendingDeletes;
pendingDeletes = pending;
/*
* NOTE: if the relation was created in this transaction, it will now
@@ -647,25 +647,45 @@ smgrimmedsync(SMgrRelation reln)
/*
* smgrDoPendingDeletes() -- Take care of relation deletes at end of xact.
*
* This also runs when aborting a subxact; we want to clean up a failed
* subxact immediately.
*/
void
smgrDoPendingDeletes(bool isCommit)
{
ListCell *p;
int nestLevel = GetCurrentTransactionNestLevel();
PendingRelDelete *pending;
PendingRelDelete *prev;
PendingRelDelete *next;
foreach(p, pendingDeletes)
prev = NULL;
for (pending = pendingDeletes; pending != NULL; pending = next)
{
PendingRelDelete *pending = lfirst(p);
if (pending->atCommit == isCommit)
smgr_internal_unlink(pending->relnode,
pending->which,
pending->isTemp,
false);
next = pending->next;
if (pending->nestLevel < nestLevel)
{
/* outer-level entries should not be processed yet */
prev = pending;
}
else
{
/* unlink list entry first, so we don't retry on failure */
if (prev)
prev->next = next;
else
pendingDeletes = next;
/* do deletion if called for */
if (pending->atCommit == isCommit)
smgr_internal_unlink(pending->relnode,
pending->which,
pending->isTemp,
false);
/* must explicitly free the list entry */
pfree(pending);
/* prev does not change */
}
}
/* We needn't free the cells since they are in CurTransactionContext */
pendingDeletes = NIL;
}
/*
@@ -681,16 +701,15 @@ smgrDoPendingDeletes(bool isCommit)
int
smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
{
int nestLevel = GetCurrentTransactionNestLevel();
int nrels;
RelFileNode *rptr;
ListCell *p;
PendingRelDelete *pending;
nrels = 0;
foreach(p, pendingDeletes)
for (pending = pendingDeletes; pending != NULL; pending = pending->next)
{
PendingRelDelete *pending = lfirst(p);
if (pending->atCommit == forCommit)
if (pending->nestLevel >= nestLevel && pending->atCommit == forCommit)
nrels++;
}
if (nrels == 0)
@@ -700,50 +719,30 @@ smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
}
rptr = (RelFileNode *) palloc(nrels * sizeof(RelFileNode));
*ptr = rptr;
foreach(p, pendingDeletes)
for (pending = pendingDeletes; pending != NULL; pending = pending->next)
{
PendingRelDelete *pending = lfirst(p);
if (pending->atCommit == forCommit)
if (pending->nestLevel >= nestLevel && pending->atCommit == forCommit)
*rptr++ = pending->relnode;
}
return nrels;
}
/*
* AtSubStart_smgr() --- Take care of subtransaction start.
*
* Push empty state for the new subtransaction.
*/
void
AtSubStart_smgr(void)
{
MemoryContext old_cxt;
/* Keep the list-of-lists in TopTransactionContext for simplicity */
old_cxt = MemoryContextSwitchTo(TopTransactionContext);
upperPendingDeletes = lcons(pendingDeletes, upperPendingDeletes);
pendingDeletes = NIL;
MemoryContextSwitchTo(old_cxt);
}
/*
* AtSubCommit_smgr() --- Take care of subtransaction commit.
*
* Reassign all items in the pending deletes list to the parent transaction.
* Reassign all items in the pending-deletes list to the parent transaction.
*/
void
AtSubCommit_smgr(void)
{
List *parentPendingDeletes;
int nestLevel = GetCurrentTransactionNestLevel();
PendingRelDelete *pending;
parentPendingDeletes = (List *) linitial(upperPendingDeletes);
upperPendingDeletes = list_delete_first(upperPendingDeletes);
pendingDeletes = list_concat(parentPendingDeletes, pendingDeletes);
for (pending = pendingDeletes; pending != NULL; pending = pending->next)
{
if (pending->nestLevel >= nestLevel)
pending->nestLevel = nestLevel - 1;
}
}
/*
@@ -757,10 +756,6 @@ void
AtSubAbort_smgr(void)
{
smgrDoPendingDeletes(false);
/* Must pop the stack, too */
pendingDeletes = (List *) linitial(upperPendingDeletes);
upperPendingDeletes = list_delete_first(upperPendingDeletes);
}
/*