summaryrefslogtreecommitdiff
path: root/src/include
diff options
context:
space:
mode:
authorTom Lane2004-07-17 03:32:14 +0000
committerTom Lane2004-07-17 03:32:14 +0000
commitfe548629c50b753e96515ba2cfd8a85e8fba10de (patch)
treee80b54f71cb7868db3f9f9c97bccf4c48859951a /src/include
parentf4c069ca8fc80640bd1bff510697371ffaf45267 (diff)
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.
Diffstat (limited to 'src/include')
-rw-r--r--src/include/access/gistscan.h5
-rw-r--r--src/include/access/hash.h5
-rw-r--r--src/include/access/nbtree.h4
-rw-r--r--src/include/access/rtree.h5
-rw-r--r--src/include/access/xact.h59
-rw-r--r--src/include/commands/portalcmds.h4
-rw-r--r--src/include/storage/bufmgr.h30
-rw-r--r--src/include/storage/lock.h12
-rw-r--r--src/include/storage/proc.h9
-rw-r--r--src/include/storage/smgr.h3
-rw-r--r--src/include/utils/catcache.h10
-rw-r--r--src/include/utils/portal.h32
-rw-r--r--src/include/utils/rel.h31
-rw-r--r--src/include/utils/relcache.h4
-rw-r--r--src/include/utils/resowner.h121
15 files changed, 173 insertions, 161 deletions
diff --git a/src/include/access/gistscan.h b/src/include/access/gistscan.h
index 4022f542752..d0afed75b71 100644
--- a/src/include/access/gistscan.h
+++ b/src/include/access/gistscan.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/access/gistscan.h,v 1.23 2004/07/01 00:51:38 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/access/gistscan.h,v 1.24 2004/07/17 03:30:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -22,7 +22,6 @@ extern Datum gistmarkpos(PG_FUNCTION_ARGS);
extern Datum gistrestrpos(PG_FUNCTION_ARGS);
extern Datum gistendscan(PG_FUNCTION_ARGS);
extern void gistadjscans(Relation r, int op, BlockNumber blkno, OffsetNumber offnum);
-extern void AtEOXact_gist(void);
-extern void AtEOSubXact_gist(TransactionId childXid);
+extern void ReleaseResources_gist(void);
#endif /* GISTSCAN_H */
diff --git a/src/include/access/hash.h b/src/include/access/hash.h
index 2088cc2f5a6..41afe630296 100644
--- a/src/include/access/hash.h
+++ b/src/include/access/hash.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/access/hash.h,v 1.55 2004/07/01 00:51:38 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/access/hash.h,v 1.56 2004/07/17 03:30:38 tgl Exp $
*
* NOTES
* modeled after Margo Seltzer's hash implementation for unix.
@@ -292,8 +292,7 @@ extern void _hash_expandtable(Relation rel, Buffer metabuf);
extern void _hash_regscan(IndexScanDesc scan);
extern void _hash_dropscan(IndexScanDesc scan);
extern bool _hash_has_active_scan(Relation rel, Bucket bucket);
-extern void AtEOXact_hash(void);
-extern void AtEOSubXact_hash(TransactionId childXid);
+extern void ReleaseResources_hash(void);
/* hashsearch.c */
extern bool _hash_next(IndexScanDesc scan, ScanDirection dir);
diff --git a/src/include/access/nbtree.h b/src/include/access/nbtree.h
index 6e2491532ad..bf95d8ba23d 100644
--- a/src/include/access/nbtree.h
+++ b/src/include/access/nbtree.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/access/nbtree.h,v 1.79 2004/07/11 18:01:45 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/access/nbtree.h,v 1.80 2004/07/17 03:30:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -402,8 +402,6 @@ typedef BTScanOpaqueData *BTScanOpaque;
/*
* prototypes for functions in nbtree.c (external entry points for btree)
*/
-extern void AtEOXact_nbtree(void);
-
extern Datum btbuild(PG_FUNCTION_ARGS);
extern Datum btinsert(PG_FUNCTION_ARGS);
extern Datum btgettuple(PG_FUNCTION_ARGS);
diff --git a/src/include/access/rtree.h b/src/include/access/rtree.h
index 5b5347e9a08..a23b00283fb 100644
--- a/src/include/access/rtree.h
+++ b/src/include/access/rtree.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/access/rtree.h,v 1.33 2004/07/01 00:51:38 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/access/rtree.h,v 1.34 2004/07/17 03:30:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -129,8 +129,7 @@ extern void rtree_desc(char *buf, uint8 xl_info, char *rec);
/* rtscan.c */
extern void rtadjscans(Relation r, int op, BlockNumber blkno,
OffsetNumber offnum);
-extern void AtEOXact_rtree(void);
-extern void AtEOSubXact_rtree(TransactionId childXid);
+extern void ReleaseResources_rtree(void);
/* rtstrat.c */
extern StrategyNumber RTMapToInternalOperator(StrategyNumber strat);
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index c5b66afd0df..714518d308c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -7,17 +7,16 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/access/xact.h,v 1.64 2004/07/01 00:51:38 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/access/xact.h,v 1.65 2004/07/17 03:30:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#ifndef XACT_H
#define XACT_H
-#include "access/transam.h"
#include "access/xlog.h"
#include "utils/nabstime.h"
-#include "utils/timestamp.h"
+
/*
* Xact isolation levels
@@ -41,62 +40,10 @@ extern bool DefaultXactReadOnly;
extern bool XactReadOnly;
/*
- * transaction states - transaction state from server perspective
- */
-typedef enum TransState
-{
- TRANS_DEFAULT,
- TRANS_START,
- TRANS_INPROGRESS,
- TRANS_COMMIT,
- TRANS_ABORT
-} TransState;
-
-/*
- * transaction block states - transaction state of client queries
- */
-typedef enum TBlockState
-{
- TBLOCK_DEFAULT,
- TBLOCK_STARTED,
- TBLOCK_BEGIN,
- TBLOCK_INPROGRESS,
- TBLOCK_END,
- TBLOCK_ABORT,
- TBLOCK_ENDABORT,
-
- TBLOCK_SUBBEGIN,
- TBLOCK_SUBBEGINABORT,
- TBLOCK_SUBINPROGRESS,
- TBLOCK_SUBEND,
- TBLOCK_SUBABORT,
- TBLOCK_SUBENDABORT_OK,
- TBLOCK_SUBENDABORT_ERROR
-} TBlockState;
-
-/*
* end-of-transaction cleanup callbacks for dynamically loaded modules
*/
typedef void (*EOXactCallback) (bool isCommit, void *arg);
-/*
- * transaction state structure
- */
-typedef struct TransactionStateData
-{
- TransactionId transactionIdData; /* my XID */
- CommandId commandId; /* current CID */
- TransState state; /* low-level state */
- TBlockState blockState; /* high-level state */
- int nestingLevel; /* nest depth */
- MemoryContext curTransactionContext; /* my xact-lifetime context */
- List *childXids; /* subcommitted child XIDs */
- AclId currentUser; /* subxact start current_user */
- struct TransactionStateData *parent; /* back link to parent */
-} TransactionStateData;
-
-typedef TransactionStateData *TransactionState;
-
/* ----------------
* transaction-related XLOG entries
@@ -168,7 +115,7 @@ extern void UnregisterEOXactCallback(EOXactCallback callback, void *arg);
extern void RecordTransactionCommit(void);
-extern int xactGetCommittedChildren(TransactionId **ptr, bool metoo);
+extern int xactGetCommittedChildren(TransactionId **ptr);
extern void XactPushRollback(void (*func) (void *), void *data);
extern void XactPopRollback(void);
diff --git a/src/include/commands/portalcmds.h b/src/include/commands/portalcmds.h
index 0a2bc9ad19d..60e7d524d29 100644
--- a/src/include/commands/portalcmds.h
+++ b/src/include/commands/portalcmds.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/commands/portalcmds.h,v 1.14 2003/11/29 22:40:59 pgsql Exp $
+ * $PostgreSQL: pgsql/src/include/commands/portalcmds.h,v 1.15 2004/07/17 03:30:56 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -24,7 +24,7 @@ extern void PerformPortalFetch(FetchStmt *stmt, DestReceiver *dest,
extern void PerformPortalClose(const char *name);
-extern void PortalCleanup(Portal portal, bool isError);
+extern void PortalCleanup(Portal portal);
extern void PersistHoldablePortal(Portal portal);
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index e992751f856..273e63a5215 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/bufmgr.h,v 1.83 2004/07/01 00:51:43 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/storage/bufmgr.h,v 1.84 2004/07/17 03:31:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -93,31 +93,6 @@ extern int32 *LocalRefCount;
)
/*
- * IncrBufferRefCount
- * Increment the pin count on a buffer that we have *already* pinned
- * at least once.
- *
- * This macro 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...
- */
-#define IncrBufferRefCount(buffer) \
-( \
- BufferIsLocal(buffer) ? \
- ( \
- (void) AssertMacro((buffer) >= -NLocBuffer), \
- (void) AssertMacro(LocalRefCount[-(buffer) - 1] > 0), \
- (void) LocalRefCount[-(buffer) - 1]++ \
- ) \
- : \
- ( \
- (void) AssertMacro(!BAD_BUFFER_ID(buffer)), \
- (void) AssertMacro(PrivateRefCount[(buffer) - 1] > 0), \
- (void) PrivateRefCount[(buffer) - 1]++ \
- ) \
-)
-
-/*
* BufferGetBlock
* Returns a reference to a disk page image associated with a buffer.
*
@@ -138,6 +113,7 @@ extern int32 *LocalRefCount;
*/
extern Buffer ReadBuffer(Relation reln, BlockNumber blockNum);
extern void ReleaseBuffer(Buffer buffer);
+extern void IncrBufferRefCount(Buffer buffer);
extern void WriteBuffer(Buffer buffer);
extern void WriteNoReleaseBuffer(Buffer buffer);
extern Buffer ReleaseAndReadBuffer(Buffer buffer, Relation relation,
@@ -148,8 +124,6 @@ extern void InitBufferPoolAccess(void);
extern char *ShowBufferUsage(void);
extern void ResetBufferUsage(void);
extern void AtEOXact_Buffers(bool isCommit);
-extern void AtSubStart_Buffers(void);
-extern void AtEOSubXact_Buffers(bool isCommit);
extern void FlushBufferPool(void);
extern BlockNumber BufferGetBlockNumber(Buffer buffer);
extern BlockNumber RelationGetNumberOfBlocks(Relation relation);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 650b3269497..1289fee7eca 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.78 2004/07/01 00:51:43 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.79 2004/07/17 03:31:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -26,14 +26,6 @@ typedef struct PROC_QUEUE
int size; /* number of entries in list */
} PROC_QUEUE;
-/* Release options for LockReleaseAll */
-typedef enum
-{
- ReleaseAll, /* All my locks */
- ReleaseAllExceptSession, /* All except session locks (Xid = 0) */
- ReleaseGivenXids /* Only locks with Xids in given array */
-} LockReleaseWhich;
-
/* struct PGPROC is declared in storage/proc.h, but must forward-reference it */
typedef struct PGPROC PGPROC;
@@ -248,7 +240,7 @@ extern bool LockAcquire(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
extern bool LockRelease(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
TransactionId xid, LOCKMODE lockmode);
extern bool LockReleaseAll(LOCKMETHODID lockmethodid, PGPROC *proc,
- LockReleaseWhich which, int nxids, TransactionId *xids);
+ bool allxids);
extern int LockCheckConflicts(LockMethod lockMethodTable,
LOCKMODE lockmode,
LOCK *lock, PROCLOCK *proclock, PGPROC *proc,
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 1551d7568c5..8645fb1fdb5 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.68 2004/07/01 00:51:43 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.69 2004/07/17 03:31:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -34,7 +34,7 @@ struct PGPROC
SHM_QUEUE links; /* list link if process is in a list */
PGSemaphoreData sem; /* ONE semaphore to sleep on */
- int errType; /* STATUS_OK or STATUS_ERROR after wakeup */
+ int waitStatus; /* STATUS_OK or STATUS_ERROR after wakeup */
TransactionId xid; /* transaction currently being executed by
* this proc */
@@ -103,13 +103,12 @@ extern int ProcGlobalSemas(int maxBackends);
extern void InitProcGlobal(int maxBackends);
extern void InitProcess(void);
extern void InitDummyProcess(int proctype);
-extern void ProcReleaseLocks(LockReleaseWhich which,
- int nxids, TransactionId *xids);
+extern void ProcReleaseLocks(bool isCommit);
extern void ProcQueueInit(PROC_QUEUE *queue);
extern int ProcSleep(LockMethod lockMethodTable, LOCKMODE lockmode,
LOCK *lock, PROCLOCK *proclock);
-extern PGPROC *ProcWakeup(PGPROC *proc, int errType);
+extern PGPROC *ProcWakeup(PGPROC *proc, int waitStatus);
extern void ProcLockWakeup(LockMethod lockMethodTable, LOCK *lock);
extern bool LockWaitCancel(void);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index e4f0930ef7a..f808120682d 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/smgr.h,v 1.45 2004/07/01 00:51:43 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/storage/smgr.h,v 1.46 2004/07/17 03:31:26 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -66,7 +66,6 @@ extern BlockNumber smgrtruncate(SMgrRelation reln, BlockNumber nblocks,
extern void smgrimmedsync(SMgrRelation reln);
extern void smgrDoPendingDeletes(bool isCommit);
extern int smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr);
-extern void AtSubStart_smgr(void);
extern void AtSubCommit_smgr(void);
extern void AtSubAbort_smgr(void);
extern void smgrcommit(void);
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3ce54b99a25..627c172793b 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -13,7 +13,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/catcache.h,v 1.49 2004/07/01 00:51:44 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/utils/catcache.h,v 1.50 2004/07/17 03:31:47 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -101,9 +101,6 @@ typedef struct catctup
* and negative entries is identical.
*/
int refcount; /* number of active references */
- int *prev_refcount; /* refcounts for upper subtransactions */
- int numpushes; /* number of used refcounts in the array */
- int numalloc; /* allocated size of array */
bool dead; /* dead but not yet removed? */
bool negative; /* negative cache entry? */
uint32 hash_value; /* hash value for this tuple's keys */
@@ -142,9 +139,6 @@ typedef struct catclist
*/
Dlelem cache_elem; /* list member of per-catcache list */
int refcount; /* number of active references */
- int *prev_refcount; /* refcounts for upper subtransactions */
- int numpushes; /* number of used refcounts in the array */
- int numalloc; /* allocated size of array */
bool dead; /* dead but not yet removed? */
bool ordered; /* members listed in index order? */
short nkeys; /* number of lookup keys specified */
@@ -169,8 +163,6 @@ extern DLLIMPORT MemoryContext CacheMemoryContext;
extern void CreateCacheMemoryContext(void);
extern void AtEOXact_CatCache(bool isCommit);
-extern void AtSubStart_CatCache(void);
-extern void AtEOSubXact_CatCache(bool isCommit);
extern CatCache *InitCatCache(int id, const char *relname, const char *indname,
int reloidattr,
diff --git a/src/include/utils/portal.h b/src/include/utils/portal.h
index 3437dc448a3..d5a12520d0e 100644
--- a/src/include/utils/portal.h
+++ b/src/include/utils/portal.h
@@ -39,7 +39,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/portal.h,v 1.49 2004/07/01 00:51:44 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/utils/portal.h,v 1.50 2004/07/17 03:31:47 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -48,6 +48,7 @@
#include "executor/execdesc.h"
#include "nodes/memnodes.h"
+#include "utils/resowner.h"
#include "utils/tuplestore.h"
@@ -80,6 +81,20 @@ typedef enum PortalStrategy
} PortalStrategy;
/*
+ * A portal is always in one of these states. It is possible to transit
+ * from ACTIVE back to READY if the query is not run to completion;
+ * otherwise we never back up in status.
+ */
+typedef enum PortalStatus
+{
+ PORTAL_NEW, /* in process of creation */
+ PORTAL_READY, /* PortalStart complete, can run it */
+ PORTAL_ACTIVE, /* portal is running (can't delete it) */
+ PORTAL_DONE, /* portal is finished (don't re-run it) */
+ PORTAL_FAILED /* portal got error (can't re-run it) */
+} PortalStatus;
+
+/*
* Note: typedef Portal is declared in tcop/dest.h as
* typedef struct PortalData *Portal;
*/
@@ -89,7 +104,8 @@ typedef struct PortalData
/* Bookkeeping data */
const char *name; /* portal's name */
MemoryContext heap; /* subsidiary memory for portal */
- void (*cleanup) (Portal portal, bool isError); /* cleanup hook */
+ ResourceOwner resowner; /* resources owned by portal */
+ void (*cleanup) (Portal portal); /* cleanup hook */
TransactionId createXact; /* the xid of the creating xact */
/* The query or queries the portal will execute */
@@ -113,10 +129,8 @@ typedef struct PortalData
int cursorOptions; /* DECLARE CURSOR option bits */
/* Status data */
- bool portalReady; /* PortalStart complete? */
+ PortalStatus status; /* see above */
bool portalUtilReady; /* PortalRunUtility complete? */
- bool portalActive; /* portal is running (can't delete it) */
- bool portalDone; /* portal is finished (don't re-run it) */
/* If not NULL, Executor is active; call ExecutorEnd eventually: */
QueryDesc *queryDesc; /* info needed for executor invocation */
@@ -167,12 +181,14 @@ extern void EnablePortalManager(void);
extern void AtCommit_Portals(void);
extern void AtAbort_Portals(void);
extern void AtCleanup_Portals(void);
-extern void AtSubCommit_Portals(TransactionId parentXid);
-extern void AtSubAbort_Portals(void);
+extern void AtSubCommit_Portals(TransactionId parentXid,
+ ResourceOwner parentXactOwner);
+extern void AtSubAbort_Portals(TransactionId parentXid,
+ ResourceOwner parentXactOwner);
extern void AtSubCleanup_Portals(void);
extern Portal CreatePortal(const char *name, bool allowDup, bool dupSilent);
extern Portal CreateNewPortal(void);
-extern void PortalDrop(Portal portal, bool isError);
+extern void PortalDrop(Portal portal, bool isTopCommit);
extern void DropDependentPortals(MemoryContext queryContext);
extern Portal GetPortalByName(const char *name);
extern void PortalDefineQuery(Portal portal,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index b7f85eda68e..481cdb2465f 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/rel.h,v 1.75 2004/07/01 00:51:44 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/utils/rel.h,v 1.76 2004/07/17 03:31:47 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -110,9 +110,6 @@ typedef struct RelationData
BlockNumber rd_targblock; /* current insertion target block, or
* InvalidBlockNumber */
int rd_refcnt; /* reference count */
- int *rd_prevrefcnt; /* reference count stack */
- int rd_numalloc; /* stack allocated size */
- int rd_numpushed; /* stack used size */
bool rd_isnew; /* rel was created in current xact */
/*
@@ -191,28 +188,6 @@ typedef Relation *RelationPtr;
((bool)((relation)->rd_refcnt == 0))
/*
- * RelationSetReferenceCount
- * Sets relation reference count.
- */
-#define RelationSetReferenceCount(relation,count) \
- ((relation)->rd_refcnt = (count))
-
-/*
- * RelationIncrementReferenceCount
- * Increments relation reference count.
- */
-#define RelationIncrementReferenceCount(relation) \
- ((relation)->rd_refcnt += 1)
-
-/*
- * RelationDecrementReferenceCount
- * Decrements relation reference count.
- */
-#define RelationDecrementReferenceCount(relation) \
- (AssertMacro((relation)->rd_refcnt > 0), \
- (relation)->rd_refcnt -= 1)
-
-/*
* RelationGetForm
* Returns pg_class tuple for a relation.
*
@@ -255,4 +230,8 @@ typedef Relation *RelationPtr;
#define RelationGetNamespace(relation) \
((relation)->rd_rel->relnamespace)
+/* routines in utils/cache/relcache.c */
+extern void RelationIncrementReferenceCount(Relation rel);
+extern void RelationDecrementReferenceCount(Relation rel);
+
#endif /* REL_H */
diff --git a/src/include/utils/relcache.h b/src/include/utils/relcache.h
index 47f46190df7..227ef591a2c 100644
--- a/src/include/utils/relcache.h
+++ b/src/include/utils/relcache.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/relcache.h,v 1.41 2004/07/01 00:51:45 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/utils/relcache.h,v 1.42 2004/07/17 03:31:47 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -66,8 +66,6 @@ extern void RelationCacheInvalidateEntry(Oid relationId, RelFileNode *rnode);
extern void RelationCacheInvalidate(void);
extern void AtEOXact_RelationCache(bool isCommit);
-extern void AtSubStart_RelationCache(void);
-extern void AtEOSubXact_RelationCache(bool isCommit);
/*
* Routines to help manage rebuilding of relcache init file
diff --git a/src/include/utils/resowner.h b/src/include/utils/resowner.h
new file mode 100644
index 00000000000..6de270d2b0e
--- /dev/null
+++ b/src/include/utils/resowner.h
@@ -0,0 +1,121 @@
+/*-------------------------------------------------------------------------
+ *
+ * resowner.h
+ * POSTGRES resource owner definitions.
+ *
+ * Query-lifespan resources are tracked by associating them with
+ * ResourceOwner objects. This provides a simple mechanism for ensuring
+ * that such resources are freed at the right time.
+ * See utils/resowner/README for more info.
+ *
+ *
+ * Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * $PostgreSQL: pgsql/src/include/utils/resowner.h,v 1.1 2004/07/17 03:31:47 tgl Exp $
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef RESOWNER_H
+#define RESOWNER_H
+
+#include "storage/buf.h"
+#include "storage/lock.h"
+#include "utils/catcache.h"
+#include "utils/rel.h"
+
+
+/*
+ * ResourceOwner objects are an opaque data structure known only within
+ * resowner.c.
+ */
+typedef struct ResourceOwnerData *ResourceOwner;
+
+
+/*
+ * Globally known ResourceOwners
+ */
+extern DLLIMPORT ResourceOwner CurrentResourceOwner;
+extern DLLIMPORT ResourceOwner CurTransactionResourceOwner;
+extern DLLIMPORT ResourceOwner TopTransactionResourceOwner;
+
+/*
+ * Resource releasing is done in three phases: pre-locks, locks, and
+ * post-locks. The pre-lock phase must release any resources that are
+ * visible to other backends (such as pinned buffers); this ensures that
+ * when we release a lock that another backend may be waiting on, it will
+ * see us as being fully out of our transaction. The post-lock phase
+ * should be used for backend-internal cleanup.
+ */
+typedef enum
+{
+ RESOURCE_RELEASE_BEFORE_LOCKS,
+ RESOURCE_RELEASE_LOCKS,
+ RESOURCE_RELEASE_AFTER_LOCKS
+} ResourceReleasePhase;
+
+/*
+ * Dynamically loaded modules can get control during ResourceOwnerRelease
+ * by providing a callback of this form.
+ */
+typedef void (*ResourceReleaseCallback) (ResourceReleasePhase phase,
+ bool isCommit,
+ bool isTopLevel,
+ void *arg);
+
+
+/*
+ * Functions in resowner.c
+ */
+
+/* generic routines */
+extern ResourceOwner ResourceOwnerCreate(ResourceOwner parent,
+ const char *name);
+extern void ResourceOwnerRelease(ResourceOwner owner,
+ ResourceReleasePhase phase,
+ bool isCommit,
+ bool isTopLevel);
+extern void ResourceOwnerDelete(ResourceOwner owner);
+extern void ResourceOwnerNewParent(ResourceOwner owner,
+ ResourceOwner newparent);
+extern void RegisterResourceReleaseCallback(ResourceReleaseCallback callback,
+ void *arg);
+extern void UnregisterResourceReleaseCallback(ResourceReleaseCallback callback,
+ void *arg);
+
+/* support for buffer refcount management */
+extern void ResourceOwnerEnlargeBuffers(ResourceOwner owner);
+extern void ResourceOwnerRememberBuffer(ResourceOwner owner, Buffer buffer);
+extern void ResourceOwnerForgetBuffer(ResourceOwner owner, Buffer buffer);
+
+/* support for lock management */
+extern void ResourceOwnerEnlargeLocks(ResourceOwner owner);
+extern void ResourceOwnerRememberLock(ResourceOwner owner,
+ LOCKTAG *locktag,
+ TransactionId xid,
+ LOCKMODE lockmode);
+extern void ResourceOwnerForgetLock(ResourceOwner owner,
+ LOCKTAG *locktag,
+ TransactionId xid,
+ LOCKMODE lockmode);
+
+/* support for catcache refcount management */
+extern void ResourceOwnerEnlargeCatCacheRefs(ResourceOwner owner);
+extern void ResourceOwnerRememberCatCacheRef(ResourceOwner owner,
+ HeapTuple tuple);
+extern void ResourceOwnerForgetCatCacheRef(ResourceOwner owner,
+ HeapTuple tuple);
+extern void ResourceOwnerEnlargeCatCacheListRefs(ResourceOwner owner);
+extern void ResourceOwnerRememberCatCacheListRef(ResourceOwner owner,
+ CatCList *list);
+extern void ResourceOwnerForgetCatCacheListRef(ResourceOwner owner,
+ CatCList *list);
+
+/* support for relcache refcount management */
+extern void ResourceOwnerEnlargeRelationRefs(ResourceOwner owner);
+extern void ResourceOwnerRememberRelationRef(ResourceOwner owner,
+ Relation rel);
+extern void ResourceOwnerForgetRelationRef(ResourceOwner owner,
+ Relation rel);
+
+#endif /* RESOWNER_H */