* the heaptup data structure is all in local memory, not in the shared
* buffer.
*/
- CacheInvalidateHeapTuple(relation, heaptup);
+ CacheInvalidateHeapTuple(relation, heaptup, NULL);
pgstat_count_heap_insert(relation);
* boundary. We have to do this before releasing the buffer because we
* need to look at the contents of the tuple.
*/
- CacheInvalidateHeapTuple(relation, &tp);
+ CacheInvalidateHeapTuple(relation, &tp, NULL);
/* Now we can release the buffer */
ReleaseBuffer(buffer);
/*
* Mark old tuple for invalidation from system caches at next command
- * boundary. We have to do this before releasing the buffer because we
- * need to look at the contents of the tuple.
+ * boundary, and mark the new tuple for invalidation in case we abort.
+ * We have to do this before releasing the buffer because oldtup is in
+ * the buffer. (heaptup is all in local memory, but it's necessary to
+ * process both tuple versions in one call to inval.c so we can avoid
+ * redundant sinval messages.)
*/
- CacheInvalidateHeapTuple(relation, &oldtup);
+ CacheInvalidateHeapTuple(relation, &oldtup, heaptup);
/* Now we can release the buffer(s) */
if (newbuf != buffer)
if (BufferIsValid(vmbuffer))
ReleaseBuffer(vmbuffer);
- /*
- * If new tuple is cachable, mark it for invalidation from the caches in
- * case we abort. Note it is OK to do this after releasing the buffer,
- * because the heaptup data structure is all in local memory, not in the
- * shared buffer.
- */
- CacheInvalidateHeapTuple(relation, heaptup);
-
/*
* Release the lmgr tuple lock, if we had it.
*/
UnlockReleaseBuffer(buffer);
- /* Send out shared cache inval if necessary */
+ /*
+ * Send out shared cache inval if necessary. Note that because we only
+ * pass the new version of the tuple, this mustn't be used for any
+ * operations that could change catcache lookup keys. But we aren't
+ * bothering with index updates either, so that's true a fortiori.
+ */
if (!IsBootstrapProcessingMode())
- CacheInvalidateHeapTuple(relation, tuple);
+ CacheInvalidateHeapTuple(relation, tuple, NULL);
}
static void InitTempTableNamespace(void);
static void RemoveTempRelations(Oid tempNamespaceId);
static void RemoveTempRelationsCallback(int code, Datum arg);
-static void NamespaceCallback(Datum arg, int cacheid, ItemPointer tuplePtr);
+static void NamespaceCallback(Datum arg, int cacheid, uint32 hashvalue);
static bool MatchNamedCall(HeapTuple proctup, int nargs, List *argnames,
int **argnumbers);
* Syscache inval callback function
*/
static void
-NamespaceCallback(Datum arg, int cacheid, ItemPointer tuplePtr)
+NamespaceCallback(Datum arg, int cacheid, uint32 hashvalue)
{
/* Force search path to be recomputed on next use */
baseSearchPathValid = false;
PlanInvalItem *newnode = makeNode(PlanInvalItem);
COPY_SCALAR_FIELD(cacheId);
- /* tupleId isn't really a "scalar", but this works anyway */
- COPY_SCALAR_FIELD(tupleId);
+ COPY_SCALAR_FIELD(hashValue);
return newnode;
}
WRITE_NODE_TYPE("PLANINVALITEM");
WRITE_INT_FIELD(cacheId);
- appendStringInfo(str, " :tupleId (%u,%u)",
- ItemPointerGetBlockNumber(&node->tupleId),
- ItemPointerGetOffsetNumber(&node->tupleId));
+ WRITE_UINT_FIELD(hashValue);
}
/*****************************************************************************
*/
#include "postgres.h"
+#include "access/hash.h"
#include "access/transam.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
*/
if (funcid >= (Oid) FirstBootstrapObjectId)
{
- HeapTuple func_tuple;
- PlanInvalItem *inval_item;
-
- func_tuple = SearchSysCache1(PROCOID, ObjectIdGetDatum(funcid));
- if (!HeapTupleIsValid(func_tuple))
- elog(ERROR, "cache lookup failed for function %u", funcid);
-
- inval_item = makeNode(PlanInvalItem);
+ PlanInvalItem *inval_item = makeNode(PlanInvalItem);
/*
- * It would work to use any syscache on pg_proc, but plancache.c
- * expects us to use PROCOID.
+ * It would work to use any syscache on pg_proc, but the easiest is
+ * PROCOID since we already have the function's OID at hand. Note
+ * that plancache.c knows we use PROCOID. Also, we're perhaps
+ * assuming more than we should about how CatalogCacheComputeHashValue
+ * computes hash values...
*/
inval_item->cacheId = PROCOID;
- inval_item->tupleId = func_tuple->t_self;
+ inval_item->hashValue =
+ DatumGetUInt32(DirectFunctionCall1(hashoid,
+ ObjectIdGetDatum(funcid)));
glob->invalItems = lappend(glob->invalItems, inval_item);
-
- ReleaseSysCache(func_tuple);
}
}
static bool btree_predicate_proof(Expr *predicate, Node *clause,
bool refute_it);
static Oid get_btree_test_op(Oid pred_op, Oid clause_op, bool refute_it);
-static void InvalidateOprProofCacheCallBack(Datum arg, int cacheid, ItemPointer tuplePtr);
+static void InvalidateOprProofCacheCallBack(Datum arg, int cacheid, uint32 hashvalue);
/*
* Callback for pg_amop inval events
*/
static void
-InvalidateOprProofCacheCallBack(Datum arg, int cacheid, ItemPointer tuplePtr)
+InvalidateOprProofCacheCallBack(Datum arg, int cacheid, uint32 hashvalue)
{
HASH_SEQ_STATUS status;
OprProofCacheEntry *hentry;
Oid ltypeId, Oid rtypeId);
static Oid find_oper_cache_entry(OprCacheKey *key);
static void make_oper_cache_entry(OprCacheKey *key, Oid opr_oid);
-static void InvalidateOprCacheCallBack(Datum arg, int cacheid, ItemPointer tuplePtr);
+static void InvalidateOprCacheCallBack(Datum arg, int cacheid, uint32 hashvalue);
/*
* Callback for pg_operator and pg_cast inval events
*/
static void
-InvalidateOprCacheCallBack(Datum arg, int cacheid, ItemPointer tuplePtr)
+InvalidateOprCacheCallBack(Datum arg, int cacheid, uint32 hashvalue)
{
HASH_SEQ_STATUS status;
OprCacheEntry *hentry;
static AclMode convert_role_priv_string(text *priv_type_text);
static AclResult pg_role_aclcheck(Oid role_oid, Oid roleid, AclMode mode);
-static void RoleMembershipCacheCallback(Datum arg, int cacheid, ItemPointer tuplePtr);
+static void RoleMembershipCacheCallback(Datum arg, int cacheid, uint32 hashvalue);
static Oid get_role_oid_or_public(const char *rolname);
* Syscache inval callback function
*/
static void
-RoleMembershipCacheCallback(Datum arg, int cacheid, ItemPointer tuplePtr)
+RoleMembershipCacheCallback(Datum arg, int cacheid, uint32 hashvalue)
{
/* Force membership caches to be recomputed on next use */
cached_privs_role = InvalidOid;
* query execution), this seems OK.
*/
static void
-InvalidateAttoptCacheCallback(Datum arg, int cacheid, ItemPointer tuplePtr)
+InvalidateAttoptCacheCallback(Datum arg, int cacheid, uint32 hashvalue)
{
HASH_SEQ_STATUS status;
AttoptCacheEntry *attopt;
* target tuple that has to be invalidated has a different TID than it
* did when the event was created. So now we just compare hash values and
* accept the small risk of unnecessary invalidations due to false matches.
- * (The ItemPointer argument is therefore useless and should get removed.)
*
* This routine is only quasi-public: it should only be used by inval.c.
*/
void
-CatalogCacheIdInvalidate(int cacheId,
- uint32 hashValue,
- ItemPointer pointer)
+CatalogCacheIdInvalidate(int cacheId, uint32 hashValue)
{
CatCache *ccp;
- /*
- * sanity checks
- */
- Assert(ItemPointerIsValid(pointer));
CACHE1_elog(DEBUG2, "CatalogCacheIdInvalidate: called");
/*
ResetCatalogCache(cache);
/* Tell inval.c to call syscache callbacks for this cache */
- CallSyscacheCallbacks(cache->id, NULL);
+ CallSyscacheCallbacks(cache->id, 0);
}
}
* The lists of tuples that need to be flushed are kept by inval.c. This
* routine is a helper routine for inval.c. Given a tuple belonging to
* the specified relation, find all catcaches it could be in, compute the
- * correct hash value for each such catcache, and call the specified function
- * to record the cache id, hash value, and tuple ItemPointer in inval.c's
- * lists. CatalogCacheIdInvalidate will be called later, if appropriate,
+ * correct hash value for each such catcache, and call the specified
+ * function to record the cache id and hash value in inval.c's lists.
+ * CatalogCacheIdInvalidate will be called later, if appropriate,
* using the recorded information.
*
+ * For an insert or delete, tuple is the target tuple and newtuple is NULL.
+ * For an update, we are called just once, with tuple being the old tuple
+ * version and newtuple the new version. We should make two list entries
+ * if the tuple's hash value changed, but only one if it didn't.
+ *
* 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, or there might be a matching negative entry
void
PrepareToInvalidateCacheTuple(Relation relation,
HeapTuple tuple,
- void (*function) (int, uint32, ItemPointer, Oid))
+ HeapTuple newtuple,
+ void (*function) (int, uint32, Oid))
{
CatCache *ccp;
Oid reloid;
/* ----------------
* for each cache
* if the cache contains tuples from the specified relation
- * compute the tuple's hash value in this cache,
+ * compute the tuple's hash value(s) in this cache,
* and call the passed function to register the information.
* ----------------
*/
for (ccp = CacheHdr->ch_caches; ccp; ccp = ccp->cc_next)
{
+ uint32 hashvalue;
+ Oid dbid;
+
if (ccp->cc_reloid != reloid)
continue;
if (ccp->cc_tupdesc == NULL)
CatalogCacheInitializeCache(ccp);
- (*function) (ccp->id,
- CatalogCacheComputeTupleHashValue(ccp, tuple),
- &tuple->t_self,
- ccp->cc_relisshared ? (Oid) 0 : MyDatabaseId);
+ hashvalue = CatalogCacheComputeTupleHashValue(ccp, tuple);
+ dbid = ccp->cc_relisshared ? (Oid) 0 : MyDatabaseId;
+
+ (*function) (ccp->id, hashvalue, dbid);
+
+ if (newtuple)
+ {
+ uint32 newhashvalue;
+
+ newhashvalue = CatalogCacheComputeTupleHashValue(ccp, newtuple);
+
+ if (newhashvalue != hashvalue)
+ (*function) (ccp->id, newhashvalue, dbid);
+ }
}
}
*
* In short, we need to remember until xact end every insert or delete
* of a tuple that might be in the system caches. Updates are treated as
- * two events, delete + insert, for simplicity. (There are cases where
- * it'd be possible to record just one event, but we don't currently try.)
+ * two events, delete + insert, for simplicity. (If the update doesn't
+ * change the tuple hash value, catcache.c optimizes this into one event.)
*
* We do not need to register EVERY tuple operation in this way, just those
* on tuples in relations that have associated catcaches. We do, however,
*/
static void
AddCatcacheInvalidationMessage(InvalidationListHeader *hdr,
- int id, uint32 hashValue,
- ItemPointer tuplePtr, Oid dbId)
+ int id, uint32 hashValue, Oid dbId)
{
SharedInvalidationMessage msg;
Assert(id < CHAR_MAX);
msg.cc.id = (int8) id;
- msg.cc.tuplePtr = *tuplePtr;
msg.cc.dbId = dbId;
msg.cc.hashValue = hashValue;
AddInvalidationMessage(&hdr->cclist, &msg);
static void
RegisterCatcacheInvalidation(int cacheId,
uint32 hashValue,
- ItemPointer tuplePtr,
Oid dbId)
{
AddCatcacheInvalidationMessage(&transInvalInfo->CurrentCmdInvalidMsgs,
- cacheId, hashValue, tuplePtr, dbId);
+ cacheId, hashValue, dbId);
}
/*
{
if (msg->cc.dbId == MyDatabaseId || msg->cc.dbId == InvalidOid)
{
- CatalogCacheIdInvalidate(msg->cc.id,
- msg->cc.hashValue,
- &msg->cc.tuplePtr);
+ CatalogCacheIdInvalidate(msg->cc.id, msg->cc.hashValue);
- CallSyscacheCallbacks(msg->cc.id, &msg->cc.tuplePtr);
+ CallSyscacheCallbacks(msg->cc.id, msg->cc.hashValue);
}
}
else if (msg->id == SHAREDINVALCATALOG_ID)
{
struct SYSCACHECALLBACK *ccitem = syscache_callback_list + i;
- (*ccitem->function) (ccitem->arg, ccitem->id, NULL);
+ (*ccitem->function) (ccitem->arg, ccitem->id, 0);
}
for (i = 0; i < relcache_callback_count; i++)
}
}
-/*
- * PrepareForTupleInvalidation
- * Detect whether invalidation of this tuple implies invalidation
- * of catalog/relation cache entries; if so, register inval events.
- */
-static void
-PrepareForTupleInvalidation(Relation relation, HeapTuple tuple)
-{
- Oid tupleRelId;
- Oid databaseId;
- Oid relationId;
-
- /* Do nothing during bootstrap */
- 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.
- */
- if (!IsSystemRelation(relation))
- return;
-
- /*
- * TOAST tuples can likewise be ignored here. Note that TOAST tables are
- * considered system relations so they are not filtered by the above test.
- */
- if (IsToastRelation(relation))
- return;
-
- /*
- * First let the catcache do its thing
- */
- PrepareToInvalidateCacheTuple(relation, tuple,
- RegisterCatcacheInvalidation);
-
- /*
- * Now, is this tuple one of the primary definers of a relcache entry?
- */
- tupleRelId = RelationGetRelid(relation);
-
- if (tupleRelId == RelationRelationId)
- {
- Form_pg_class classtup = (Form_pg_class) GETSTRUCT(tuple);
-
- relationId = HeapTupleGetOid(tuple);
- if (classtup->relisshared)
- databaseId = InvalidOid;
- else
- databaseId = MyDatabaseId;
- }
- else if (tupleRelId == AttributeRelationId)
- {
- Form_pg_attribute atttup = (Form_pg_attribute) GETSTRUCT(tuple);
-
- relationId = atttup->attrelid;
-
- /*
- * KLUGE ALERT: we always send the relcache event with MyDatabaseId,
- * even if the rel in question is shared (which we can't easily tell).
- * 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_attribute
- * change anyway. It looks a bit ugly though. (In practice, shared
- * relations can't have schema changes after bootstrap, so we should
- * never come here for a shared rel anyway.)
- */
- databaseId = MyDatabaseId;
- }
- else if (tupleRelId == IndexRelationId)
- {
- Form_pg_index indextup = (Form_pg_index) GETSTRUCT(tuple);
-
- /*
- * When a pg_index row is updated, we should send out a relcache inval
- * for the index relation. As above, we don't know the shared status
- * of the index, but in practice it doesn't matter since indexes of
- * shared catalogs can't have such updates.
- */
- relationId = indextup->indexrelid;
- databaseId = MyDatabaseId;
- }
- else
- return;
-
- /*
- * Yes. We need to register a relcache invalidation event.
- */
- RegisterRelcacheInvalidation(databaseId, relationId);
-}
-
/* ----------------------------------------------------------------
* public functions
* CacheInvalidateHeapTuple
* Register the given tuple for invalidation at end of command
* (ie, current command is creating or outdating this tuple).
+ * Also, detect whether a relcache invalidation is implied.
+ *
+ * For an insert or delete, tuple is the target tuple and newtuple is NULL.
+ * For an update, we are called just once, with tuple being the old tuple
+ * version and newtuple the new version. This allows avoidance of duplicate
+ * effort during an update.
*/
void
-CacheInvalidateHeapTuple(Relation relation, HeapTuple tuple)
+CacheInvalidateHeapTuple(Relation relation,
+ HeapTuple tuple,
+ HeapTuple newtuple)
{
- PrepareForTupleInvalidation(relation, tuple);
+ Oid tupleRelId;
+ Oid databaseId;
+ Oid relationId;
+
+ /* Do nothing during bootstrap */
+ 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.
+ */
+ if (!IsSystemRelation(relation))
+ return;
+
+ /*
+ * TOAST tuples can likewise be ignored here. Note that TOAST tables are
+ * considered system relations so they are not filtered by the above test.
+ */
+ if (IsToastRelation(relation))
+ return;
+
+ /*
+ * First let the catcache do its thing
+ */
+ PrepareToInvalidateCacheTuple(relation, tuple, newtuple,
+ RegisterCatcacheInvalidation);
+
+ /*
+ * Now, is this tuple one of the primary definers of a relcache entry?
+ *
+ * Note we ignore newtuple here; we assume an update cannot move a tuple
+ * from being part of one relcache entry to being part of another.
+ */
+ tupleRelId = RelationGetRelid(relation);
+
+ if (tupleRelId == RelationRelationId)
+ {
+ Form_pg_class classtup = (Form_pg_class) GETSTRUCT(tuple);
+
+ relationId = HeapTupleGetOid(tuple);
+ if (classtup->relisshared)
+ databaseId = InvalidOid;
+ else
+ databaseId = MyDatabaseId;
+ }
+ else if (tupleRelId == AttributeRelationId)
+ {
+ Form_pg_attribute atttup = (Form_pg_attribute) GETSTRUCT(tuple);
+
+ relationId = atttup->attrelid;
+
+ /*
+ * KLUGE ALERT: we always send the relcache event with MyDatabaseId,
+ * even if the rel in question is shared (which we can't easily tell).
+ * 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_attribute
+ * change anyway. It looks a bit ugly though. (In practice, shared
+ * relations can't have schema changes after bootstrap, so we should
+ * never come here for a shared rel anyway.)
+ */
+ databaseId = MyDatabaseId;
+ }
+ else if (tupleRelId == IndexRelationId)
+ {
+ Form_pg_index indextup = (Form_pg_index) GETSTRUCT(tuple);
+
+ /*
+ * When a pg_index row is updated, we should send out a relcache inval
+ * for the index relation. As above, we don't know the shared status
+ * of the index, but in practice it doesn't matter since indexes of
+ * shared catalogs can't have such updates.
+ */
+ relationId = indextup->indexrelid;
+ databaseId = MyDatabaseId;
+ }
+ else
+ return;
+
+ /*
+ * Yes. We need to register a relcache invalidation event.
+ */
+ RegisterRelcacheInvalidation(databaseId, relationId);
}
/*
*
* This is used in places that need to force relcache rebuild but aren't
* changing any of the tuples recognized as contributors to the relcache
- * entry by PrepareForTupleInvalidation. (An example is dropping an index.)
+ * entry by CacheInvalidateHeapTuple. (An example is dropping an index.)
*/
void
CacheInvalidateRelcache(Relation relation)
* CacheRegisterSyscacheCallback
* Register the specified function to be called for all future
* invalidation events in the specified cache. The cache ID and the
- * TID of the tuple being invalidated will be passed to the function.
+ * hash value of the tuple being invalidated will be passed to the
+ * function.
*
- * NOTE: NULL will be passed for the TID if a cache reset request is received.
+ * NOTE: Hash value zero will be passed if a cache reset request is received.
* In this case the called routines should flush all cached state.
+ * Yes, there's a possibility of a false match to zero, but it doesn't seem
+ * worth troubling over, especially since most of the current callees just
+ * flush all cached state anyway.
*/
void
CacheRegisterSyscacheCallback(int cacheid,
* this module from knowing which catcache IDs correspond to which catalogs.
*/
void
-CallSyscacheCallbacks(int cacheid, ItemPointer tuplePtr)
+CallSyscacheCallbacks(int cacheid, uint32 hashvalue)
{
int i;
struct SYSCACHECALLBACK *ccitem = syscache_callback_list + i;
if (ccitem->id == cacheid)
- (*ccitem->function) (ccitem->arg, cacheid, tuplePtr);
+ (*ccitem->function) (ccitem->arg, cacheid, hashvalue);
}
}
static bool ScanQueryWalker(Node *node, bool *acquire);
static bool plan_list_is_transient(List *stmt_list);
static void PlanCacheRelCallback(Datum arg, Oid relid);
-static void PlanCacheFuncCallback(Datum arg, int cacheid, ItemPointer tuplePtr);
-static void PlanCacheSysCallback(Datum arg, int cacheid, ItemPointer tuplePtr);
+static void PlanCacheFuncCallback(Datum arg, int cacheid, uint32 hashvalue);
+static void PlanCacheSysCallback(Datum arg, int cacheid, uint32 hashvalue);
/*
* PlanCacheFuncCallback
* Syscache inval callback function for PROCOID cache
*
- * Invalidate all plans mentioning the given catalog entry, or all plans
- * mentioning any member of this cache if tuplePtr == NULL.
+ * Invalidate all plans mentioning the object with the specified hash value,
+ * or all plans mentioning any member of this cache if hashvalue == 0.
*
* Note that the coding would support use for multiple caches, but right
* now only user-defined functions are tracked this way.
*/
static void
-PlanCacheFuncCallback(Datum arg, int cacheid, ItemPointer tuplePtr)
+PlanCacheFuncCallback(Datum arg, int cacheid, uint32 hashvalue)
{
ListCell *lc1;
if (item->cacheId != cacheid)
continue;
- if (tuplePtr == NULL ||
- ItemPointerEquals(tuplePtr, &item->tupleId))
+ if (hashvalue == 0 ||
+ item->hashValue == hashvalue)
{
/* Invalidate the plan! */
plan->dead = true;
if (item->cacheId != cacheid)
continue;
- if (tuplePtr == NULL ||
- ItemPointerEquals(tuplePtr, &item->tupleId))
+ if (hashvalue == 0 ||
+ item->hashValue == hashvalue)
{
/* Invalidate the plan! */
plan->dead = true;
* Just invalidate everything...
*/
static void
-PlanCacheSysCallback(Datum arg, int cacheid, ItemPointer tuplePtr)
+PlanCacheSysCallback(Datum arg, int cacheid, uint32 hashvalue)
{
ResetPlanCache();
}
* tablespaces, nor do we expect them to be frequently modified.
*/
static void
-InvalidateTableSpaceCacheCallback(Datum arg, int cacheid, ItemPointer tuplePtr)
+InvalidateTableSpaceCacheCallback(Datum arg, int cacheid, uint32 hashvalue)
{
HASH_SEQ_STATUS status;
TableSpaceCacheEntry *spc;
* table address as the "arg".
*/
static void
-InvalidateTSCacheCallBack(Datum arg, int cacheid, ItemPointer tuplePtr)
+InvalidateTSCacheCallBack(Datum arg, int cacheid, uint32 hashvalue)
{
HTAB *hash = (HTAB *) DatumGetPointer(arg);
HASH_SEQ_STATUS status;
static bool last_roleid_is_super = false;
static bool roleid_callback_registered = false;
-static void RoleidCallback(Datum arg, int cacheid, ItemPointer tuplePtr);
+static void RoleidCallback(Datum arg, int cacheid, uint32 hashvalue);
/*
}
/*
- * UseridCallback
+ * RoleidCallback
* Syscache inval callback function
*/
static void
-RoleidCallback(Datum arg, int cacheid, ItemPointer tuplePtr)
+RoleidCallback(Datum arg, int cacheid, uint32 hashvalue)
{
/* Invalidate our local cache in case role's superuserness changed */
last_roleid = InvalidOid;
#include "access/sdir.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
-#include "storage/itemptr.h"
/* ----------------------------------------------------------------
* relations are recorded as a simple list of OIDs, and everything else
* is represented as a list of PlanInvalItems. A PlanInvalItem is designed
* to be used with the syscache invalidation mechanism, so it identifies a
- * system catalog entry by cache ID and tuple TID.
+ * system catalog entry by cache ID and hash value.
*/
typedef struct PlanInvalItem
{
NodeTag type;
int cacheId; /* a syscache ID, see utils/syscache.h */
- ItemPointerData tupleId; /* TID of the object's catalog tuple */
+ uint32 hashValue; /* hash value of object's cache lookup key */
} PlanInvalItem;
#endif /* PLANNODES_H */
#ifndef SINVAL_H
#define SINVAL_H
-#include "storage/itemptr.h"
#include "storage/relfilenode.h"
*
* Catcache inval events are initially driven by detecting tuple inserts,
* updates and deletions in system catalogs (see CacheInvalidateHeapTuple).
- * An update generates two inval events, one for the old tuple and one for
- * the new --- this is needed to get rid of both positive entries for the
- * old tuple, and negative cache entries associated with the new tuple's
- * cache key. (This could perhaps be optimized down to one event when the
- * cache key is not changing, but for now we don't bother to try.) Note that
- * the inval events themselves don't actually say whether the tuple is being
- * inserted or deleted.
+ * An update can generate two inval events, one for the old tuple and one for
+ * the new, but this is reduced to one event if the tuple's hash key doesn't
+ * change. Note that the inval events themselves don't actually say whether
+ * the tuple is being inserted or deleted. Also, since we transmit only a
+ * hash key, there is a small risk of unnecessary invalidations due to chance
+ * matches of hash keys.
*
* Note that some system catalogs have multiple caches on them (with different
* indexes). On detecting a tuple invalidation in such a catalog, separate
- * catcache inval messages must be generated for each of its caches. The
- * catcache inval messages carry the hash value for the target tuple, so
- * that the catcache only needs to search one hash chain not all its chains,
- * and so that negative cache entries can be recognized with good accuracy.
- * (Of course this assumes that all the backends are using identical hashing
- * code, but that should be OK.)
+ * catcache inval messages must be generated for each of its caches, since
+ * the hash keys will generally be different.
*
* Catcache and relcache invalidations are transactional, and so are sent
* to other backends upon commit. Internally to the generating backend,
typedef struct
{
- /* note: field layout chosen with an eye to alignment concerns */
int8 id; /* cache ID --- must be first */
- ItemPointerData tuplePtr; /* tuple identifier in cached relation */
Oid dbId; /* database ID, or 0 if a shared relation */
uint32 hashValue; /* hash value of key for this catcache */
} SharedInvalCatcacheMsg;
typedef struct
{
+ /* note: field layout chosen to pack into 16 bytes */
int8 id; /* type field --- must be first */
int8 backend_hi; /* high bits of backend ID, if temprel */
uint16 backend_lo; /* low bits of backend ID, if temprel */
extern void ResetCatalogCaches(void);
extern void CatalogCacheFlushCatalog(Oid catId);
-extern void CatalogCacheIdInvalidate(int cacheId, uint32 hashValue,
- ItemPointer pointer);
+extern void CatalogCacheIdInvalidate(int cacheId, uint32 hashValue);
extern void PrepareToInvalidateCacheTuple(Relation relation,
HeapTuple tuple,
- void (*function) (int, uint32, ItemPointer, Oid));
+ HeapTuple newtuple,
+ void (*function) (int, uint32, Oid));
extern void PrintCatCacheLeakWarning(HeapTuple tuple);
extern void PrintCatCacheListLeakWarning(CatCList *list);
#include "utils/relcache.h"
-typedef void (*SyscacheCallbackFunction) (Datum arg, int cacheid, ItemPointer tuplePtr);
+typedef void (*SyscacheCallbackFunction) (Datum arg, int cacheid, uint32 hashvalue);
typedef void (*RelcacheCallbackFunction) (Datum arg, Oid relid);
extern void CommandEndInvalidationMessages(void);
-extern void CacheInvalidateHeapTuple(Relation relation, HeapTuple tuple);
+extern void CacheInvalidateHeapTuple(Relation relation,
+ HeapTuple tuple,
+ HeapTuple newtuple);
extern void CacheInvalidateCatalog(Oid catalogId);
extern void CacheRegisterRelcacheCallback(RelcacheCallbackFunction func,
Datum arg);
-extern void CallSyscacheCallbacks(int cacheid, ItemPointer tuplePtr);
+extern void CallSyscacheCallbacks(int cacheid, uint32 hashvalue);
extern void inval_twophase_postcommit(TransactionId xid, uint16 info,
void *recdata, uint32 len);