Location via proxy:   [ UP ]  
[Report a bug]   [Manage cookies]                
summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
Diffstat (limited to 'src/backend')
-rw-r--r--src/backend/access/heap/heapam.c60
-rw-r--r--src/backend/access/heap/heapam_xlog.c6
-rw-r--r--src/backend/access/rmgrdesc/heapdesc.c4
-rw-r--r--src/backend/access/rmgrdesc/standbydesc.c6
-rw-r--r--src/backend/access/transam/xact.c26
-rw-r--r--src/backend/catalog/index.c11
-rw-r--r--src/backend/commands/event_trigger.c5
-rw-r--r--src/backend/replication/logical/decode.c26
-rw-r--r--src/backend/utils/cache/catcache.c7
-rw-r--r--src/backend/utils/cache/inval.c348
-rw-r--r--src/backend/utils/cache/syscache.c3
11 files changed, 368 insertions, 134 deletions
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 82a0492aac5..3a13671a1ef 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6326,6 +6326,9 @@ heap_inplace_update_and_unlock(Relation relation,
HeapTupleHeader htup = oldtup->t_data;
uint32 oldlen;
uint32 newlen;
+ int nmsgs = 0;
+ SharedInvalidationMessage *invalMessages = NULL;
+ bool RelcacheInitFileInval = false;
Assert(ItemPointerEquals(&oldtup->t_self, &tuple->t_self));
oldlen = oldtup->t_len - htup->t_hoff;
@@ -6333,6 +6336,29 @@ heap_inplace_update_and_unlock(Relation relation,
if (oldlen != newlen || htup->t_hoff != tuple->t_data->t_hoff)
elog(ERROR, "wrong tuple length");
+ /*
+ * Construct 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.
+ */
+ CacheInvalidateHeapTupleInplace(relation, tuple, NULL);
+
+ /* Like RecordTransactionCommit(), log only if needed */
+ if (XLogStandbyInfoActive())
+ nmsgs = inplaceGetInvalidationMessages(&invalMessages,
+ &RelcacheInitFileInval);
+
+ /*
+ * Unlink relcache init files as needed. If unlinking, acquire
+ * RelCacheInitLock until after associated invalidations. By doing this
+ * in advance, if we checkpoint and then crash between inplace
+ * XLogInsert() and inval, we don't rely on StartupXLOG() ->
+ * RelationCacheInitFileRemove(). That uses elevel==LOG, so replay would
+ * neglect to PANIC on EIO.
+ */
+ PreInplace_Inval();
+
/* NO EREPORT(ERROR) from here till changes are logged */
START_CRIT_SECTION();
@@ -6362,9 +6388,16 @@ heap_inplace_update_and_unlock(Relation relation,
XLogRecPtr recptr;
xlrec.offnum = ItemPointerGetOffsetNumber(&tuple->t_self);
+ xlrec.dbId = MyDatabaseId;
+ xlrec.tsId = MyDatabaseTableSpace;
+ xlrec.relcacheInitFileInval = RelcacheInitFileInval;
+ xlrec.nmsgs = nmsgs;
XLogBeginInsert();
- XLogRegisterData((char *) &xlrec, SizeOfHeapInplace);
+ XLogRegisterData((char *) &xlrec, MinSizeOfHeapInplace);
+ if (nmsgs != 0)
+ XLogRegisterData((char *) invalMessages,
+ nmsgs * sizeof(SharedInvalidationMessage));
XLogRegisterBuffer(0, buffer, REGBUF_STANDARD);
XLogRegisterBufData(0, (char *) htup + htup->t_hoff, newlen);
@@ -6376,17 +6409,28 @@ heap_inplace_update_and_unlock(Relation relation,
PageSetLSN(BufferGetPage(buffer), recptr);
}
+ LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+
+ /*
+ * Send invalidations to shared queue. SearchSysCacheLocked1() assumes we
+ * do this before UnlockTuple().
+ *
+ * If we're mutating a tuple visible only to this transaction, there's an
+ * equivalent transactional inval from the action that created the tuple,
+ * and this inval is superfluous.
+ */
+ AtInplace_Inval();
+
END_CRIT_SECTION();
+ UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
- heap_inplace_unlock(relation, oldtup, buffer);
+ AcceptInvalidationMessages(); /* local processing of just-sent inval */
/*
- * 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.
- *
- * XXX ROLLBACK discards the invalidation. See test inplace-inval.spec.
+ * Queue a transactional inval. The immediate invalidation we just sent
+ * is the only one known to be necessary. To reduce risk from the
+ * transition to immediate invalidation, continue sending a transactional
+ * invalidation like we've long done. Third-party code might rely on it.
*/
if (!IsBootstrapProcessingMode())
CacheInvalidateHeapTuple(relation, tuple, NULL);
diff --git a/src/backend/access/heap/heapam_xlog.c b/src/backend/access/heap/heapam_xlog.c
index 6dae7233ecb..c5208f3df61 100644
--- a/src/backend/access/heap/heapam_xlog.c
+++ b/src/backend/access/heap/heapam_xlog.c
@@ -1170,6 +1170,12 @@ heap_xlog_inplace(XLogReaderState *record)
}
if (BufferIsValid(buffer))
UnlockReleaseBuffer(buffer);
+
+ ProcessCommittedInvalidationMessages(xlrec->msgs,
+ xlrec->nmsgs,
+ xlrec->relcacheInitFileInval,
+ xlrec->dbId,
+ xlrec->tsId);
}
void
diff --git a/src/backend/access/rmgrdesc/heapdesc.c b/src/backend/access/rmgrdesc/heapdesc.c
index 5f5673e088b..f31cc3a4df4 100644
--- a/src/backend/access/rmgrdesc/heapdesc.c
+++ b/src/backend/access/rmgrdesc/heapdesc.c
@@ -16,6 +16,7 @@
#include "access/heapam_xlog.h"
#include "access/rmgrdesc_utils.h"
+#include "storage/standbydefs.h"
/*
* NOTE: "keyname" argument cannot have trailing spaces or punctuation
@@ -253,6 +254,9 @@ heap_desc(StringInfo buf, XLogReaderState *record)
xl_heap_inplace *xlrec = (xl_heap_inplace *) rec;
appendStringInfo(buf, "off: %u", xlrec->offnum);
+ standby_desc_invalidations(buf, xlrec->nmsgs, xlrec->msgs,
+ xlrec->dbId, xlrec->tsId,
+ xlrec->relcacheInitFileInval);
}
}
diff --git a/src/backend/access/rmgrdesc/standbydesc.c b/src/backend/access/rmgrdesc/standbydesc.c
index 25f870b187e..32e509a4006 100644
--- a/src/backend/access/rmgrdesc/standbydesc.c
+++ b/src/backend/access/rmgrdesc/standbydesc.c
@@ -96,11 +96,7 @@ standby_identify(uint8 info)
return id;
}
-/*
- * This routine is used by both standby_desc and xact_desc, because
- * transaction commits and XLOG_INVALIDATIONS messages contain invalidations;
- * it seems pointless to duplicate the code.
- */
+/* also used by non-standby records having analogous invalidation fields */
void
standby_desc_invalidations(StringInfo buf,
int nmsgs, SharedInvalidationMessage *msgs,
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index d8f6c658420..004f7e10e55 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1369,14 +1369,24 @@ RecordTransactionCommit(void)
/*
* Transactions without an assigned xid can contain invalidation
- * messages (e.g. explicit relcache invalidations or catcache
- * invalidations for inplace updates); standbys need to process those.
- * We can't emit a commit record without an xid, and we don't want to
- * force assigning an xid, because that'd be problematic for e.g.
- * vacuum. Hence we emit a bespoke record for the invalidations. We
- * don't want to use that in case a commit record is emitted, so they
- * happen synchronously with commits (besides not wanting to emit more
- * WAL records).
+ * messages. While inplace updates do this, this is not known to be
+ * necessary; see comment at inplace CacheInvalidateHeapTuple().
+ * Extensions might still rely on this capability, and standbys may
+ * need to process those invals. We can't emit a commit record
+ * without an xid, and we don't want to force assigning an xid,
+ * because that'd be problematic for e.g. vacuum. Hence we emit a
+ * bespoke record for the invalidations. We don't want to use that in
+ * case a commit record is emitted, so they happen synchronously with
+ * commits (besides not wanting to emit more WAL records).
+ *
+ * XXX Every known use of this capability is a defect. Since an XID
+ * isn't controlling visibility of the change that prompted invals,
+ * other sessions need the inval even if this transactions aborts.
+ *
+ * ON COMMIT DELETE ROWS does a nontransactional index_build(), which
+ * queues a relcache inval, including in transactions without an xid
+ * that had read the (empty) table. Standbys don't need any ON COMMIT
+ * DELETE ROWS invals, but we've not done the work to withhold them.
*/
if (nmsgs != 0)
{
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 12822d0b140..9162b9f81a2 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2918,12 +2918,19 @@ index_update_stats(Relation rel,
if (dirty)
{
systable_inplace_update_finish(state, tuple);
- /* the above sends a cache inval message */
+ /* the above sends transactional and immediate cache inval messages */
}
else
{
systable_inplace_update_cancel(state);
- /* no need to change tuple, but force relcache inval anyway */
+
+ /*
+ * While we didn't change relhasindex, CREATE INDEX needs a
+ * transactional inval for when the new index's catalog rows become
+ * visible. Other CREATE INDEX and REINDEX code happens to also queue
+ * this inval, but keep this in case rare callers rely on this part of
+ * our API contract.
+ */
CacheInvalidateRelcacheByTuple(tuple);
}
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 05a6de68ba3..a586d246ece 100644
--- a/src/backend/commands/event_trigger.c
+++ b/src/backend/commands/event_trigger.c
@@ -975,11 +975,6 @@ EventTriggerOnLogin(void)
* this instead of regular updates serves two purposes. First,
* that avoids possible waiting on the row-level lock. Second,
* that avoids dealing with TOAST.
- *
- * Changes made by inplace update may be lost due to
- * concurrent normal updates; see inplace-inval.spec. However,
- * we are OK with that. The subsequent connections will still
- * have a chance to set "dathasloginevt" to false.
*/
systable_inplace_update_finish(state, tuple);
}
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index d687ceee339..e73576ad12f 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -509,23 +509,19 @@ heap_decode(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
/*
* Inplace updates are only ever performed on catalog tuples and
- * can, per definition, not change tuple visibility. Since we
- * don't decode catalog tuples, we're not interested in the
- * record's contents.
+ * can, per definition, not change tuple visibility. Inplace
+ * updates don't affect storage or interpretation of table rows,
+ * so they don't affect logicalrep_write_tuple() outcomes. Hence,
+ * we don't process invalidations from the original operation. If
+ * inplace updates did affect those things, invalidations wouldn't
+ * make it work, since there are no snapshot-specific versions of
+ * inplace-updated values. Since we also don't decode catalog
+ * tuples, we're not interested in the record's contents.
*
- * In-place updates can be used either by XID-bearing transactions
- * (e.g. in CREATE INDEX CONCURRENTLY) or by XID-less
- * transactions (e.g. VACUUM). In the former case, the commit
- * record will include cache invalidations, so we mark the
- * transaction as catalog modifying here. Currently that's
- * redundant because the commit will do that as well, but once we
- * support decoding in-progress relations, this will be important.
+ * WAL contains likely-unnecessary commit-time invals from the
+ * CacheInvalidateHeapTuple() call in heap_inplace_update().
+ * Excess invalidation is safe.
*/
- if (!TransactionIdIsValid(xid))
- break;
-
- (void) SnapBuildProcessChange(builder, xid, buf->origptr);
- ReorderBufferXidSetCatalogChanges(ctx->reorder, xid, buf->origptr);
break;
case XLOG_HEAP_CONFIRM:
diff --git a/src/backend/utils/cache/catcache.c b/src/backend/utils/cache/catcache.c
index 10276aa1db1..ee303dc501d 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2286,7 +2286,8 @@ void
PrepareToInvalidateCacheTuple(Relation relation,
HeapTuple tuple,
HeapTuple newtuple,
- void (*function) (int, uint32, Oid))
+ void (*function) (int, uint32, Oid, void *),
+ void *context)
{
slist_iter iter;
Oid reloid;
@@ -2327,7 +2328,7 @@ PrepareToInvalidateCacheTuple(Relation relation,
hashvalue = CatalogCacheComputeTupleHashValue(ccp, ccp->cc_nkeys, tuple);
dbid = ccp->cc_relisshared ? (Oid) 0 : MyDatabaseId;
- (*function) (ccp->id, hashvalue, dbid);
+ (*function) (ccp->id, hashvalue, dbid, context);
if (newtuple)
{
@@ -2336,7 +2337,7 @@ PrepareToInvalidateCacheTuple(Relation relation,
newhashvalue = CatalogCacheComputeTupleHashValue(ccp, ccp->cc_nkeys, newtuple);
if (newhashvalue != hashvalue)
- (*function) (ccp->id, newhashvalue, dbid);
+ (*function) (ccp->id, newhashvalue, dbid, context);
}
}
}
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index 603aa4157be..986850ccda9 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -94,6 +94,10 @@
* worth trying to avoid sending such inval traffic in the future, if those
* problems can be overcome cheaply.
*
+ * When making a nontransactional change to a cacheable object, we must
+ * likewise send the invalidation immediately, before ending the change's
+ * critical section. This includes inplace heap updates, relmap, and smgr.
+ *
* When wal_level=logical, write invalidations into WAL at each command end to
* support the decoding of the in-progress transactions. See
* CommandEndInvalidationMessages.
@@ -130,13 +134,15 @@
/*
* Pending requests are stored as ready-to-send SharedInvalidationMessages.
- * We keep the messages themselves in arrays in TopTransactionContext
- * (there are separate arrays for catcache and relcache messages). Control
- * information is kept in a chain of TransInvalidationInfo structs, also
- * allocated in TopTransactionContext. (We could keep a subtransaction's
- * TransInvalidationInfo in its CurTransactionContext; but that's more
- * wasteful not less so, since in very many scenarios it'd be the only
- * allocation in the subtransaction's CurTransactionContext.)
+ * We keep the messages themselves in arrays in TopTransactionContext (there
+ * are separate arrays for catcache and relcache messages). For transactional
+ * messages, control information is kept in a chain of TransInvalidationInfo
+ * structs, also allocated in TopTransactionContext. (We could keep a
+ * subtransaction's TransInvalidationInfo in its CurTransactionContext; but
+ * that's more wasteful not less so, since in very many scenarios it'd be the
+ * only allocation in the subtransaction's CurTransactionContext.) For
+ * inplace update messages, control information appears in an
+ * InvalidationInfo, allocated in CurrentMemoryContext.
*
* We can store the message arrays densely, and yet avoid moving data around
* within an array, because within any one subtransaction we need only
@@ -147,7 +153,9 @@
* struct. Similarly, we need distinguish messages of prior subtransactions
* from those of the current subtransaction only until the subtransaction
* completes, after which we adjust the array indexes in the parent's
- * TransInvalidationInfo to include the subtransaction's messages.
+ * TransInvalidationInfo to include the subtransaction's messages. Inplace
+ * invalidations don't need a concept of command or subtransaction boundaries,
+ * since we send them during the WAL insertion critical section.
*
* The ordering of the individual messages within a command's or
* subtransaction's output is not considered significant, although this
@@ -200,7 +208,7 @@ typedef struct InvalidationMsgsGroup
/*----------------
- * Invalidation messages are divided into two groups:
+ * Transactional invalidation messages are divided into two groups:
* 1) events so far in current command, not yet reflected to caches.
* 2) events in previous commands of current transaction; these have
* been reflected to local caches, and must be either broadcast to
@@ -216,26 +224,36 @@ typedef struct InvalidationMsgsGroup
*----------------
*/
-typedef struct TransInvalidationInfo
+/* fields common to both transactional and inplace invalidation */
+typedef struct InvalidationInfo
{
- /* Back link to parent transaction's info */
- struct TransInvalidationInfo *parent;
-
- /* Subtransaction nesting depth */
- int my_level;
-
/* Events emitted by current command */
InvalidationMsgsGroup CurrentCmdInvalidMsgs;
+ /* init file must be invalidated? */
+ bool RelcacheInitFileInval;
+} InvalidationInfo;
+
+/* subclass adding fields specific to transactional invalidation */
+typedef struct TransInvalidationInfo
+{
+ /* Base class */
+ struct InvalidationInfo ii;
+
/* Events emitted by previous commands of this (sub)transaction */
InvalidationMsgsGroup PriorCmdInvalidMsgs;
- /* init file must be invalidated? */
- bool RelcacheInitFileInval;
+ /* Back link to parent transaction's info */
+ struct TransInvalidationInfo *parent;
+
+ /* Subtransaction nesting depth */
+ int my_level;
} TransInvalidationInfo;
static TransInvalidationInfo *transInvalInfo = NULL;
+static InvalidationInfo *inplaceInvalInfo = NULL;
+
/* GUC storage */
int debug_discard_caches = 0;
@@ -543,9 +561,12 @@ ProcessInvalidationMessagesMulti(InvalidationMsgsGroup *group,
static void
RegisterCatcacheInvalidation(int cacheId,
uint32 hashValue,
- Oid dbId)
+ Oid dbId,
+ void *context)
{
- AddCatcacheInvalidationMessage(&transInvalInfo->CurrentCmdInvalidMsgs,
+ InvalidationInfo *info = (InvalidationInfo *) context;
+
+ AddCatcacheInvalidationMessage(&info->CurrentCmdInvalidMsgs,
cacheId, hashValue, dbId);
}
@@ -555,10 +576,9 @@ RegisterCatcacheInvalidation(int cacheId,
* Register an invalidation event for all catcache entries from a catalog.
*/
static void
-RegisterCatalogInvalidation(Oid dbId, Oid catId)
+RegisterCatalogInvalidation(InvalidationInfo *info, Oid dbId, Oid catId)
{
- AddCatalogInvalidationMessage(&transInvalInfo->CurrentCmdInvalidMsgs,
- dbId, catId);
+ AddCatalogInvalidationMessage(&info->CurrentCmdInvalidMsgs, dbId, catId);
}
/*
@@ -567,10 +587,9 @@ RegisterCatalogInvalidation(Oid dbId, Oid catId)
* As above, but register a relcache invalidation event.
*/
static void
-RegisterRelcacheInvalidation(Oid dbId, Oid relId)
+RegisterRelcacheInvalidation(InvalidationInfo *info, Oid dbId, Oid relId)
{
- AddRelcacheInvalidationMessage(&transInvalInfo->CurrentCmdInvalidMsgs,
- dbId, relId);
+ AddRelcacheInvalidationMessage(&info->CurrentCmdInvalidMsgs, dbId, relId);
/*
* Most of the time, relcache invalidation is associated with system
@@ -587,7 +606,7 @@ RegisterRelcacheInvalidation(Oid dbId, Oid relId)
* as well. Also zap when we are invalidating whole relcache.
*/
if (relId == InvalidOid || RelationIdIsInInitFile(relId))
- transInvalInfo->RelcacheInitFileInval = true;
+ info->RelcacheInitFileInval = true;
}
/*
@@ -597,24 +616,27 @@ RegisterRelcacheInvalidation(Oid dbId, Oid relId)
* Only needed for catalogs that don't have catcaches.
*/
static void
-RegisterSnapshotInvalidation(Oid dbId, Oid relId)
+RegisterSnapshotInvalidation(InvalidationInfo *info, Oid dbId, Oid relId)
{
- AddSnapshotInvalidationMessage(&transInvalInfo->CurrentCmdInvalidMsgs,
- dbId, relId);
+ AddSnapshotInvalidationMessage(&info->CurrentCmdInvalidMsgs, dbId, relId);
}
/*
* PrepareInvalidationState
* Initialize inval data for the current (sub)transaction.
*/
-static void
+static InvalidationInfo *
PrepareInvalidationState(void)
{
TransInvalidationInfo *myInfo;
+ Assert(IsTransactionState());
+ /* Can't queue transactional message while collecting inplace messages. */
+ Assert(inplaceInvalInfo == NULL);
+
if (transInvalInfo != NULL &&
transInvalInfo->my_level == GetCurrentTransactionNestLevel())
- return;
+ return (InvalidationInfo *) transInvalInfo;
myInfo = (TransInvalidationInfo *)
MemoryContextAllocZero(TopTransactionContext,
@@ -637,7 +659,7 @@ PrepareInvalidationState(void)
* counter. This is a convenient place to check for that, as well as
* being important to keep management of the message arrays simple.
*/
- if (NumMessagesInGroup(&transInvalInfo->CurrentCmdInvalidMsgs) != 0)
+ if (NumMessagesInGroup(&transInvalInfo->ii.CurrentCmdInvalidMsgs) != 0)
elog(ERROR, "cannot start a subtransaction when there are unprocessed inval messages");
/*
@@ -646,8 +668,8 @@ PrepareInvalidationState(void)
* to update them to follow whatever is already in the arrays.
*/
SetGroupToFollow(&myInfo->PriorCmdInvalidMsgs,
- &transInvalInfo->CurrentCmdInvalidMsgs);
- SetGroupToFollow(&myInfo->CurrentCmdInvalidMsgs,
+ &transInvalInfo->ii.CurrentCmdInvalidMsgs);
+ SetGroupToFollow(&myInfo->ii.CurrentCmdInvalidMsgs,
&myInfo->PriorCmdInvalidMsgs);
}
else
@@ -663,6 +685,41 @@ PrepareInvalidationState(void)
}
transInvalInfo = myInfo;
+ return (InvalidationInfo *) myInfo;
+}
+
+/*
+ * PrepareInplaceInvalidationState
+ * Initialize inval data for an inplace update.
+ *
+ * See previous function for more background.
+ */
+static InvalidationInfo *
+PrepareInplaceInvalidationState(void)
+{
+ InvalidationInfo *myInfo;
+
+ Assert(IsTransactionState());
+ /* limit of one inplace update under assembly */
+ Assert(inplaceInvalInfo == NULL);
+
+ /* gone after WAL insertion CritSection ends, so use current context */
+ myInfo = (InvalidationInfo *) palloc0(sizeof(InvalidationInfo));
+
+ /* Stash our messages past end of the transactional messages, if any. */
+ if (transInvalInfo != NULL)
+ SetGroupToFollow(&myInfo->CurrentCmdInvalidMsgs,
+ &transInvalInfo->ii.CurrentCmdInvalidMsgs);
+ else
+ {
+ InvalMessageArrays[CatCacheMsgs].msgs = NULL;
+ InvalMessageArrays[CatCacheMsgs].maxmsgs = 0;
+ InvalMessageArrays[RelCacheMsgs].msgs = NULL;
+ InvalMessageArrays[RelCacheMsgs].maxmsgs = 0;
+ }
+
+ inplaceInvalInfo = myInfo;
+ return myInfo;
}
/* ----------------------------------------------------------------
@@ -902,7 +959,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
* after we send the SI messages. However, we need not do anything unless
* we committed.
*/
- *RelcacheInitFileInval = transInvalInfo->RelcacheInitFileInval;
+ *RelcacheInitFileInval = transInvalInfo->ii.RelcacheInitFileInval;
/*
* Collect all the pending messages into a single contiguous array of
@@ -913,7 +970,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
* not new ones.
*/
nummsgs = NumMessagesInGroup(&transInvalInfo->PriorCmdInvalidMsgs) +
- NumMessagesInGroup(&transInvalInfo->CurrentCmdInvalidMsgs);
+ NumMessagesInGroup(&transInvalInfo->ii.CurrentCmdInvalidMsgs);
*msgs = msgarray = (SharedInvalidationMessage *)
MemoryContextAlloc(CurTransactionContext,
@@ -926,7 +983,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
msgs,
n * sizeof(SharedInvalidationMessage)),
nmsgs += n));
- ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+ ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
CatCacheMsgs,
(memcpy(msgarray + nmsgs,
msgs,
@@ -938,7 +995,51 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
msgs,
n * sizeof(SharedInvalidationMessage)),
nmsgs += n));
- ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+ ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
+ RelCacheMsgs,
+ (memcpy(msgarray + nmsgs,
+ msgs,
+ n * sizeof(SharedInvalidationMessage)),
+ nmsgs += n));
+ Assert(nmsgs == nummsgs);
+
+ return nmsgs;
+}
+
+/*
+ * inplaceGetInvalidationMessages() is called by the inplace update to collect
+ * invalidation messages to add to its WAL record. Like the previous
+ * function, we might still fail.
+ */
+int
+inplaceGetInvalidationMessages(SharedInvalidationMessage **msgs,
+ bool *RelcacheInitFileInval)
+{
+ SharedInvalidationMessage *msgarray;
+ int nummsgs;
+ int nmsgs;
+
+ /* Quick exit if we haven't done anything with invalidation messages. */
+ if (inplaceInvalInfo == NULL)
+ {
+ *RelcacheInitFileInval = false;
+ *msgs = NULL;
+ return 0;
+ }
+
+ *RelcacheInitFileInval = inplaceInvalInfo->RelcacheInitFileInval;
+ nummsgs = NumMessagesInGroup(&inplaceInvalInfo->CurrentCmdInvalidMsgs);
+ *msgs = msgarray = (SharedInvalidationMessage *)
+ palloc(nummsgs * sizeof(SharedInvalidationMessage));
+
+ nmsgs = 0;
+ ProcessMessageSubGroupMulti(&inplaceInvalInfo->CurrentCmdInvalidMsgs,
+ CatCacheMsgs,
+ (memcpy(msgarray + nmsgs,
+ msgs,
+ n * sizeof(SharedInvalidationMessage)),
+ nmsgs += n));
+ ProcessMessageSubGroupMulti(&inplaceInvalInfo->CurrentCmdInvalidMsgs,
RelCacheMsgs,
(memcpy(msgarray + nmsgs,
msgs,
@@ -1024,7 +1125,9 @@ ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
void
AtEOXact_Inval(bool isCommit)
{
- /* Quick exit if no messages */
+ inplaceInvalInfo = NULL;
+
+ /* Quick exit if no transactional messages */
if (transInvalInfo == NULL)
return;
@@ -1038,16 +1141,16 @@ AtEOXact_Inval(bool isCommit)
* after we send the SI messages. However, we need not do anything
* unless we committed.
*/
- if (transInvalInfo->RelcacheInitFileInval)
+ if (transInvalInfo->ii.RelcacheInitFileInval)
RelationCacheInitFilePreInvalidate();
AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
- &transInvalInfo->CurrentCmdInvalidMsgs);
+ &transInvalInfo->ii.CurrentCmdInvalidMsgs);
ProcessInvalidationMessagesMulti(&transInvalInfo->PriorCmdInvalidMsgs,
SendSharedInvalidMessages);
- if (transInvalInfo->RelcacheInitFileInval)
+ if (transInvalInfo->ii.RelcacheInitFileInval)
RelationCacheInitFilePostInvalidate();
}
else
@@ -1061,6 +1164,44 @@ AtEOXact_Inval(bool isCommit)
}
/*
+ * PreInplace_Inval
+ * Process queued-up invalidation before inplace update critical section.
+ *
+ * Tasks belong here if they are safe even if the inplace update does not
+ * complete. Currently, this just unlinks a cache file, which can fail. The
+ * sum of this and AtInplace_Inval() mirrors AtEOXact_Inval(isCommit=true).
+ */
+void
+PreInplace_Inval(void)
+{
+ Assert(CritSectionCount == 0);
+
+ if (inplaceInvalInfo && inplaceInvalInfo->RelcacheInitFileInval)
+ RelationCacheInitFilePreInvalidate();
+}
+
+/*
+ * AtInplace_Inval
+ * Process queued-up invalidations after inplace update buffer mutation.
+ */
+void
+AtInplace_Inval(void)
+{
+ Assert(CritSectionCount > 0);
+
+ if (inplaceInvalInfo == NULL)
+ return;
+
+ ProcessInvalidationMessagesMulti(&inplaceInvalInfo->CurrentCmdInvalidMsgs,
+ SendSharedInvalidMessages);
+
+ if (inplaceInvalInfo->RelcacheInitFileInval)
+ RelationCacheInitFilePostInvalidate();
+
+ inplaceInvalInfo = NULL;
+}
+
+/*
* AtEOSubXact_Inval
* Process queued-up invalidation messages at end of subtransaction.
*
@@ -1082,9 +1223,20 @@ void
AtEOSubXact_Inval(bool isCommit)
{
int my_level;
- TransInvalidationInfo *myInfo = transInvalInfo;
+ TransInvalidationInfo *myInfo;
+
+ /*
+ * Successful inplace update must clear this, but we clear it on abort.
+ * Inplace updates allocate this in CurrentMemoryContext, which has
+ * lifespan <= subtransaction lifespan. Hence, don't free it explicitly.
+ */
+ if (isCommit)
+ Assert(inplaceInvalInfo == NULL);
+ else
+ inplaceInvalInfo = NULL;
- /* Quick exit if no messages. */
+ /* Quick exit if no transactional messages. */
+ myInfo = transInvalInfo;
if (myInfo == NULL)
return;
@@ -1125,12 +1277,12 @@ AtEOSubXact_Inval(bool isCommit)
&myInfo->PriorCmdInvalidMsgs);
/* Must readjust parent's CurrentCmdInvalidMsgs indexes now */
- SetGroupToFollow(&myInfo->parent->CurrentCmdInvalidMsgs,
+ SetGroupToFollow(&myInfo->parent->ii.CurrentCmdInvalidMsgs,
&myInfo->parent->PriorCmdInvalidMsgs);
/* Pending relcache inval becomes parent's problem too */
- if (myInfo->RelcacheInitFileInval)
- myInfo->parent->RelcacheInitFileInval = true;
+ if (myInfo->ii.RelcacheInitFileInval)
+ myInfo->parent->ii.RelcacheInitFileInval = true;
/* Pop the transaction state stack */
transInvalInfo = myInfo->parent;
@@ -1177,7 +1329,7 @@ CommandEndInvalidationMessages(void)
if (transInvalInfo == NULL)
return;
- ProcessInvalidationMessages(&transInvalInfo->CurrentCmdInvalidMsgs,
+ ProcessInvalidationMessages(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
LocalExecuteInvalidationMessage);
/* WAL Log per-command invalidation messages for wal_level=logical */
@@ -1185,26 +1337,21 @@ CommandEndInvalidationMessages(void)
LogLogicalInvalidations();
AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
- &transInvalInfo->CurrentCmdInvalidMsgs);
+ &transInvalInfo->ii.CurrentCmdInvalidMsgs);
}
/*
- * 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.
+ * CacheInvalidateHeapTupleCommon
+ * Common logic for end-of-command and inplace variants.
*/
-void
-CacheInvalidateHeapTuple(Relation relation,
- HeapTuple tuple,
- HeapTuple newtuple)
+static void
+CacheInvalidateHeapTupleCommon(Relation relation,
+ HeapTuple tuple,
+ HeapTuple newtuple,
+ InvalidationInfo *(*prepare_callback) (void))
{
+ InvalidationInfo *info;
Oid tupleRelId;
Oid databaseId;
Oid relationId;
@@ -1228,11 +1375,8 @@ CacheInvalidateHeapTuple(Relation relation,
if (IsToastRelation(relation))
return;
- /*
- * If we're not prepared to queue invalidation messages for this
- * subtransaction level, get ready now.
- */
- PrepareInvalidationState();
+ /* Allocate any required resources. */
+ info = prepare_callback();
/*
* First let the catcache do its thing
@@ -1241,11 +1385,12 @@ CacheInvalidateHeapTuple(Relation relation,
if (RelationInvalidatesSnapshotsOnly(tupleRelId))
{
databaseId = IsSharedRelation(tupleRelId) ? InvalidOid : MyDatabaseId;
- RegisterSnapshotInvalidation(databaseId, tupleRelId);
+ RegisterSnapshotInvalidation(info, databaseId, tupleRelId);
}
else
PrepareToInvalidateCacheTuple(relation, tuple, newtuple,
- RegisterCatcacheInvalidation);
+ RegisterCatcacheInvalidation,
+ (void *) info);
/*
* Now, is this tuple one of the primary definers of a relcache entry? See
@@ -1318,7 +1463,44 @@ CacheInvalidateHeapTuple(Relation relation,
/*
* Yes. We need to register a relcache invalidation event.
*/
- RegisterRelcacheInvalidation(databaseId, relationId);
+ RegisterRelcacheInvalidation(info, databaseId, relationId);
+}
+
+/*
+ * CacheInvalidateHeapTuple
+ * Register the given tuple for invalidation at end of command
+ * (ie, current command is creating or outdating this tuple) and end of
+ * transaction. 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,
+ HeapTuple newtuple)
+{
+ CacheInvalidateHeapTupleCommon(relation, tuple, newtuple,
+ PrepareInvalidationState);
+}
+
+/*
+ * CacheInvalidateHeapTupleInplace
+ * Register the given tuple for nontransactional invalidation pertaining
+ * to an inplace update. Also, detect whether a relcache invalidation is
+ * implied.
+ *
+ * Like CacheInvalidateHeapTuple(), but for inplace updates.
+ */
+void
+CacheInvalidateHeapTupleInplace(Relation relation,
+ HeapTuple tuple,
+ HeapTuple newtuple)
+{
+ CacheInvalidateHeapTupleCommon(relation, tuple, newtuple,
+ PrepareInplaceInvalidationState);
}
/*
@@ -1337,14 +1519,13 @@ CacheInvalidateCatalog(Oid catalogId)
{
Oid databaseId;
- PrepareInvalidationState();
-
if (IsSharedRelation(catalogId))
databaseId = InvalidOid;
else
databaseId = MyDatabaseId;
- RegisterCatalogInvalidation(databaseId, catalogId);
+ RegisterCatalogInvalidation(PrepareInvalidationState(),
+ databaseId, catalogId);
}
/*
@@ -1362,15 +1543,14 @@ CacheInvalidateRelcache(Relation relation)
Oid databaseId;
Oid relationId;
- PrepareInvalidationState();
-
relationId = RelationGetRelid(relation);
if (relation->rd_rel->relisshared)
databaseId = InvalidOid;
else
databaseId = MyDatabaseId;
- RegisterRelcacheInvalidation(databaseId, relationId);
+ RegisterRelcacheInvalidation(PrepareInvalidationState(),
+ databaseId, relationId);
}
/*
@@ -1383,9 +1563,8 @@ CacheInvalidateRelcache(Relation relation)
void
CacheInvalidateRelcacheAll(void)
{
- PrepareInvalidationState();
-
- RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+ RegisterRelcacheInvalidation(PrepareInvalidationState(),
+ InvalidOid, InvalidOid);
}
/*
@@ -1399,14 +1578,13 @@ CacheInvalidateRelcacheByTuple(HeapTuple classTuple)
Oid databaseId;
Oid relationId;
- PrepareInvalidationState();
-
relationId = classtup->oid;
if (classtup->relisshared)
databaseId = InvalidOid;
else
databaseId = MyDatabaseId;
- RegisterRelcacheInvalidation(databaseId, relationId);
+ RegisterRelcacheInvalidation(PrepareInvalidationState(),
+ databaseId, relationId);
}
/*
@@ -1420,8 +1598,6 @@ CacheInvalidateRelcacheByRelid(Oid relid)
{
HeapTuple tup;
- PrepareInvalidationState();
-
tup = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
if (!HeapTupleIsValid(tup))
elog(ERROR, "cache lookup failed for relation %u", relid);
@@ -1611,7 +1787,7 @@ LogLogicalInvalidations(void)
if (transInvalInfo == NULL)
return;
- group = &transInvalInfo->CurrentCmdInvalidMsgs;
+ group = &transInvalInfo->ii.CurrentCmdInvalidMsgs;
nmsgs = NumMessagesInGroup(group);
if (nmsgs > 0)
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index 50c9440f792..f41b1c221a1 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -351,8 +351,7 @@ SearchSysCacheLocked1(int cacheId,
/*
* If an inplace update just finished, ensure we process the syscache
- * inval. XXX this is insufficient: the inplace updater may not yet
- * have reached AtEOXact_Inval(). See test at inplace-inval.spec.
+ * inval.
*
* If a heap_update() call just released its LOCKTAG_TUPLE, we'll
* probably find the old tuple and reach "tuple concurrently updated".