Commit 55432fed authored by Tom Lane's avatar Tom Lane

Implement LockBufferForCleanup(), which will allow concurrent VACUUM

to wait until it's safe to remove tuples and compact free space in a
shared buffer page.  Miscellaneous small code cleanups in bufmgr, too.
parent 1e9e5def
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/access/transam/xact.c,v 1.104 2001/06/22 19:16:21 wieck Exp $
* $Header: /cvsroot/pgsql/src/backend/access/transam/xact.c,v 1.105 2001/07/06 21:04:25 tgl Exp $
*
* NOTES
* Transaction aborts can now occur two ways:
......@@ -653,7 +653,7 @@ void
RecordTransactionCommit()
{
TransactionId xid;
int leak;
bool leak;
xid = GetCurrentTransactionId();
......
$Header: /cvsroot/pgsql/src/backend/storage/buffer/README,v 1.1 2001/07/06 21:04:25 tgl Exp $
Notes about shared buffer access rules
--------------------------------------
There are two separate access control mechanisms for shared disk buffers:
reference counts (a/k/a pin counts) and buffer locks. (Actually, there's
a third level of access control: one must hold the appropriate kind of
lock on a relation before one can legally access any page belonging to
the relation. Relation-level locks are not discussed here.)
Pins: one must "hold a pin on" a buffer (increment its reference count)
before being allowed to do anything at all with it. An unpinned buffer is
subject to being reclaimed and reused for a different page at any instant,
so touching it is unsafe. Typically a pin is acquired via ReadBuffer and
released via WriteBuffer (if one modified the page) or ReleaseBuffer (if not).
It is OK and indeed common for a single backend to pin a page more than
once concurrently; the buffer manager handles this efficiently. It is
considered OK to hold a pin for long intervals --- for example, sequential
scans hold a pin on the current page until done processing all the tuples
on the page, which could be quite a while if the scan is the outer scan of
a join. Similarly, btree index scans hold a pin on the current index page.
This is OK because normal operations never wait for a page's pin count to
drop to zero. (Anything that might need to do such a wait is instead
handled by waiting to obtain the relation-level lock, which is why you'd
better hold one first.) Pins may not be held across transaction
boundaries, however.
Buffer locks: there are two kinds of buffer locks, shared and exclusive,
which act just as you'd expect: multiple backends can hold shared locks on
the same buffer, but an exclusive lock prevents anyone else from holding
either shared or exclusive lock. (These can alternatively be called READ
and WRITE locks.) These locks are short-term: they should not be held for
long. They are implemented as per-buffer spinlocks, so another backend
trying to acquire a competing lock will spin as long as you hold yours!
Buffer locks are acquired and released by LockBuffer(). It will *not* work
for a single backend to try to acquire multiple locks on the same buffer.
One must pin a buffer before trying to lock it.
Buffer access rules:
1. To scan a page for tuples, one must hold a pin and either shared or
exclusive lock. To examine the commit status (XIDs and status bits) of
a tuple in a shared buffer, one must likewise hold a pin and either shared
or exclusive lock.
2. Once one has determined that a tuple is interesting (visible to the
current transaction) one may drop the buffer lock, yet continue to access
the tuple's data for as long as one holds the buffer pin. This is what is
typically done by heap scans, since the tuple returned by heap_fetch
contains a pointer to tuple data in the shared buffer. Therefore the
tuple cannot go away while the pin is held (see rule #5). Its state could
change, but that is assumed not to matter after the initial determination
of visibility is made.
3. To add a tuple or change the xmin/xmax fields of an existing tuple,
one must hold a pin and an exclusive lock on the containing buffer.
This ensures that no one else might see a partially-updated state of the
tuple.
4. It is considered OK to update tuple commit status bits (ie, OR the
values HEAP_XMIN_COMMITTED, HEAP_XMIN_INVALID, HEAP_XMAX_COMMITTED, or
HEAP_XMAX_INVALID into t_infomask) while holding only a shared lock and
pin on a buffer. This is OK because another backend looking at the tuple
at about the same time would OR the same bits into the field, so there
is little or no risk of conflicting update; what's more, if there did
manage to be a conflict it would merely mean that one bit-update would
be lost and need to be done again later. These four bits are only hints
(they cache the results of transaction status lookups in pg_log), so no
great harm is done if they get reset to zero by conflicting updates.
5. To physically remove a tuple or compact free space on a page, one
must hold a pin and an exclusive lock, *and* observe while holding the
exclusive lock that the buffer's shared reference count is one (ie,
no other backend holds a pin). If these conditions are met then no other
backend can perform a page scan until the exclusive lock is dropped, and
no other backend can be holding a reference to an existing tuple that it
might expect to examine again. Note that another backend might pin the
buffer (increment the refcount) while one is performing the cleanup, but
it won't be able to actually examine the page until it acquires shared
or exclusive lock.
As of 7.1, the only operation that removes tuples or compacts free space is
(oldstyle) VACUUM. It does not have to implement rule #5 directly, because
it instead acquires exclusive lock at the relation level, which ensures
indirectly that no one else is accessing pages of the relation at all.
To implement concurrent VACUUM we will need to make it obey rule #5 fully.
To do this, we'll create a new buffer manager operation
LockBufferForCleanup() that gets an exclusive lock and then checks to see
if the shared pin count is currently 1. If not, it releases the exclusive
lock (but not the caller's pin) and waits until signaled by another backend,
whereupon it tries again. The signal will occur when UnpinBuffer
decrements the shared pin count to 1. As indicated above, this operation
might have to wait a good while before it acquires lock, but that shouldn't
matter much for concurrent VACUUM. The current implementation only
supports a single waiter for pin-count-1 on any particular shared buffer.
This is enough for VACUUM's use, since we don't allow multiple VACUUMs
concurrently on a single relation anyway.
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/buffer/buf_init.c,v 1.42 2001/03/22 03:59:44 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/storage/buffer/buf_init.c,v 1.43 2001/07/06 21:04:25 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -63,7 +63,6 @@ long *PrivateRefCount; /* also used in freelist.c */
bits8 *BufferLocks; /* flag bits showing locks I have set */
BufferTag *BufferTagLastDirtied; /* tag buffer had when last
* dirtied by me */
BufferBlindId *BufferBlindLastDirtied;
bool *BufferDirtiedByMe; /* T if buf has been dirtied in cur xact */
......@@ -237,7 +236,6 @@ InitBufferPoolAccess(void)
PrivateRefCount = (long *) calloc(NBuffers, sizeof(long));
BufferLocks = (bits8 *) calloc(NBuffers, sizeof(bits8));
BufferTagLastDirtied = (BufferTag *) calloc(NBuffers, sizeof(BufferTag));
BufferBlindLastDirtied = (BufferBlindId *) calloc(NBuffers, sizeof(BufferBlindId));
BufferDirtiedByMe = (bool *) calloc(NBuffers, sizeof(bool));
/*
......
This diff is collapsed.
......@@ -9,7 +9,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/buffer/freelist.c,v 1.23 2001/01/24 19:43:06 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/storage/buffer/freelist.c,v 1.24 2001/07/06 21:04:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -29,14 +29,14 @@
#include "storage/buf_internals.h"
#include "storage/bufmgr.h"
#include "storage/proc.h"
static BufferDesc *SharedFreeList;
/* only actually used in debugging. The lock
* should be acquired before calling the freelist manager.
/*
* State-checking macros
*/
extern SPINLOCK BufMgrLock;
#define IsInQueue(bf) \
( \
......@@ -45,7 +45,7 @@ extern SPINLOCK BufMgrLock;
AssertMacro((bf->flags & BM_FREE)) \
)
#define NotInQueue(bf) \
#define IsNotInQueue(bf) \
( \
AssertMacro((bf->freeNext == INVALID_DESCRIPTOR)), \
AssertMacro((bf->freePrev == INVALID_DESCRIPTOR)), \
......@@ -61,14 +61,14 @@ extern SPINLOCK BufMgrLock;
* the manner in which buffers are added to the freelist queue.
* Currently, they are added on an LRU basis.
*/
void
static void
AddBufferToFreelist(BufferDesc *bf)
{
#ifdef BMTRACE
_bm_trace(bf->tag.relId.dbId, bf->tag.relId.relId, bf->tag.blockNum,
BufferDescriptorGetBuffer(bf), BMT_DEALLOC);
#endif /* BMTRACE */
NotInQueue(bf);
IsNotInQueue(bf);
/* change bf so it points to inFrontOfNew and its successor */
bf->freePrev = SharedFreeList->freePrev;
......@@ -83,13 +83,14 @@ AddBufferToFreelist(BufferDesc *bf)
/*
* PinBuffer -- make buffer unavailable for replacement.
*
* This should be applied only to shared buffers, never local ones.
* Bufmgr lock must be held by caller.
*/
void
PinBuffer(BufferDesc *buf)
{
long b;
/* Assert (buf->refcount < 25); */
int b = BufferDescriptorGetBuffer(buf) - 1;
if (buf->refcount == 0)
{
......@@ -104,13 +105,12 @@ PinBuffer(BufferDesc *buf)
buf->flags &= ~BM_FREE;
}
else
NotInQueue(buf);
IsNotInQueue(buf);
b = BufferDescriptorGetBuffer(buf) - 1;
Assert(PrivateRefCount[b] >= 0);
if (PrivateRefCount[b] == 0)
buf->refcount++;
PrivateRefCount[b]++;
Assert(PrivateRefCount[b] > 0);
}
#ifdef NOT_USED
......@@ -135,24 +135,35 @@ refcount = %ld, file: %s, line: %d\n",
/*
* UnpinBuffer -- make buffer available for replacement.
*
* This should be applied only to shared buffers, never local ones.
* Bufmgr lock must be held by caller.
*/
void
UnpinBuffer(BufferDesc *buf)
{
long b = BufferDescriptorGetBuffer(buf) - 1;
int b = BufferDescriptorGetBuffer(buf) - 1;
IsNotInQueue(buf);
Assert(buf->refcount > 0);
Assert(PrivateRefCount[b] > 0);
PrivateRefCount[b]--;
if (PrivateRefCount[b] == 0)
buf->refcount--;
NotInQueue(buf);
if (buf->refcount == 0)
{
/* buffer is now unpinned */
AddBufferToFreelist(buf);
buf->flags |= BM_FREE;
}
else if ((buf->flags & BM_PIN_COUNT_WAITER) != 0 &&
buf->refcount == 1)
{
/* we just released the last pin other than the waiter's */
buf->flags &= ~BM_PIN_COUNT_WAITER;
ProcSendSignal(buf->wait_backend_id);
}
else
{
/* do nothing */
......@@ -179,18 +190,16 @@ refcount = %ld, file: %s, line: %d\n",
/*
* GetFreeBuffer() -- get the 'next' buffer from the freelist.
*
*/
BufferDesc *
GetFreeBuffer()
GetFreeBuffer(void)
{
BufferDesc *buf;
if (Free_List_Descriptor == SharedFreeList->freeNext)
{
/* queue is empty. All buffers in the buffer pool are pinned. */
elog(ERROR, "out of free buffers: time to abort !\n");
elog(ERROR, "out of free buffers: time to abort!");
return NULL;
}
buf = &(BufferDescriptors[SharedFreeList->freeNext]);
......@@ -220,7 +229,7 @@ InitFreeList(bool init)
if (init)
{
/* we only do this once, normally the postmaster */
/* we only do this once, normally in the postmaster */
SharedFreeList->data = INVALID_OFFSET;
SharedFreeList->flags = 0;
SharedFreeList->flags &= ~(BM_VALID | BM_DELETED | BM_FREE);
......@@ -249,37 +258,23 @@ DBG_FreeListCheck(int nfree)
buf = &(BufferDescriptors[SharedFreeList->freeNext]);
for (i = 0; i < nfree; i++, buf = &(BufferDescriptors[buf->freeNext]))
{
if (!(buf->flags & (BM_FREE)))
{
if (buf != SharedFreeList)
{
printf("\tfree list corrupted: %d flags %x\n",
buf->buf_id, buf->flags);
}
else
{
printf("\tfree list corrupted: too short -- %d not %d\n",
i, nfree);
}
}
if ((BufferDescriptors[buf->freeNext].freePrev != buf->buf_id) ||
(BufferDescriptors[buf->freePrev].freeNext != buf->buf_id))
{
printf("\tfree list links corrupted: %d %ld %ld\n",
buf->buf_id, buf->freePrev, buf->freeNext);
}
}
if (buf != SharedFreeList)
{
printf("\tfree list corrupted: %d-th buffer is %d\n",
nfree, buf->buf_id);
}
}
#endif
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinval.c,v 1.34 2001/06/19 19:42:15 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinval.c,v 1.35 2001/07/06 21:04:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -16,7 +16,6 @@
#include <sys/types.h>
#include "storage/backendid.h"
#include "storage/proc.h"
#include "storage/sinval.h"
#include "storage/sinvaladt.h"
......@@ -411,3 +410,31 @@ GetUndoRecPtr(void)
return (urec);
}
/*
* BackendIdGetProc - given a BackendId, find its PROC structure
*
* This is a trivial lookup in the ProcState array. We assume that the caller
* knows that the backend isn't going to go away, so we do not bother with
* locking.
*/
struct proc *
BackendIdGetProc(BackendId procId)
{
SISeg *segP = shmInvalBuffer;
if (procId > 0 && procId <= segP->lastBackend)
{
ProcState *stateP = &segP->procState[procId - 1];
SHMEM_OFFSET pOffset = stateP->procStruct;
if (pOffset != INVALID_OFFSET)
{
PROC *proc = (PROC *) MAKE_PTR(pOffset);
return proc;
}
}
return NULL;
}
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.103 2001/06/16 22:58:16 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.104 2001/07/06 21:04:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -74,6 +74,7 @@
#include "access/xact.h"
#include "storage/proc.h"
#include "storage/sinval.h"
int DeadlockTimeout = 1000;
......@@ -92,6 +93,7 @@ static PROC_HDR *ProcGlobal = NULL;
PROC *MyProc = NULL;
static bool waitingForLock = false;
static bool waitingForSignal = false;
static void ProcKill(void);
static void ProcGetNewSemIdAndNum(IpcSemaphoreId *semId, int *semNum);
......@@ -894,6 +896,49 @@ ProcReleaseSpins(PROC *proc)
AbortBufferIO();
}
/*
* ProcWaitForSignal - wait for a signal from another backend.
*
* This can share the semaphore normally used for waiting for locks,
* since a backend could never be waiting for a lock and a signal at
* the same time. As with locks, it's OK if the signal arrives just
* before we actually reach the waiting state.
*/
void
ProcWaitForSignal(void)
{
waitingForSignal = true;
IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum, true);
waitingForSignal = false;
}
/*
* ProcCancelWaitForSignal - clean up an aborted wait for signal
*
* We need this in case the signal arrived after we aborted waiting,
* or if it arrived but we never reached ProcWaitForSignal() at all.
* Caller should call this after resetting the signal request status.
*/
void
ProcCancelWaitForSignal(void)
{
ZeroProcSemaphore(MyProc);
waitingForSignal = false;
}
/*
* ProcSendSignal - send a signal to a backend identified by BackendId
*/
void
ProcSendSignal(BackendId procId)
{
PROC *proc = BackendIdGetProc(procId);
if (proc != NULL)
IpcSemaphoreUnlock(proc->sem.semId, proc->sem.semNum);
}
/*****************************************************************************
*
*****************************************************************************/
......
......@@ -7,17 +7,19 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: buf_internals.h,v 1.48 2001/03/22 04:01:05 momjian Exp $
* $Id: buf_internals.h,v 1.49 2001/07/06 21:04:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#ifndef BUFMGR_INTERNALS_H
#define BUFMGR_INTERNALS_H
#include "storage/backendid.h"
#include "storage/buf.h"
#include "storage/lmgr.h"
#include "storage/s_lock.h"
/* Buf Mgr constants */
/* in bufmgr.c */
extern int Data_Descriptors;
......@@ -38,9 +40,19 @@ extern int ShowPinTrace;
#define BM_IO_IN_PROGRESS (1 << 5)
#define BM_IO_ERROR (1 << 6)
#define BM_JUST_DIRTIED (1 << 7)
#define BM_PIN_COUNT_WAITER (1 << 8)
typedef bits16 BufFlags;
/*
* Buffer tag identifies which disk block the buffer contains.
*
* Note: the BufferTag data must be sufficient to determine where to write the
* block, even during a "blind write" with no relcache entry. It's possible
* that the backend flushing the buffer doesn't even believe the relation is
* visible yet (its xact may have started before the xact that created the
* rel). The storage manager must be able to cope anyway.
*/
typedef struct buftag
{
RelFileNode rnode;
......@@ -60,28 +72,9 @@ typedef struct buftag
(a)->rnode = (xx_reln)->rd_node \
)
/*
* We don't need this data any more but it allows more user
* friendly error messages. Feel free to get rid of it
* (and change a lot of places -:))
*/
typedef struct bufblindid
{
char dbname[NAMEDATALEN]; /* name of db in which buf belongs */
char relname[NAMEDATALEN]; /* name of reln */
} BufferBlindId;
/*
* BufferDesc -- shared buffer cache metadata for a single
* shared buffer descriptor.
*
* We keep the name of the database and relation in which this
* buffer appears in order to avoid a catalog lookup on cache
* flush if we don't have the reldesc in the cache. It is also
* possible that the relation to which this buffer belongs is
* not visible to all backends at the time that it gets flushed.
* Dbname, relname, dbid, and relid are enough to determine where
* to put the buffer, for all storage managers.
*/
typedef struct sbufdesc
{
......@@ -89,14 +82,14 @@ typedef struct sbufdesc
Buffer freePrev;
SHMEM_OFFSET data; /* pointer to data in buf pool */
/* tag and id must be together for table lookup to work */
/* tag and id must be together for table lookup (still true?) */
BufferTag tag; /* file/block identifier */
int buf_id; /* maps global desc to local desc */
int buf_id; /* buffer's index number (from 0) */
BufFlags flags; /* see bit definitions above */
unsigned refcount; /* # of times buffer is pinned */
unsigned refcount; /* # of backends holding pins on buffer */
slock_t io_in_progress_lock; /* to block for I/O to complete */
slock_t io_in_progress_lock; /* to wait for I/O to complete */
slock_t cntx_lock; /* to lock access to page context */
unsigned r_locks; /* # of shared locks */
......@@ -105,15 +98,14 @@ typedef struct sbufdesc
bool cntxDirty; /* new way to mark block as dirty */
BufferBlindId blind; /* was used to support blind write */
/*
* When we can't delete item from page (someone else has buffer
* pinned) we mark buffer for cleanup by specifying appropriate for
* buffer content cleanup function. Buffer will be cleaned up from
* release buffer functions.
* We can't physically remove items from a disk page if another backend
* has the buffer pinned. Hence, a backend may need to wait for all
* other pins to go away. This is signaled by setting its own backend ID
* into wait_backend_id and setting flag bit BM_PIN_COUNT_WAITER.
* At present, there can be only one such waiter per buffer.
*/
void (*CleanupFunc) (Buffer);
BackendId wait_backend_id; /* backend ID of pin-count waiter */
} BufferDesc;
#define BufferDescriptorGetBuffer(bdesc) ((bdesc)->buf_id + 1)
......@@ -128,21 +120,23 @@ typedef struct sbufdesc
#define BL_R_LOCK (1 << 1)
#define BL_RI_LOCK (1 << 2)
#define BL_W_LOCK (1 << 3)
#define BL_PIN_COUNT_LOCK (1 << 4)
/*
* mao tracing buffer allocation
*/
/*#define BMTRACE*/
#ifdef BMTRACE
typedef struct _bmtrace
{
int bmt_pid;
long bmt_buf;
long bmt_dbid;
long bmt_relid;
int bmt_blkno;
int bmt_buf;
Oid bmt_dbid;
Oid bmt_relid;
BlockNumber bmt_blkno;
int bmt_op;
#define BMT_NOTUSED 0
......@@ -162,9 +156,7 @@ typedef struct _bmtrace
/* Internal routines: only called by buf.c */
/*freelist.c*/
extern void AddBufferToFreelist(BufferDesc *bf);
extern void PinBuffer(BufferDesc *buf);
extern void PinBuffer_Debug(char *file, int line, BufferDesc *buf);
extern void UnpinBuffer(BufferDesc *buf);
extern BufferDesc *GetFreeBuffer(void);
extern void InitFreeList(bool init);
......@@ -179,7 +171,6 @@ extern bool BufTableInsert(BufferDesc *buf);
extern BufferDesc *BufferDescriptors;
extern bits8 *BufferLocks;
extern BufferTag *BufferTagLastDirtied;
extern BufferBlindId *BufferBlindLastDirtied;
extern LockRelId *BufferRelidLastDirtied;
extern bool *BufferDirtiedByMe;
extern SPINLOCK BufMgrLock;
......
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: bufmgr.h,v 1.53 2001/06/29 21:08:25 tgl Exp $
* $Id: bufmgr.h,v 1.54 2001/07/06 21:04:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -167,7 +167,7 @@ extern void InitBufferPoolAccess(void);
extern void PrintBufferUsage(FILE *statfp);
extern void ResetBufferUsage(void);
extern void ResetBufferPool(bool isCommit);
extern int BufferPoolCheckLeak(void);
extern bool BufferPoolCheckLeak(void);
extern void FlushBufferPool(void);
extern BlockNumber BufferGetBlockNumber(Buffer buffer);
extern BlockNumber RelationGetNumberOfBlocks(Relation relation);
......@@ -183,10 +183,9 @@ extern void SetBufferCommitInfoNeedsSave(Buffer buffer);
extern void UnlockBuffers(void);
extern void LockBuffer(Buffer buffer, int mode);
extern void AbortBufferIO(void);
extern void LockBufferForCleanup(Buffer buffer);
extern bool BufferIsUpdatable(Buffer buffer);
extern void MarkBufferForCleanup(Buffer buffer, void (*CleanupFunc) (Buffer));
extern void AbortBufferIO(void);
extern void BufmgrCommit(void);
extern void BufferSync(void);
......
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: proc.h,v 1.44 2001/06/16 22:58:17 tgl Exp $
* $Id: proc.h,v 1.45 2001/07/06 21:04:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -15,6 +15,7 @@
#define _PROC_H_
#include "access/xlog.h"
#include "storage/backendid.h"
#include "storage/lock.h"
/* configurable option */
......@@ -139,4 +140,8 @@ extern void ProcReleaseSpins(PROC *proc);
extern bool LockWaitCancel(void);
extern void HandleDeadLock(SIGNAL_ARGS);
extern void ProcWaitForSignal(void);
extern void ProcCancelWaitForSignal(void);
extern void ProcSendSignal(BackendId procId);
#endif /* PROC_H */
......@@ -7,13 +7,14 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: sinval.h,v 1.19 2001/06/19 19:42:16 tgl Exp $
* $Id: sinval.h,v 1.20 2001/07/06 21:04:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#ifndef SINVAL_H
#define SINVAL_H
#include "storage/backendid.h"
#include "storage/itemptr.h"
#include "storage/spin.h"
......@@ -77,5 +78,7 @@ extern bool DatabaseHasActiveBackends(Oid databaseId, bool ignoreMyself);
extern bool TransactionIdIsInProgress(TransactionId xid);
extern void GetXmaxRecent(TransactionId *XmaxRecent);
extern int CountActiveBackends(void);
/* Use "struct proc", not PROC, to avoid including proc.h here */
extern struct proc *BackendIdGetProc(BackendId procId);
#endif /* SINVAL_H */
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment