Commit bbbc00af authored by Tom Lane's avatar Tom Lane

Clean up some longstanding problems in shared-cache invalidation.

SI messages now include the relevant database OID, so that operations
in one database do not cause useless cache flushes in backends attached
to other databases.  Declare SI messages properly using a union, to
eliminate the former assumption that Oid is the same size as int or Index.
Rewrite the nearly-unreadable code in inval.c, and document it better.
Arrange for catcache flushes at end of command/transaction to happen before
relcache flushes do --- this avoids loading a new tuple into the catcache
while setting up new relcache entry, only to have it be flushed again
immediately.
parent d9a069e2
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/access/transam/xact.c,v 1.102 2001/05/04 18:39:16 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/access/transam/xact.c,v 1.103 2001/06/19 19:42:15 tgl Exp $
*
* NOTES
* Transaction aborts can now occur two ways:
......@@ -544,7 +544,6 @@ CommandCounterIncrement(void)
*/
AtCommit_LocalCache();
AtStart_Cache();
}
void
......@@ -577,7 +576,7 @@ InitializeTransactionSystem(void)
static void
AtStart_Cache(void)
{
DiscardInvalid();
AcceptInvalidationMessages();
}
/* --------------------------------
......@@ -725,11 +724,10 @@ RecordTransactionCommit()
static void
AtCommit_Cache(void)
{
/*
* Make catalog changes visible to all backend.
* Make catalog changes visible to all backends.
*/
RegisterInvalid(true);
AtEOXactInvalidationMessages(true);
}
/* --------------------------------
......@@ -739,11 +737,10 @@ AtCommit_Cache(void)
static void
AtCommit_LocalCache(void)
{
/*
* Make catalog changes visible to me for the next command.
*/
ImmediateLocalInvalidation(true);
CommandEndInvalidationMessages(true);
}
/* --------------------------------
......@@ -753,7 +750,6 @@ AtCommit_LocalCache(void)
static void
AtCommit_Locks(void)
{
/*
* XXX What if ProcReleaseLocks fails? (race condition?)
*
......@@ -769,7 +765,6 @@ AtCommit_Locks(void)
static void
AtCommit_Memory(void)
{
/*
* Now that we're "out" of a transaction, have the system allocate
* things in the top memory context instead of per-transaction
......@@ -844,7 +839,7 @@ static void
AtAbort_Cache(void)
{
RelationCacheAbort();
RegisterInvalid(false);
AtEOXactInvalidationMessages(false);
}
/* --------------------------------
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinval.c,v 1.33 2001/06/16 22:58:13 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinval.c,v 1.34 2001/06/19 19:42:15 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -55,56 +55,31 @@ InitBackendSharedInvalidationState(void)
}
/*
* RegisterSharedInvalid
* SendSharedInvalidMessage
* Add a shared-cache-invalidation message to the global SI message queue.
*
* Note:
* Assumes hash index is valid.
* Assumes item pointer is valid.
*/
void
RegisterSharedInvalid(int cacheId, /* XXX */
Index hashIndex,
ItemPointer pointer)
SendSharedInvalidMessage(SharedInvalidationMessage *msg)
{
SharedInvalidData newInvalid;
bool insertOK;
/*
* This code has been hacked to accept two types of messages. This
* might be treated more generally in the future.
*
* (1) cacheId= system cache id hashIndex= system cache hash index for a
* (possibly) cached tuple pointer= pointer of (possibly) cached tuple
*
* (2) cacheId= special non-syscache id hashIndex= object id contained in
* (possibly) cached relation descriptor pointer= null
*/
newInvalid.cacheId = cacheId;
newInvalid.hashIndex = hashIndex;
if (ItemPointerIsValid(pointer))
ItemPointerCopy(pointer, &newInvalid.pointerData);
else
ItemPointerSetInvalid(&newInvalid.pointerData);
SpinAcquire(SInvalLock);
insertOK = SIInsertDataEntry(shmInvalBuffer, &newInvalid);
insertOK = SIInsertDataEntry(shmInvalBuffer, msg);
SpinRelease(SInvalLock);
if (!insertOK)
elog(DEBUG, "RegisterSharedInvalid: SI buffer overflow");
elog(DEBUG, "SendSharedInvalidMessage: SI buffer overflow");
}
/*
* InvalidateSharedInvalid
* ReceiveSharedInvalidMessages
* Process shared-cache-invalidation messages waiting for this backend
*/
void
InvalidateSharedInvalid(void (*invalFunction) (),
void (*resetFunction) ())
ReceiveSharedInvalidMessages(
void (*invalFunction) (SharedInvalidationMessage *msg),
void (*resetFunction) (void))
{
SharedInvalidData data;
SharedInvalidationMessage data;
int getResult;
bool gotMessage = false;
......@@ -118,15 +93,13 @@ void
if (getResult < 0)
{
/* got a reset message */
elog(DEBUG, "InvalidateSharedInvalid: cache state reset");
elog(DEBUG, "ReceiveSharedInvalidMessages: cache state reset");
resetFunction();
}
else
{
/* got a normal data message */
invalFunction(data.cacheId,
data.hashIndex,
&data.pointerData);
invalFunction(&data);
}
gotMessage = true;
}
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinvaladt.c,v 1.39 2001/06/16 22:58:15 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinvaladt.c,v 1.40 2001/06/19 19:42:15 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -20,7 +20,6 @@
#include "miscadmin.h"
#include "storage/backendid.h"
#include "storage/proc.h"
#include "storage/sinval.h"
#include "storage/sinvaladt.h"
SISeg *shmInvalBuffer;
......@@ -35,7 +34,6 @@ static void SISetProcStateInvalid(SISeg *segP);
int
SInvalShmemSize(int maxBackends)
{
/*
* Figure space needed. Note sizeof(SISeg) includes the first
* ProcState entry.
......@@ -183,14 +181,13 @@ CleanupInvalidationState(int status, Datum arg)
* Returns true for normal successful insertion, false if had to reset.
*/
bool
SIInsertDataEntry(SISeg *segP, SharedInvalidData *data)
SIInsertDataEntry(SISeg *segP, SharedInvalidationMessage *data)
{
int numMsgs = segP->maxMsgNum - segP->minMsgNum;
/* Is the buffer full? */
if (numMsgs >= MAXNUMMESSAGES)
{
/*
* Don't panic just yet: slowest backend might have consumed some
* messages but not yet have done SIDelExpiredDataEntries() to
......@@ -273,13 +270,12 @@ SISetProcStateInvalid(SISeg *segP)
*/
int
SIGetDataEntry(SISeg *segP, int backendId,
SharedInvalidData *data)
SharedInvalidationMessage *data)
{
ProcState *stateP = &segP->procState[backendId - 1];
if (stateP->resetState)
{
/*
* Force reset. We can say we have dealt with any messages added
* since the reset, as well...
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/lmgr/lmgr.c,v 1.46 2001/06/12 05:55:49 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/storage/lmgr/lmgr.c,v 1.47 2001/06/19 19:42:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -144,7 +144,7 @@ LockRelation(Relation relation, LOCKMODE lockmode)
* rebuild it and not just delete it.
*/
RelationIncrementReferenceCount(relation);
DiscardInvalid();
AcceptInvalidationMessages();
RelationDecrementReferenceCount(relation);
}
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/cache/catcache.c,v 1.79 2001/06/18 03:35:07 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/utils/cache/catcache.c,v 1.80 2001/06/19 19:42:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -31,8 +31,18 @@
/* #define CACHEDEBUG */ /* turns DEBUG elogs on */
/* voodoo constants */
#define NCCBUCKETS 257 /* Hash buckets per CatCache (prime!) */
/*
* Constants related to size of the catcache.
*
* NCCBUCKETS should be prime and must be less than 64K (because
* SharedInvalCatcacheMsg crams hash indexes into a uint16 field). In
* practice it should be a lot less, anyway, to avoid chewing up too much
* space on hash bucket headers.
*
* MAXCCTUPLES could be as small as a few hundred, if per-backend memory
* consumption is at a premium.
*/
#define NCCBUCKETS 257 /* Hash buckets per CatCache */
#define MAXCCTUPLES 5000 /* Maximum # of tuples in all caches */
......@@ -218,6 +228,11 @@ CatalogCacheInitializeCache(CatCache *cache)
*/
tupdesc = CreateTupleDescCopyConstr(RelationGetDescr(relation));
/*
* get the relation's relisshared flag, too
*/
cache->cc_relisshared = RelationGetForm(relation)->relisshared;
/*
* return to the caller's memory context and close the rel
*/
......@@ -737,6 +752,7 @@ InitCatCache(int id,
cp->cc_relname = relname;
cp->cc_indname = indname;
cp->cc_reloidattr = reloidattr;
cp->cc_relisshared = false; /* temporary */
cp->cc_tupdesc = (TupleDesc) NULL;
cp->cc_ntup = 0;
cp->cc_size = NCCBUCKETS;
......@@ -1116,7 +1132,8 @@ ReleaseCatCache(HeapTuple tuple)
*
* Note that it is irrelevant whether the given tuple is actually loaded
* into the catcache at the moment. Even if it's not there now, it might
* be by the end of the command, so we have to be prepared to flush it.
* be by the end of the command --- or might be in other backends' caches
* --- so we have to be prepared to flush it.
*
* Also note that it's not an error if there are no catcaches for the
* specified relation. inval.c doesn't know exactly which rels have
......@@ -1126,7 +1143,7 @@ ReleaseCatCache(HeapTuple tuple)
void
PrepareToInvalidateCacheTuple(Relation relation,
HeapTuple tuple,
void (*function) (int, Index, ItemPointer))
void (*function) (int, Index, ItemPointer, Oid))
{
CatCache *ccp;
......@@ -1159,6 +1176,7 @@ PrepareToInvalidateCacheTuple(Relation relation,
(*function) (ccp->id,
CatalogCacheComputeTupleHashIndex(ccp, tuple),
&tuple->t_self);
&tuple->t_self,
ccp->cc_relisshared ? (Oid) 0 : MyDatabaseId);
}
}
......@@ -22,22 +22,41 @@
* second lives till end of transaction. Finally, we need a third list of
* all tuples outdated in the current transaction; if we commit, we send
* those invalidation events to all other backends (via the SI message queue)
* so that they can flush obsolete entries from their caches.
* so that they can flush obsolete entries from their caches. This list
* definitely can't be processed until after we commit, otherwise the other
* backends won't see our updated tuples as good.
*
* We do not need to register EVERY tuple operation in this way, just those
* on tuples in relations that have associated catcaches. Also, whenever
* we see an operation on a pg_class or pg_attribute tuple, we register
* a relcache flush operation for the relation described by that tuple.
* on tuples in relations that have associated catcaches. We do, however,
* have to register every operation on every tuple that *could* be in a
* catcache, whether or not it currently is in our cache. Also, if the
* tuple is in a relation that has multiple catcaches, we need to register
* an invalidation message for each such catcache. catcache.c's
* PrepareToInvalidateCacheTuple() routine provides the knowledge of which
* catcaches may need invalidation for a given tuple.
*
* Also, whenever we see an operation on a pg_class or pg_attribute tuple,
* we register a relcache flush operation for the relation described by that
* tuple.
*
* We keep the relcache flush requests in lists separate from the catcache
* tuple flush requests. This allows us to issue all the pending catcache
* flushes before we issue relcache flushes, which saves us from loading
* a catcache tuple during relcache load only to flush it again right away.
* Also, we avoid queuing multiple relcache flush requests for the same
* relation, since a relcache flush is relatively expensive to do.
* (XXX is it worth testing likewise for duplicate catcache flush entries?
* Probably not.)
*
* All the request lists are kept in TopTransactionContext memory, since
* they need not live beyond the end of the current transaction.
*
*
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/cache/inval.c,v 1.44 2001/06/18 03:35:07 tgl Exp $
*
* Note - this code is real crufty... badly needs a rewrite to improve
* readability and portability. (Shouldn't assume Oid == Index, for example)
* $Header: /cvsroot/pgsql/src/backend/utils/cache/inval.c,v 1.45 2001/06/19 19:42:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -48,65 +67,34 @@
#include "storage/sinval.h"
#include "utils/catcache.h"
#include "utils/inval.h"
#include "utils/memutils.h"
#include "utils/relcache.h"
/*
* private invalidation structures
* To minimize palloc traffic, we keep pending requests in successively-
* larger chunks (a slightly more sophisticated version of an expansible
* array). All request types can be stored as SharedInvalidationMessage
* records.
*/
typedef struct InvalidationUserData
{
struct InvalidationUserData *dataP[1]; /* VARIABLE LENGTH */
} InvalidationUserData; /* VARIABLE LENGTH STRUCTURE */
typedef struct InvalidationEntryData
{
InvalidationUserData *nextP;
InvalidationUserData userData; /* VARIABLE LENGTH ARRAY */
} InvalidationEntryData; /* VARIABLE LENGTH STRUCTURE */
typedef Pointer InvalidationEntry;
typedef InvalidationEntry LocalInvalid;
#define EmptyLocalInvalid NULL
typedef struct CatalogInvalidationData
typedef struct InvalidationChunk
{
Index cacheId;
Index hashIndex;
ItemPointerData pointerData;
} CatalogInvalidationData;
struct InvalidationChunk *next; /* list link */
int nitems; /* # items currently stored in chunk */
int maxitems; /* size of allocated array in this chunk */
SharedInvalidationMessage msgs[1]; /* VARIABLE LENGTH ARRAY */
} InvalidationChunk; /* VARIABLE LENGTH STRUCTURE */
typedef struct RelationInvalidationData
typedef struct InvalidationListHeader
{
Oid relationId;
Oid objectId;
} RelationInvalidationData;
typedef union AnyInvalidation
{
CatalogInvalidationData catalog;
RelationInvalidationData relation;
} AnyInvalidation;
typedef struct InvalidationMessageData
{
char kind;
AnyInvalidation any;
} InvalidationMessageData;
typedef InvalidationMessageData *InvalidationMessage;
/*
* variables and macros
*/
InvalidationChunk *cclist; /* list of chunks holding catcache msgs */
InvalidationChunk *rclist; /* list of chunks holding relcache msgs */
} InvalidationListHeader;
/*
* ----------------
* Invalidation info is divided into three parts.
* 1) shared invalidation to be registered for all backends
* 1) shared invalidation to be sent to all backends at commit
* 2) local invalidation for the transaction itself (actually, just
* for the current command within the transaction)
* 3) rollback information for the transaction itself (in case we abort)
......@@ -114,367 +102,297 @@ typedef InvalidationMessageData *InvalidationMessage;
*/
/*
* head of invalidation linked list for all backends
* head of invalidation message list for all backends
* eaten by AtCommit_Cache() in CommitTransaction()
*/
static LocalInvalid InvalidForall = EmptyLocalInvalid;
static InvalidationListHeader GlobalInvalidMsgs;
/*
* head of invalidation linked list for the backend itself
* head of invalidation message list for the current command
* eaten by AtCommit_LocalCache() in CommandCounterIncrement()
*/
static LocalInvalid InvalidLocal = EmptyLocalInvalid;
static InvalidationListHeader LocalInvalidMsgs;
/*
* head of rollback linked list for the backend itself
* head of rollback message list for abort-time processing
* eaten by AtAbort_Cache() in AbortTransaction()
*/
static LocalInvalid RollbackStack = EmptyLocalInvalid;
static InvalidationEntry InvalidationEntryAllocate(uint16 size);
static void LocalInvalidInvalidate(LocalInvalid invalid,
void (*function) (InvalidationMessage),
bool freemember);
static LocalInvalid LocalInvalidRegister(LocalInvalid invalid,
InvalidationEntry entry);
static void DiscardInvalidStack(LocalInvalid *invalid);
static void InvalidationMessageRegisterSharedInvalid(InvalidationMessage message);
static InvalidationListHeader RollbackMsgs;
/* ----------------------------------------------------------------
* "local" invalidation support functions
* Invalidation list support functions
*
* These three routines encapsulate processing of the "chunked"
* representation of what is logically just a list of messages.
* ----------------------------------------------------------------
*/
/*
* InvalidationEntryAllocate
* Allocates an invalidation entry.
*/
static InvalidationEntry
InvalidationEntryAllocate(uint16 size)
{
InvalidationEntryData *entryDataP;
entryDataP = (InvalidationEntryData *)
malloc(sizeof(char *) + size); /* XXX alignment */
if (entryDataP == NULL)
elog(ERROR, "Memory exhausted in InvalidationEntryAllocate");
entryDataP->nextP = NULL;
return (Pointer) &entryDataP->userData;
}
/*
* LocalInvalidRegister
* Link an invalidation entry into a chain of them. Really ugly
* coding here.
* AddInvalidationMessage
* Add an invalidation message to a list (of chunks).
*
* Note that we do not pay any great attention to maintaining the original
* ordering of the messages.
*/
static LocalInvalid
LocalInvalidRegister(LocalInvalid invalid,
InvalidationEntry entry)
static void
AddInvalidationMessage(InvalidationChunk **listHdr,
SharedInvalidationMessage *msg)
{
Assert(PointerIsValid(entry));
InvalidationChunk *chunk = *listHdr;
((InvalidationUserData *) entry)->dataP[-1] =
(InvalidationUserData *) invalid;
return entry;
if (chunk == NULL)
{
/* First time through; create initial chunk */
#define FIRSTCHUNKSIZE 16
chunk = (InvalidationChunk *)
MemoryContextAlloc(TopTransactionContext,
sizeof(InvalidationChunk) +
(FIRSTCHUNKSIZE-1) * sizeof(SharedInvalidationMessage));
chunk->nitems = 0;
chunk->maxitems = FIRSTCHUNKSIZE;
chunk->next = *listHdr;
*listHdr = chunk;
}
else if (chunk->nitems >= chunk->maxitems)
{
/* Need another chunk; double size of last chunk */
int chunksize = 2 * chunk->maxitems;
chunk = (InvalidationChunk *)
MemoryContextAlloc(TopTransactionContext,
sizeof(InvalidationChunk) +
(chunksize-1) * sizeof(SharedInvalidationMessage));
chunk->nitems = 0;
chunk->maxitems = chunksize;
chunk->next = *listHdr;
*listHdr = chunk;
}
/* Okay, add message to current chunk */
chunk->msgs[chunk->nitems] = *msg;
chunk->nitems++;
}
/*
* LocalInvalidInvalidate
* Processes, then frees all entries in a local cache
* invalidation list unless freemember parameter is false.
* Free a list of inval message chunks.
*
* NOTE: when we are about to commit or abort a transaction, it's
* not really necessary to pfree the lists explicitly, since they will
* go away anyway when TopTransactionContext is destroyed.
*/
static void
LocalInvalidInvalidate(LocalInvalid invalid,
void (*function) (InvalidationMessage),
bool freemember)
FreeInvalidationMessageList(InvalidationChunk **listHdr)
{
InvalidationEntryData *entryDataP;
while (PointerIsValid(invalid))
{
entryDataP = (InvalidationEntryData *)
&((InvalidationUserData *) invalid)->dataP[-1];
InvalidationChunk *chunk = *listHdr;
if (PointerIsValid(function))
(*function) ((InvalidationMessage) &entryDataP->userData);
*listHdr = NULL;
invalid = (Pointer) entryDataP->nextP;
if (!freemember)
continue;
/* help catch errors */
entryDataP->nextP = (InvalidationUserData *) NULL;
while (chunk != NULL)
{
InvalidationChunk *nextchunk = chunk->next;
free((Pointer) entryDataP);
pfree(chunk);
chunk = nextchunk;
}
}
static void
DiscardInvalidStack(LocalInvalid *invalid)
{
LocalInvalid locinv;
locinv = *invalid;
*invalid = EmptyLocalInvalid;
if (locinv)
LocalInvalidInvalidate(locinv,
(void (*) (InvalidationMessage)) NULL,
true);
}
/*
* Process a list of invalidation messages.
*
* This is a macro that executes the given code fragment for each message in
* a message chunk list. The fragment should refer to the message as *msg.
*/
#define ProcessMessageList(listHdr, codeFragment) \
do { \
InvalidationChunk *_chunk; \
for (_chunk = (listHdr); _chunk != NULL; _chunk = _chunk->next) \
{ \
int _cindex; \
for (_cindex = 0; _cindex < _chunk->nitems; _cindex++) \
{ \
SharedInvalidationMessage *msg = &_chunk->msgs[_cindex]; \
codeFragment; \
} \
} \
} while (0)
/* ----------------------------------------------------------------
* private support functions
* Invalidation set support functions
*
* These routines understand about the division of a logical invalidation
* list into separate physical lists for catcache and relcache entries.
* ----------------------------------------------------------------
*/
/* --------------------------------
* CacheIdRegister.......
* RelationIdRegister....
* --------------------------------
*/
#ifdef INVALIDDEBUG
#define CacheIdRegisterSpecifiedLocalInvalid_DEBUG1 \
elog(DEBUG, "CacheIdRegisterSpecifiedLocalInvalid(%d, %d, [%d, %d])", \
cacheId, hashIndex, ItemPointerGetBlockNumber(pointer), \
ItemPointerGetOffsetNumber(pointer))
#define CacheIdRegisterLocalInvalid_DEBUG1 \
elog(DEBUG, "CacheIdRegisterLocalInvalid(%d, %d, [%d, %d])", \
cacheId, hashIndex, ItemPointerGetBlockNumber(pointer), \
ItemPointerGetOffsetNumber(pointer))
#define CacheIdRegisterLocalRollback_DEBUG1 \
elog(DEBUG, "CacheIdRegisterLocalRollback(%d, %d, [%d, %d])", \
cacheId, hashIndex, ItemPointerGetBlockNumber(pointer), \
ItemPointerGetOffsetNumber(pointer))
#else
#define CacheIdRegisterSpecifiedLocalInvalid_DEBUG1
#define CacheIdRegisterLocalInvalid_DEBUG1
#define CacheIdRegisterLocalRollback_DEBUG1
#endif /* INVALIDDEBUG */
/*
* CacheIdRegisterSpecifiedLocalInvalid
* Add a catcache inval entry
*/
static LocalInvalid
CacheIdRegisterSpecifiedLocalInvalid(LocalInvalid invalid,
Index cacheId,
Index hashIndex,
ItemPointer pointer)
static void
AddCatcacheInvalidationMessage(InvalidationListHeader *hdr,
int id, Index hashIndex,
ItemPointer tuplePtr, Oid dbId)
{
InvalidationMessage message;
/*
* debugging stuff
*/
CacheIdRegisterSpecifiedLocalInvalid_DEBUG1;
SharedInvalidationMessage msg;
/*
* create a message describing the system catalog tuple we wish to
* invalidate.
*/
message = (InvalidationMessage)
InvalidationEntryAllocate(sizeof(InvalidationMessageData));
message->kind = 'c';
message->any.catalog.cacheId = cacheId;
message->any.catalog.hashIndex = hashIndex;
ItemPointerCopy(pointer, &message->any.catalog.pointerData);
/*
* Add message to linked list of unprocessed messages.
*/
invalid = LocalInvalidRegister(invalid, (InvalidationEntry) message);
return invalid;
msg.cc.id = (int16) id;
msg.cc.hashIndex = (uint16) hashIndex;
msg.cc.dbId = dbId;
msg.cc.tuplePtr = *tuplePtr;
AddInvalidationMessage(&hdr->cclist, &msg);
}
/*
* CacheIdRegisterLocalInvalid
* Add a relcache inval entry
*/
static void
CacheIdRegisterLocalInvalid(int cacheId,
Index hashIndex,
ItemPointer pointer)
AddRelcacheInvalidationMessage(InvalidationListHeader *hdr,
Oid dbId, Oid relId)
{
/*
* debugging stuff
*/
CacheIdRegisterLocalInvalid_DEBUG1;
/*
* Add message to InvalidForall linked list.
*/
InvalidForall = CacheIdRegisterSpecifiedLocalInvalid(InvalidForall,
cacheId, hashIndex, pointer);
/*
* Add message to InvalidLocal linked list.
*/
InvalidLocal = CacheIdRegisterSpecifiedLocalInvalid(InvalidLocal,
cacheId, hashIndex, pointer);
SharedInvalidationMessage msg;
/* Don't add a duplicate item */
/* We assume comparing relId is sufficient, needn't check dbId */
ProcessMessageList(hdr->rclist,
if (msg->rc.relId == relId) return);
/* OK, add the item */
msg.rc.id = SHAREDINVALRELCACHE_ID;
msg.rc.dbId = dbId;
msg.rc.relId = relId;
AddInvalidationMessage(&hdr->rclist, &msg);
}
/*
* CacheIdRegisterLocalRollback
* Reset an invalidation list to empty
*
* physicalFree may be set false if caller knows transaction is ending
*/
static void
CacheIdRegisterLocalRollback(int cacheId,
Index hashIndex,
ItemPointer pointer)
DiscardInvalidationMessages(InvalidationListHeader *hdr, bool physicalFree)
{
/*
* debugging stuff
*/
CacheIdRegisterLocalRollback_DEBUG1;
/*
* Add message to RollbackStack linked list.
*/
RollbackStack = CacheIdRegisterSpecifiedLocalInvalid(
RollbackStack, cacheId, hashIndex, pointer);
if (physicalFree)
{
/* Physically pfree the list data */
FreeInvalidationMessageList(&hdr->cclist);
FreeInvalidationMessageList(&hdr->rclist);
}
else
{
/* Assume the storage will go away at xact end, just reset pointers */
hdr->cclist = NULL;
hdr->rclist = NULL;
}
}
/*
* RelationIdRegisterSpecifiedLocalInvalid
* Execute the given function for all the messages in an invalidation list.
* The list is not altered.
*
* catcache entries are processed first, for reasons mentioned above.
*/
static LocalInvalid
RelationIdRegisterSpecifiedLocalInvalid(LocalInvalid invalid,
Oid relationId, Oid objectId)
static void
ProcessInvalidationMessages(InvalidationListHeader *hdr,
void (*func) (SharedInvalidationMessage *msg))
{
InvalidationMessage message;
/*
* debugging stuff
*/
#ifdef INVALIDDEBUG
elog(DEBUG, "RelationRegisterSpecifiedLocalInvalid(%u, %u)", relationId,
objectId);
#endif /* defined(INVALIDDEBUG) */
ProcessMessageList(hdr->cclist, func(msg));
ProcessMessageList(hdr->rclist, func(msg));
}
/*
* create a message describing the relation descriptor we wish to
* invalidate.
/* ----------------------------------------------------------------
* private support functions
* ----------------------------------------------------------------
*/
message = (InvalidationMessage)
InvalidationEntryAllocate(sizeof(InvalidationMessageData));
message->kind = 'r';
message->any.relation.relationId = relationId;
message->any.relation.objectId = objectId;
/*
* Add message to linked list of unprocessed messages.
/*
* RegisterCatcacheInvalidation
*
* Register an invalidation event for an updated/deleted catcache entry.
* We insert the event into both GlobalInvalidMsgs (for transmission
* to other backends at transaction commit) and LocalInvalidMsgs (for
* my local invalidation at end of command within xact).
*/
invalid = LocalInvalidRegister(invalid, (InvalidationEntry) message);
return invalid;
static void
RegisterCatcacheInvalidation(int cacheId,
Index hashIndex,
ItemPointer tuplePtr,
Oid dbId)
{
AddCatcacheInvalidationMessage(&GlobalInvalidMsgs,
cacheId, hashIndex, tuplePtr, dbId);
AddCatcacheInvalidationMessage(&LocalInvalidMsgs,
cacheId, hashIndex, tuplePtr, dbId);
}
/*
* RelationIdRegisterLocalInvalid
* RegisterRelcacheInvalidation
*
* As above, but register a relcache invalidation event.
*/
static void
RelationIdRegisterLocalInvalid(Oid relationId, Oid objectId)
RegisterRelcacheInvalidation(Oid dbId, Oid relId)
{
/*
* debugging stuff
*/
#ifdef INVALIDDEBUG
elog(DEBUG, "RelationRegisterLocalInvalid(%u, %u)", relationId,
objectId);
#endif /* defined(INVALIDDEBUG) */
/*
* Add message to InvalidForall linked list.
*/
InvalidForall = RelationIdRegisterSpecifiedLocalInvalid(InvalidForall,
relationId, objectId);
/*
* Add message to InvalidLocal linked list.
*/
InvalidLocal = RelationIdRegisterSpecifiedLocalInvalid(InvalidLocal,
relationId, objectId);
AddRelcacheInvalidationMessage(&GlobalInvalidMsgs,
dbId, relId);
AddRelcacheInvalidationMessage(&LocalInvalidMsgs,
dbId, relId);
}
/*
* RelationIdRegisterLocalRollback
* RegisterCatcacheRollback
*
* Register an invalidation event for an inserted catcache entry.
* This only needs to be flushed out of my local catcache, if I abort.
*/
static void
RelationIdRegisterLocalRollback(Oid relationId, Oid objectId)
RegisterCatcacheRollback(int cacheId,
Index hashIndex,
ItemPointer tuplePtr,
Oid dbId)
{
/*
* debugging stuff
*/
#ifdef INVALIDDEBUG
elog(DEBUG, "RelationRegisterLocalRollback(%u, %u)", relationId,
objectId);
#endif /* defined(INVALIDDEBUG) */
/*
* Add message to RollbackStack linked list.
*/
RollbackStack = RelationIdRegisterSpecifiedLocalInvalid(
RollbackStack, relationId, objectId);
AddCatcacheInvalidationMessage(&RollbackMsgs,
cacheId, hashIndex, tuplePtr, dbId);
}
/*
* CacheIdInvalidate
* RegisterRelcacheRollback
*
* This routine can invalidate a tuple in a system catalog cache
* or a cached relation descriptor. You pay your money and you
* take your chances...
* As above, but register a relcache invalidation event.
*/
#ifdef INVALIDDEBUG
#define CacheIdInvalidate_DEBUG1 \
elog(DEBUG, "CacheIdInvalidate(%d, %d, 0x%x[%d])", cacheId, hashIndex,\
pointer, ItemPointerIsValid(pointer))
#else
#define CacheIdInvalidate_DEBUG1
#endif /* defined(INVALIDDEBUG) */
static void
CacheIdInvalidate(Index cacheId,
Index hashIndex,
ItemPointer pointer)
RegisterRelcacheRollback(Oid dbId, Oid relId)
{
AddRelcacheInvalidationMessage(&RollbackMsgs,
dbId, relId);
}
/*
* assume that if the item pointer is valid, then we are invalidating
* an item in the specified system catalog cache.
/*
* LocalExecuteInvalidationMessage
*
* Process a single invalidation message (which could be either type).
* Only the local caches are flushed; this does not transmit the message
* to other backends.
*/
if (ItemPointerIsValid(pointer))
static void
LocalExecuteInvalidationMessage(SharedInvalidationMessage *msg)
{
if (msg->id >= 0)
{
CatalogCacheIdInvalidate(cacheId, hashIndex, pointer);
return;
if (msg->cc.dbId == MyDatabaseId || msg->cc.dbId == 0)
CatalogCacheIdInvalidate(msg->cc.id,
msg->cc.hashIndex,
&msg->cc.tuplePtr);
}
CacheIdInvalidate_DEBUG1;
/*
* if the cacheId is the oid of any of the following system relations,
* then assume we are invalidating a relation descriptor
*/
if (cacheId == RelOid_pg_class)
else if (msg->id == SHAREDINVALRELCACHE_ID)
{
RelationIdInvalidateRelationCacheByRelationId(hashIndex);
return;
if (msg->rc.dbId == MyDatabaseId || msg->rc.dbId == 0)
RelationIdInvalidateRelationCacheByRelationId(msg->rc.relId);
}
if (cacheId == RelOid_pg_attribute)
else
{
RelationIdInvalidateRelationCacheByRelationId(hashIndex);
return;
elog(FATAL, "ExecuteInvalidationMessage: bogus message id %d",
msg->id);
}
/*
* Yow! the caller asked us to invalidate something else.
*/
elog(FATAL, "CacheIdInvalidate: cacheId=%d relation id?", cacheId);
}
/*
......@@ -496,299 +414,160 @@ InvalidateSystemCaches(void)
}
/*
* InvalidationMessageRegisterSharedInvalid
*/
#ifdef INVALIDDEBUG
#define InvalidationMessageRegisterSharedInvalid_DEBUG1 \
elog(DEBUG,\
"InvalidationMessageRegisterSharedInvalid(c, %d, %d, [%d, %d])",\
message->any.catalog.cacheId,\
message->any.catalog.hashIndex,\
ItemPointerGetBlockNumber(&message->any.catalog.pointerData),\
ItemPointerGetOffsetNumber(&message->any.catalog.pointerData))
#define InvalidationMessageRegisterSharedInvalid_DEBUG2 \
elog(DEBUG, \
"InvalidationMessageRegisterSharedInvalid(r, %u, %u)", \
message->any.relation.relationId, \
message->any.relation.objectId)
#else
#define InvalidationMessageRegisterSharedInvalid_DEBUG1
#define InvalidationMessageRegisterSharedInvalid_DEBUG2
#endif /* INVALIDDEBUG */
static void
InvalidationMessageRegisterSharedInvalid(InvalidationMessage message)
{
Assert(PointerIsValid(message));
switch (message->kind)
{
case 'c': /* cached system catalog tuple */
InvalidationMessageRegisterSharedInvalid_DEBUG1;
RegisterSharedInvalid(message->any.catalog.cacheId,
message->any.catalog.hashIndex,
&message->any.catalog.pointerData);
break;
case 'r': /* cached relation descriptor */
InvalidationMessageRegisterSharedInvalid_DEBUG2;
RegisterSharedInvalid(message->any.relation.relationId,
message->any.relation.objectId,
(ItemPointer) NULL);
break;
default:
elog(FATAL,
"InvalidationMessageRegisterSharedInvalid: `%c' kind",
message->kind);
}
}
/*
* InvalidationMessageCacheInvalidate
*/
#ifdef INVALIDDEBUG
#define InvalidationMessageCacheInvalidate_DEBUG1 \
elog(DEBUG, "InvalidationMessageCacheInvalidate(c, %d, %d, [%d, %d])",\
message->any.catalog.cacheId,\
message->any.catalog.hashIndex,\
ItemPointerGetBlockNumber(&message->any.catalog.pointerData),\
ItemPointerGetOffsetNumber(&message->any.catalog.pointerData))
#define InvalidationMessageCacheInvalidate_DEBUG2 \
elog(DEBUG, "InvalidationMessageCacheInvalidate(r, %u, %u)", \
message->any.relation.relationId, \
message->any.relation.objectId)
#else
#define InvalidationMessageCacheInvalidate_DEBUG1
#define InvalidationMessageCacheInvalidate_DEBUG2
#endif /* defined(INVALIDDEBUG) */
* PrepareForTupleInvalidation
* Invoke functions for the tuple which register invalidation
* of catalog/relation cache.
*/
static void
InvalidationMessageCacheInvalidate(InvalidationMessage message)
PrepareForTupleInvalidation(Relation relation, HeapTuple tuple,
void (*CacheIdRegisterFunc) (int, Index,
ItemPointer, Oid),
void (*RelationIdRegisterFunc) (Oid, Oid))
{
Assert(PointerIsValid(message));
switch (message->kind)
{
case 'c': /* cached system catalog tuple */
InvalidationMessageCacheInvalidate_DEBUG1;
CacheIdInvalidate(message->any.catalog.cacheId,
message->any.catalog.hashIndex,
&message->any.catalog.pointerData);
break;
case 'r': /* cached relation descriptor */
InvalidationMessageCacheInvalidate_DEBUG2;
CacheIdInvalidate(message->any.relation.relationId,
message->any.relation.objectId,
(ItemPointer) NULL);
break;
Oid tupleRelId;
Oid relationId;
default:
elog(FATAL, "InvalidationMessageCacheInvalidate: `%c' kind",
message->kind);
}
}
if (IsBootstrapProcessingMode())
return;
/*
* PrepareToInvalidateRelationCache
/*
* We only need to worry about invalidation for tuples that are in
* system relations; user-relation tuples are never in catcaches and
* can't affect the relcache either.
*/
static void
PrepareToInvalidateRelationCache(Relation relation,
HeapTuple tuple,
void (*function) (Oid, Oid))
{
Oid relationId;
Oid objectId;
if (!IsSystemRelationName(NameStr(RelationGetForm(relation)->relname)))
return;
/*
* get the relation object id
* First let the catcache do its thing
*/
relationId = RelationGetRelid(relation);
PrepareToInvalidateCacheTuple(relation, tuple,
CacheIdRegisterFunc);
/*
* is it one of the ones we need to send an SI message for?
* Now, is this tuple one of the primary definers of a relcache entry?
*/
if (relationId == RelOid_pg_class)
objectId = tuple->t_data->t_oid;
else if (relationId == RelOid_pg_attribute)
objectId = ((Form_pg_attribute) GETSTRUCT(tuple))->attrelid;
tupleRelId = RelationGetRelid(relation);
if (tupleRelId == RelOid_pg_class)
relationId = tuple->t_data->t_oid;
else if (tupleRelId == RelOid_pg_attribute)
relationId = ((Form_pg_attribute) GETSTRUCT(tuple))->attrelid;
else
return;
/*
* register the relcache-invalidation action in the appropriate list
*/
Assert(PointerIsValid(function));
(*function) (relationId, objectId);
* Yes. We need to register a relcache invalidation event for the
* relation identified by relationId.
*
* KLUGE ALERT: we always send the relcache event with MyDatabaseId,
* even if the rel in question is shared. This essentially means that
* only backends in this same database will react to the relcache flush
* request. This is in fact appropriate, since only those backends could
* see our pg_class or pg_attribute change anyway. It looks a bit ugly
* though.
*/
(*RelationIdRegisterFunc) (MyDatabaseId, relationId);
}
/* ----------------------------------------------------------------
* public functions
* ----------------------------------------------------------------
*/
/*
* DiscardInvalid
* Causes the invalidated cache state to be discarded.
* AcceptInvalidationMessages
* Read and process invalidation messages from the shared invalidation
* message queue.
*
* Note:
* This should be called as the first step in processing a transaction.
*/
void
DiscardInvalid(void)
AcceptInvalidationMessages(void)
{
/*
* debugging stuff
*/
#ifdef INVALIDDEBUG
elog(DEBUG, "DiscardInvalid called");
#endif /* defined(INVALIDDEBUG) */
InvalidateSharedInvalid(CacheIdInvalidate, InvalidateSystemCaches);
ReceiveSharedInvalidMessages(LocalExecuteInvalidationMessage,
InvalidateSystemCaches);
}
/*
* RegisterInvalid
* Causes registration of invalidated state with other backends iff true.
* AtEOXactInvalidationMessages
* Process queued-up invalidation messages at end of transaction.
*
* If isCommit, we must send out the messages in our GlobalInvalidMsgs list
* to the shared invalidation message queue. Note that these will be read
* not only by other backends, but also by our own backend at the next
* transaction start (via AcceptInvalidationMessages). Therefore, it's okay
* to discard any pending LocalInvalidMsgs, since these will be redundant
* with the global list.
*
* If not isCommit, we are aborting, and must locally process the messages
* in our RollbackMsgs list. No messages need be sent to other backends,
* since they'll not have seen our changed tuples anyway.
*
* In any case, reset the various lists to empty. We need not physically
* free memory here, since TopTransactionContext is about to be emptied
* anyway.
*
* Note:
* This should be called as the last step in processing a transaction.
*/
void
RegisterInvalid(bool send)
AtEOXactInvalidationMessages(bool isCommit)
{
LocalInvalid invalid;
/*
* debugging stuff
*/
#ifdef INVALIDDEBUG
elog(DEBUG, "RegisterInvalid(%d) called", send);
#endif /* defined(INVALIDDEBUG) */
/*
* Process and free the current list of inval messages.
*/
DiscardInvalidStack(&InvalidLocal);
if (send)
if (isCommit)
{
DiscardInvalidStack(&RollbackStack);
invalid = InvalidForall;
InvalidForall = EmptyLocalInvalid; /* clear InvalidForall */
LocalInvalidInvalidate(invalid, InvalidationMessageRegisterSharedInvalid, true);
ProcessInvalidationMessages(&GlobalInvalidMsgs,
SendSharedInvalidMessage);
}
else
{
DiscardInvalidStack(&InvalidForall);
invalid = RollbackStack;
RollbackStack = EmptyLocalInvalid; /* clear RollbackStack */
LocalInvalidInvalidate(invalid, InvalidationMessageCacheInvalidate, true);
ProcessInvalidationMessages(&RollbackMsgs,
LocalExecuteInvalidationMessage);
}
DiscardInvalidationMessages(&GlobalInvalidMsgs, false);
DiscardInvalidationMessages(&LocalInvalidMsgs, false);
DiscardInvalidationMessages(&RollbackMsgs, false);
}
/*
* ImmediateLocalInvalidation
* Causes invalidation immediately for the next command of the transaction.
* CommandEndInvalidationMessages
* Process queued-up invalidation messages at end of one command
* in a transaction.
*
* Here, we send no messages to the shared queue, since we don't know yet if
* we will commit. But we do need to locally process the LocalInvalidMsgs
* list, so as to flush our caches of any tuples we have outdated in the
* current command.
*
* The isCommit = false case is not currently used, but may someday be
* needed to support rollback to a savepoint within a transaction.
* (I suspect it needs more work first --- tgl.)
*
* Note:
* This should be called during CommandCounterIncrement(),
* after we have advanced the command ID.
*/
void
ImmediateLocalInvalidation(bool send)
CommandEndInvalidationMessages(bool isCommit)
{
LocalInvalid invalid;
/*
* debugging stuff
*/
#ifdef INVALIDDEBUG
elog(DEBUG, "ImmediateLocalInvalidation(%d) called", send);
#endif /* defined(INVALIDDEBUG) */
/*
* Process and free the local list of inval messages.
*/
if (send)
if (isCommit)
{
invalid = InvalidLocal;
InvalidLocal = EmptyLocalInvalid; /* clear InvalidLocal */
LocalInvalidInvalidate(invalid, InvalidationMessageCacheInvalidate, true);
ProcessInvalidationMessages(&LocalInvalidMsgs,
LocalExecuteInvalidationMessage);
}
else
{
/*
* This may be used for rollback to a savepoint. Don't clear
* InvalidForall and RollbackStack here.
*/
DiscardInvalidStack(&InvalidLocal);
invalid = RollbackStack;
LocalInvalidInvalidate(invalid, InvalidationMessageCacheInvalidate, false);
ProcessInvalidationMessages(&RollbackMsgs,
LocalExecuteInvalidationMessage);
}
}
/*
* PrepareForTupleInvalidation
* Invoke functions for the tuple which register invalidation
* of catalog/relation cache.
* Note:
* Assumes object id is valid.
* Assumes tuple is valid.
*/
#ifdef INVALIDDEBUG
#define PrepareForTupleInvalidation_DEBUG1 \
elog(DEBUG, "%s(%s, [%d,%d])", \
funcname,\
RelationGetPhysicalRelationName(relation), \
ItemPointerGetBlockNumber(&tuple->t_self), \
ItemPointerGetOffsetNumber(&tuple->t_self))
#else
#define PrepareForTupleInvalidation_DEBUG1
#endif /* defined(INVALIDDEBUG) */
static void
PrepareForTupleInvalidation(Relation relation, HeapTuple tuple,
void (*CacheIdRegisterFunc) (int, Index,
ItemPointer),
void (*RelationIdRegisterFunc) (Oid, Oid),
const char *funcname)
{
/*
* sanity checks
*/
Assert(RelationIsValid(relation));
Assert(HeapTupleIsValid(tuple));
if (IsBootstrapProcessingMode())
return;
/*
* We only need to worry about invalidation for tuples that are in
* system relations; user-relation tuples are never in catcaches and
* can't affect the relcache either.
* LocalInvalidMsgs list is not interesting anymore, so flush it
* (for real). Do *not* clear GlobalInvalidMsgs or RollbackMsgs.
*/
if (!IsSystemRelationName(NameStr(RelationGetForm(relation)->relname)))
return;
/*
* debugging stuff
*/
PrepareForTupleInvalidation_DEBUG1;
PrepareToInvalidateCacheTuple(relation, tuple,
CacheIdRegisterFunc);
PrepareToInvalidateRelationCache(relation, tuple,
RelationIdRegisterFunc);
DiscardInvalidationMessages(&LocalInvalidMsgs, true);
}
/*
......@@ -800,9 +579,8 @@ void
RelationInvalidateHeapTuple(Relation relation, HeapTuple tuple)
{
PrepareForTupleInvalidation(relation, tuple,
CacheIdRegisterLocalInvalid,
RelationIdRegisterLocalInvalid,
"RelationInvalidateHeapTuple");
RegisterCatcacheInvalidation,
RegisterRelcacheInvalidation);
}
/*
......@@ -814,7 +592,6 @@ void
RelationMark4RollbackHeapTuple(Relation relation, HeapTuple tuple)
{
PrepareForTupleInvalidation(relation, tuple,
CacheIdRegisterLocalRollback,
RelationIdRegisterLocalRollback,
"RelationMark4RollbackHeapTuple");
RegisterCatcacheRollback,
RegisterRelcacheRollback);
}
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: sinval.h,v 1.18 2001/02/26 00:50:08 tgl Exp $
* $Id: sinval.h,v 1.19 2001/06/19 19:42:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -17,15 +17,61 @@
#include "storage/itemptr.h"
#include "storage/spin.h"
/*
* We currently support two types of shared-invalidation messages: one that
* invalidates an entry in a catcache, and one that invalidates a relcache
* entry. More types could be added if needed. The message type is
* identified by the first "int16" field of the message struct. Zero or
* positive means a catcache inval message (and also serves as the catcache
* ID field). -1 means a relcache inval message. Other negative values
* are available to identify other inval message types.
*
* Shared-inval events are initially driven by detecting tuple inserts,
* updates and deletions in system catalogs (see RelationInvalidateHeapTuple
* and RelationMark4RollbackHeapTuple). Note that some system catalogs have
* multiple caches on them (with different indexes). On detecting a tuple
* invalidation in such a catalog, a separate catcache inval message must be
* generated for each of its caches. The catcache inval message carries the
* hash index for the target tuple, so that the catcache only needs to search
* one hash chain not all its chains. Of course this assumes that all the
* backends are using identical hashing code, but that should be OK.
*/
typedef struct
{
int16 id; /* cache ID --- must be first */
uint16 hashIndex; /* hashchain index within this catcache */
Oid dbId; /* database ID, or 0 if a shared relation */
ItemPointerData tuplePtr; /* tuple identifier in cached relation */
} SharedInvalCatcacheMsg;
#define SHAREDINVALRELCACHE_ID (-1)
typedef struct
{
int16 id; /* type field --- must be first */
Oid dbId; /* database ID, or 0 if a shared relation */
Oid relId; /* relation ID */
} SharedInvalRelcacheMsg;
typedef union
{
int16 id; /* type field --- must be first */
SharedInvalCatcacheMsg cc;
SharedInvalRelcacheMsg rc;
} SharedInvalidationMessage;
extern SPINLOCK SInvalLock;
extern int SInvalShmemSize(int maxBackends);
extern void CreateSharedInvalidationState(int maxBackends);
extern void InitBackendSharedInvalidationState(void);
extern void RegisterSharedInvalid(int cacheId, Index hashIndex,
ItemPointer pointer);
extern void InvalidateSharedInvalid(void (*invalFunction) (),
void (*resetFunction) ());
extern void SendSharedInvalidMessage(SharedInvalidationMessage *msg);
extern void ReceiveSharedInvalidMessages(
void (*invalFunction) (SharedInvalidationMessage *msg),
void (*resetFunction) (void));
extern bool DatabaseHasActiveBackends(Oid databaseId, bool ignoreMyself);
extern bool TransactionIdIsInProgress(TransactionId xid);
......
......@@ -7,15 +7,15 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: sinvaladt.h,v 1.26 2001/03/22 04:01:09 momjian Exp $
* $Id: sinvaladt.h,v 1.27 2001/06/19 19:42:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#ifndef SINVALADT_H
#define SINVALADT_H
#include "storage/itemptr.h"
#include "storage/shmem.h"
#include "storage/sinval.h"
/*
* The shared cache invalidation manager is responsible for transmitting
......@@ -45,6 +45,9 @@
* large so that we don't need to do this often. It must be a multiple of
* MAXNUMMESSAGES so that the existing circular-buffer entries don't need
* to be moved when we do it.
*
* The struct type SharedInvalidationMessage, defining the contents of
* a single message, is defined in sinval.h.
*/
......@@ -61,15 +64,6 @@
#define MAXNUMMESSAGES 4096
#define MSGNUMWRAPAROUND (MAXNUMMESSAGES * 4096)
/* The content of one shared-invalidation message */
typedef struct SharedInvalidData
{
int cacheId; /* XXX */
Index hashIndex;
ItemPointerData pointerData;
} SharedInvalidData;
typedef SharedInvalidData *SharedInvalid;
/* Per-backend state in shared invalidation structure */
typedef struct ProcState
......@@ -83,7 +77,6 @@ typedef struct ProcState
/* Shared cache invalidation memory segment */
typedef struct SISeg
{
/*
* General state information
*/
......@@ -96,7 +89,7 @@ typedef struct SISeg
/*
* Circular buffer holding shared-inval messages
*/
SharedInvalidData buffer[MAXNUMMESSAGES];
SharedInvalidationMessage buffer[MAXNUMMESSAGES];
/*
* Per-backend state info.
......@@ -117,9 +110,9 @@ extern SISeg *shmInvalBuffer; /* pointer to the shared inval buffer */
extern void SIBufferInit(int maxBackends);
extern int SIBackendInit(SISeg *segP);
extern bool SIInsertDataEntry(SISeg *segP, SharedInvalidData *data);
extern bool SIInsertDataEntry(SISeg *segP, SharedInvalidationMessage *data);
extern int SIGetDataEntry(SISeg *segP, int backendId,
SharedInvalidData *data);
SharedInvalidationMessage *data);
extern void SIDelExpiredDataEntries(SISeg *segP);
#endif /* SINVALADT_H */
......@@ -13,7 +13,7 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: catcache.h,v 1.33 2001/06/18 03:35:07 tgl Exp $
* $Id: catcache.h,v 1.34 2001/06/19 19:42:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -36,6 +36,7 @@ typedef struct catcache
char *cc_relname; /* name of relation the tuples come from */
char *cc_indname; /* name of index matching cache keys */
int cc_reloidattr; /* AttrNumber of relation OID, or 0 */
bool cc_relisshared; /* is relation shared? */
TupleDesc cc_tupdesc; /* tuple descriptor (copied from reldesc) */
int cc_ntup; /* # of tuples currently in this cache */
int cc_size; /* # of hash buckets in this cache */
......@@ -99,6 +100,6 @@ extern void CatalogCacheIdInvalidate(int cacheId, Index hashIndex,
ItemPointer pointer);
extern void PrepareToInvalidateCacheTuple(Relation relation,
HeapTuple tuple,
void (*function) (int, Index, ItemPointer));
void (*function) (int, Index, ItemPointer, Oid));
#endif /* CATCACHE_H */
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: inval.h,v 1.19 2001/01/24 19:43:28 momjian Exp $
* $Id: inval.h,v 1.20 2001/06/19 19:42:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -16,11 +16,12 @@
#include "access/htup.h"
extern void DiscardInvalid(void);
extern void RegisterInvalid(bool send);
extern void AcceptInvalidationMessages(void);
extern void ImmediateLocalInvalidation(bool send);
extern void AtEOXactInvalidationMessages(bool isCommit);
extern void CommandEndInvalidationMessages(bool isCommit);
extern void RelationInvalidateHeapTuple(Relation relation, HeapTuple tuple);
......
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