Inval reliability, especially for inplace updates

Started by Noah Mischover 1 year ago35 messages
#1Noah Misch
noah@leadboat.com

/messages/by-id/20240512232923.aa.nmisch@google.com wrote:

Separable, nontrivial things not fixed in the attached patch stack:

- Inplace update uses transactional CacheInvalidateHeapTuple(). ROLLBACK of
CREATE INDEX wrongly discards the inval, leading to the relhasindex=t loss
still seen in inplace-inval.spec. CacheInvalidateRelmap() does this right.

I plan to fix that like CacheInvalidateRelmap(): send the inval immediately,
inside the critical section. Send it in heap_xlog_inplace(), too. The
interesting decision is how to handle RelationCacheInitFilePreInvalidate(),
which has an unlink_initfile() that can fail with e.g. EIO. Options:

1. Unlink during critical section, and accept that EIO becomes PANIC. Replay
may reach the same EIO, and the system won't reopen to connections until
the storage starts cooperating.a Interaction with checkpoints is not ideal.
If we checkpoint and then crash between inplace XLogInsert() and inval,
we'd be relying on StartupXLOG() -> RelationCacheInitFileRemove(). That
uses elevel==LOG, so replay would neglect to PANIC on EIO.

2. Unlink before critical section, so normal xact abort suffices. This would
hold RelCacheInitLock and a buffer content lock at the same time. In
RecordTransactionCommit(), it would hold RelCacheInitLock and e.g. slru
locks at the same time.

The PANIC risk of (1) seems similar to the risk of PANIC at
RecordTransactionCommit() -> XLogFlush(), which hasn't been a problem. The
checkpoint-related risk bothers me more, and (1) generally makes it harder to
reason about checkpoint interactions. The lock order risk of (2) feels
tolerable. I'm leaning toward (2), but that might change. Other preferences?

Another decision is what to do about LogLogicalInvalidations(). Currently,
inplace update invalidations do reach WAL via LogLogicalInvalidations() at the
next CCI. Options:

a. Within logical decoding, cease processing invalidations for inplace
updates. Inplace updates don't affect storage or interpretation of table
rows, so they don't affect logicalrep_write_tuple() outcomes. If they did,
invalidations wouldn't make it work. Decoding has no way to retrieve a
snapshot-appropriate version of the inplace-updated value.

b. Make heap_decode() of XLOG_HEAP_INPLACE recreate the invalidation. This
would be, essentially, cheap insurance against invalidations having a
benefit I missed in (a).

I plan to pick (a).

- AtEOXact_Inval(true) is outside the RecordTransactionCommit() critical
section, but it is critical. We must not commit transactional DDL without
other backends receiving an inval. (When the inplace inval becomes
nontransactional, it will face the same threat.)

This faces the same RelationCacheInitFilePreInvalidate() decision, and I think
the conclusion should be the same as for inplace update.

Thanks,
nm

#2Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#1)
4 attachment(s)
Re: Inval reliability, especially for inplace updates

On Wed, May 22, 2024 at 05:05:48PM -0700, Noah Misch wrote:

/messages/by-id/20240512232923.aa.nmisch@google.com wrote:

Separable, nontrivial things not fixed in the attached patch stack:

- Inplace update uses transactional CacheInvalidateHeapTuple(). ROLLBACK of
CREATE INDEX wrongly discards the inval, leading to the relhasindex=t loss
still seen in inplace-inval.spec. CacheInvalidateRelmap() does this right.

I plan to fix that like CacheInvalidateRelmap(): send the inval immediately,
inside the critical section. Send it in heap_xlog_inplace(), too.

a. Within logical decoding, cease processing invalidations for inplace

I'm attaching the implementation. This applies atop the v3 patch stack from
/messages/by-id/20240614003549.c2.nmisch@google.com, but the threads are
mostly orthogonal and intended for independent review. Translating a tuple
into inval messages uses more infrastructure than relmapper, which needs just
a database ID. Hence, this ended up more like a miniature of inval.c's
participation in the transaction commit sequence.

I waffled on whether to back-patch inplace150-inval-durability-atcommit. The
consequences of that bug are plenty bad, but reaching them requires an error
between TransactionIdCommitTree() and AtEOXact_Inval(). I've not heard
reports of that, and I don't have a recipe for making it happen on demand.
For now, I'm leaning toward back-patch. The main risk would be me overlooking
an LWLock deadlock scenario reachable from the new, earlier RelCacheInitLock
timing. Alternatives for RelCacheInitLock:

- RelCacheInitLock before PreCommit_Notify(), because notify concurrency
matters more than init file concurrency. I chose this.
- RelCacheInitLock after PreCommit_Notify(), because PreCommit_Notify() uses a
heavyweight lock, giving it less risk of undetected deadlock.
- Replace RelCacheInitLock with a heavyweight lock, and keep it before
PreCommit_Notify().
- Fold PreCommit_Inval() back into AtCommit_Inval(), accepting that EIO in
unlink_initfile() will PANIC.

Opinions on that?

The patch changes xl_heap_inplace of XLOG_HEAP_INPLACE. For back branches, we
could choose between:

- Same change, no WAL version bump. Standby must update before primary. This
is best long-term, but the transition is more disruptive. I'm leaning
toward this one, but the second option isn't bad:

- heap_xlog_inplace() could set the shared-inval-queue overflow signal on
every backend. This is more wasteful, but inplace updates might be rare
enough (~once per VACUUM) to make it tolerable.

- Use LogStandbyInvalidations() just after XLOG_HEAP_INPLACE. This isn't
correct if one ends recovery between the two records, but you'd need to be
unlucky to notice. Noticing would need a procedure like the following. A
hot standby backend populates a relcache entry, then does DDL on the rel
after recovery ends.

Future cleanup work could eliminate LogStandbyInvalidations() and the case of
!markXidCommitted && nmsgs != 0. Currently, the src/test/regress suite still
reaches that case:

- AlterDomainDropConstraint() queues an inval even if !found; it can stop
that.

- ON COMMIT DELETE ROWS nontransactionally rebuilds an index, which sends a
relcache inval. The point of that inval is, I think, to force access
methods like btree and hash to reload the metapage copy that they store in
rd_amcache. Since no assigned XID implies no changes to the temp index, the
no-XID case could simply skip the index rebuild. (temp.sql reaches this
with a read-only transaction that selects from an ON COMMIT DELETE ROWS
table. Realistic usage will tend not to do that.) ON COMMIT DELETE ROWS
has another preexisting problem for indexes, mentioned in a code comment.

Thanks,
nm

Attachments:

inplace160-inval-durability-inplace-v1.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Core code no longer needs XLOG_INVALIDATIONS, but this leaves removing
    it for future work.  Extensions could be relying on that mechanism, so
    that removal would not be back-patch material.  Back-patch to v12 (all
    supported versions).
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 797bddf..d7e417f 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6305,6 +6305,9 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 	Relation	relation = scan->heap_rel;
 	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;
@@ -6312,6 +6315,29 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 	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();
 
@@ -6341,9 +6367,16 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 		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);
@@ -6355,22 +6388,23 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
-	END_CRIT_SECTION();
-
 	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);
 	systable_endscan(scan);
 
-	/*
-	 * 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.
-	 */
-	if (!IsBootstrapProcessingMode())
-		CacheInvalidateHeapTuple(relation, tuple, NULL);
+	AcceptInvalidationMessages();	/* local processing of just-sent inval */
 }
 
 /*
@@ -10268,6 +10302,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 5f5673e..f31cc3a 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/transam/xact.c b/src/backend/access/transam/xact.c
index 6f0f80c..bffbcb4 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1358,13 +1358,14 @@ RecordTransactionCommit(void)
 
 		/*
 		 * Transactions without an assigned xid can contain invalidation
-		 * messages.  Inplace updates do so, and standbys 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).
+		 * messages.  While inplace updates formerly did so, they now send
+		 * immediate invalidations.  Extensions might still do so, 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,
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index b4b68b1..ba576c6 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2884,19 +2884,21 @@ index_update_stats(Relation rel,
 	}
 
 	/*
-	 * If anything changed, write out the tuple
+	 * If anything changed, write out the tuple and immediate invals
 	 */
 	if (dirty)
-	{
 		heap_inplace_update_finish(state, tuple);
-		/* the above sends a cache inval message */
-	}
 	else
-	{
 		heap_inplace_update_cancel(state);
-		/* no need to change tuple, but force relcache inval anyway */
-		CacheInvalidateRelcacheByTuple(tuple);
-	}
+
+	/*
+	 * Queue a transactional relcache inval.  CREATE INDEX needs an immediate
+	 * inval for the relhasindex change, but it also needs a transactional
+	 * inval for when the new index's rows become visible.  Other CREATE INDEX
+	 * and REINDEX code happens to also queue a transactional inval, but keep
+	 * this in case rare callers rely on this part of our API contract.
+	 */
+	CacheInvalidateRelcacheByTuple(tuple);
 
 	heap_freetuple(tuple);
 
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 36d82bd..5d4173a 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.
 				 */
 				heap_inplace_update_finish(state, tuple);
 			}
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 8ec5adf..b2cf14e 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -508,23 +508,18 @@ 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.
+			 * Older WAL may contain commit-time invals for inplace updates.
+			 * 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 98aa527..b40e38c 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2287,7 +2287,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;
@@ -2328,7 +2329,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)
 		{
@@ -2337,7 +2338,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 0772314..00566fb 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;
 
-	/* Events emitted by previous commands of this (sub)transaction */
-	InvalidationMsgsGroup PriorCmdInvalidMsgs;
-
 	/* 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;
+
+	/* 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,26 +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,
@@ -639,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");
 
 		/*
@@ -648,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
@@ -665,6 +685,40 @@ 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;
+
+	/* 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;
 }
 
 /* ----------------------------------------------------------------
@@ -904,7 +958,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
@@ -915,7 +969,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 	 * not new ones.
 	 */
 	nummsgs = NumMessagesInGroup(&transInvalInfo->PriorCmdInvalidMsgs) +
-		NumMessagesInGroup(&transInvalInfo->CurrentCmdInvalidMsgs);
+		NumMessagesInGroup(&transInvalInfo->ii.CurrentCmdInvalidMsgs);
 
 	*msgs = msgarray = (SharedInvalidationMessage *)
 		MemoryContextAlloc(CurTransactionContext,
@@ -928,7 +982,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 										msgs,
 										n * sizeof(SharedInvalidationMessage)),
 								 nmsgs += n));
-	ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 								CatCacheMsgs,
 								(memcpy(msgarray + nmsgs,
 										msgs,
@@ -940,7 +994,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,
@@ -1015,6 +1113,7 @@ ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 void
 PreCommit_Inval(void)
 {
+	Assert(inplaceInvalInfo == NULL);
 	/* This is a separate function just to run before the critical section. */
 	Assert(CritSectionCount == 0);
 
@@ -1028,7 +1127,20 @@ PreCommit_Inval(void)
 	 * Relcache init file invalidation requires processing both before and
 	 * after we send the SI messages.
 	 */
-	if (transInvalInfo->RelcacheInitFileInval)
+	if (transInvalInfo->ii.RelcacheInitFileInval)
+		RelationCacheInitFilePreInvalidate();
+}
+
+/*
+ * PreInplace_Inval
+ *		Like previous function, but for inplace updates.
+ */
+void
+PreInplace_Inval(void)
+{
+	Assert(CritSectionCount == 0);
+
+	if (inplaceInvalInfo && inplaceInvalInfo->RelcacheInitFileInval)
 		RelationCacheInitFilePreInvalidate();
 }
 
@@ -1062,12 +1174,12 @@ AtCommit_Inval(void)
 		return;
 
 	AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
-							   &transInvalInfo->CurrentCmdInvalidMsgs);
+							   &transInvalInfo->ii.CurrentCmdInvalidMsgs);
 
 	ProcessInvalidationMessagesMulti(&transInvalInfo->PriorCmdInvalidMsgs,
 									 SendSharedInvalidMessages);
 
-	if (transInvalInfo->RelcacheInitFileInval)
+	if (transInvalInfo->ii.RelcacheInitFileInval)
 		RelationCacheInitFilePostInvalidate();
 
 	/* Need not free anything explicitly */
@@ -1075,6 +1187,27 @@ AtCommit_Inval(void)
 }
 
 /*
+ * AtInplace_Inval
+ *		Like previous function, but for inplace updates.
+ */
+void
+AtInplace_Inval(void)
+{
+	Assert(CritSectionCount > 0);
+
+	if (inplaceInvalInfo == NULL)
+		return;
+
+	ProcessInvalidationMessagesMulti(&inplaceInvalInfo->CurrentCmdInvalidMsgs,
+									 SendSharedInvalidMessages);
+
+	if (inplaceInvalInfo->RelcacheInitFileInval)
+		RelationCacheInitFilePostInvalidate();
+
+	inplaceInvalInfo = NULL;
+}
+
+/*
  * AtAbort_Inval
  *		Process queued-up invalidation messages at abort of main transaction.
  *
@@ -1097,6 +1230,7 @@ AtAbort_Inval(void)
 
 	/* Need not free anything explicitly */
 	transInvalInfo = NULL;
+	inplaceInvalInfo = NULL;
 }
 
 /*
@@ -1164,18 +1298,21 @@ 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;
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Successful inplace update must clear this. */
+		Assert(inplaceInvalInfo == NULL);
 	}
 	else
 	{
@@ -1187,6 +1324,9 @@ AtEOSubXact_Inval(bool isCommit)
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Reset from aborted inplace update. */
+		inplaceInvalInfo = NULL;
 	}
 }
 
@@ -1216,7 +1356,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 */
@@ -1224,26 +1364,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;
@@ -1267,11 +1402,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
@@ -1280,11 +1412,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
@@ -1357,7 +1490,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);
 }
 
 /*
@@ -1376,14 +1546,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1401,15 +1570,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);
 }
 
 /*
@@ -1422,9 +1590,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1438,14 +1605,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);
 }
 
 /*
@@ -1459,8 +1625,6 @@ CacheInvalidateRelcacheByRelid(Oid relid)
 {
 	HeapTuple	tup;
 
-	PrepareInvalidationState();
-
 	tup = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
 	if (!HeapTupleIsValid(tup))
 		elog(ERROR, "cache lookup failed for relation %u", relid);
@@ -1650,7 +1814,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 8147cf6..f41b1c2 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 AtCommit_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".
diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 42736f3..4591e9a 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -20,6 +20,7 @@
 #include "storage/buf.h"
 #include "storage/bufpage.h"
 #include "storage/relfilelocator.h"
+#include "storage/sinval.h"
 #include "utils/relcache.h"
 
 
@@ -425,9 +426,14 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
+	Oid			dbId;			/* MyDatabaseId */
+	Oid			tsId;			/* MyDatabaseTableSpace */
+	bool		relcacheInitFileInval;	/* invalidate relcache init files */
+	int			nmsgs;			/* number of shared inval msgs */
+	SharedInvalidationMessage msgs[FLEXIBLE_ARRAY_MEMBER];
 } xl_heap_inplace;
 
-#define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
+#define MinSizeOfHeapInplace	(offsetof(xl_heap_inplace, nmsgs) + sizeof(int))
 
 /*
  * This is what we need to know about setting a visibility map bit
diff --git a/src/include/storage/sinval.h b/src/include/storage/sinval.h
index 8f5744b..c812237 100644
--- a/src/include/storage/sinval.h
+++ b/src/include/storage/sinval.h
@@ -144,6 +144,8 @@ extern void ProcessCatchupInterrupt(void);
 
 extern int	xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 												 bool *RelcacheInitFileInval);
+extern int	inplaceGetInvalidationMessages(SharedInvalidationMessage **msgs,
+										   bool *RelcacheInitFileInval);
 extern void ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 												 int nmsgs, bool RelcacheInitFileInval,
 												 Oid dbid, Oid tsid);
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 94d8b44..64c0265 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -30,6 +30,9 @@ extern void PreCommit_Inval(void);
 extern void AtCommit_Inval(void);
 extern void AtAbort_Inval(void);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -39,6 +42,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index 67b34ad..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -14,7 +14,7 @@ step read1:
 
 relhasindex
 -----------
-f          
+t          
 (1 row)
 
 
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index d8e1c98..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -32,7 +32,7 @@ permutation
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
-	read1	# observe damage XXX is an extant bug
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 5696604..b2b565f 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1252,6 +1252,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
inplace130-AtEOXact_RelationCache-comments-v1.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Fix comments in and referring to AtEOXact_RelationCache().
    
    The first point has prompted this change.  We can fix a bug by calling
    AtEOXact_Inval(true) earlier, in the COMMIT critical section.  Remove
    comment text that would have required AtEOXact_RelationCache() to move
    with it.  Full list of defects fixed:
    
    - Commit 8de3e410faa06ab20ec1aa6d0abb0a2c040261ba (2014-02) made
      relcache.c defer rebuilds if !IsTransactionState().  That removed the
      functional implications of swapping the order of
      AtEOXact_RelationCache() and AtEOXact_Inval().  Even without that
      change, we'd have other layers of defense.  At COMMIT:
    
      - Code that opened rels already closed them.
      - AtEOXact_RelationCache() essentially sets some non-pointer fields
        and calls RelationDestroyRelation() to pfree() rels for which we're
        committing both a CREATE and a DROP.  None of that needs a
        transaction.
      - AtEOXact_Inval() doesn't locally-execute messages.  The next
        AcceptInvalidationMessages() does that.
    
      At ABORT:
    
      - resowner.c already closed all rels.
      - AtEOXact_RelationCache() essentially sets some non-pointer fields
        and calls RelationDestroyRelation() to pfree() rels for which we're
        aborting a CREATE.
    
    - Commit f884dca4959f64bd47e78102d1dadd2c77d49201 (2019-05) removed
      "forced index lists".
    
    - The header comment listed just a subset of the function's activities.
      Commit fdd965d074d46765c295223b119ca437dbcac973 (back-patched to v9.6)
      added in_progress_list cleanup.  Commit
      e5550d5fec66aa74caad1f79b79826ec64898688 (2014-04) added
      EOXactTupleDescArray cleanup.
    
    - When commit d5b31cc32b0762fa8920a9c1f70af2f82fb0aaa5 (2013-01)
      introduced the eoxact_list mechanism, the function ceased crashing
      when called before initialization.
    
    - While "current transaction created any relations" is the activity with
      the most code here, that appeared at the end of a paragraph that
      started with pre-8.1 behavior.
    
    Back-patch to v12 (all supported versions), the plan for calling
    AtEOXact_Inval(true) earlier.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 9bda1aa..ee389fc 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -2369,11 +2369,10 @@ CommitTransaction(void)
 	AtEOXact_RelationCache(true);
 
 	/*
-	 * Make catalog changes visible to all backends.  This has to happen after
-	 * relcache references are dropped (see comments for
-	 * AtEOXact_RelationCache), but before locks are released (if anyone is
-	 * waiting for lock on a relation we've modified, we want them to know
-	 * about the catalog change before they start using the relation).
+	 * Make catalog changes visible to all backends.  This has to happen
+	 * before locks are released (if anyone is waiting for lock on a relation
+	 * we've modified, we want them to know about the catalog change before
+	 * they start using the relation).
 	 */
 	AtEOXact_Inval(true);
 
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index cbf9cf2..ed4a8eb 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -3218,17 +3218,6 @@ AssertPendingSyncs_RelationCache(void)
  * AtEOXact_RelationCache
  *
  *	Clean up the relcache at main-transaction commit or abort.
- *
- * Note: this must be called *before* processing invalidation messages.
- * In the case of abort, we don't want to try to rebuild any invalidated
- * cache entries (since we can't safely do database accesses).  Therefore
- * we must reset refcnts before handling pending invalidations.
- *
- * As of PostgreSQL 8.1, relcache refcnts should get released by the
- * ResourceOwner mechanism.  This routine just does a debugging
- * cross-check that no pins remain.  However, we also need to do special
- * cleanup when the current transaction created any relations or made use
- * of forced index lists.
  */
 void
 AtEOXact_RelationCache(bool isCommit)
@@ -3245,8 +3234,9 @@ AtEOXact_RelationCache(bool isCommit)
 	in_progress_list_len = 0;
 
 	/*
-	 * Unless the eoxact_list[] overflowed, we only need to examine the rels
-	 * listed in it.  Otherwise fall back on a hash_seq_search scan.
+	 * Cleanup rels created in the current transaction.  Unless the
+	 * eoxact_list[] overflowed, we only need to examine the rels listed in
+	 * it.  Otherwise fall back on a hash_seq_search scan.
 	 *
 	 * For simplicity, eoxact_list[] entries are not deleted till end of
 	 * top-level transaction, even though we could remove them at
@@ -3307,7 +3297,8 @@ AtEOXact_cleanup(Relation relation, bool isCommit)
 
 	/*
 	 * The relcache entry's ref count should be back to its normal
-	 * not-in-a-transaction state: 0 unless it's nailed in cache.
+	 * not-in-a-transaction state: 0 unless it's nailed in cache.  The
+	 * ResourceOwner mechanism handles that.
 	 *
 	 * In bootstrap mode, this is NOT true, so don't check it --- the
 	 * bootstrap code expects relations to stay open across start/commit
@@ -3982,10 +3973,7 @@ RelationAssumeNewRelfilelocator(Relation relation)
  *		This initializes the relation descriptor cache.  At the time
  *		that this is invoked, we can't do database access yet (mainly
  *		because the transaction subsystem is not up); all we are doing
- *		is making an empty cache hashtable.  This must be done before
- *		starting the initialization transaction, because otherwise
- *		AtEOXact_RelationCache would crash if that transaction aborts
- *		before we can get the relcache set up.
+ *		is making an empty cache hashtable.
  */
 
 #define INITRELCACHESIZE		400
inplace140-heapam_xlog-comment-v1.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Remove comment about xl_heap_inplace "AT END OF STRUCT".
    
    Commit 2c03216d831160bedd72d45f712601b6f7d03f1c moved the tuple data
    from there to the buffer-0 data.  Back-patch to v12 (all supported
    versions), the plan for the next change to this struct.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 22a1747..42736f3 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -425,7 +425,6 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
-	/* TUPLE DATA FOLLOWS AT END OF STRUCT */
 } xl_heap_inplace;
 
 #define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
inplace150-inval-durability-atcommit-v1.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Move SendSharedInvalidMessages() into the COMMIT critical section.
    
    If a backend commits transactional DDL without making associated invals
    global, other backends may, for example, perform I/O on the wrong
    relfilenode.  This change doesn't raise or lower protection for inplace
    updates or ON COMMIT DELETE ROWS.  Back-patch to v12 (all supported
    versions).  PGXN has no AtEOXact_Inval() references, and there's no
    known use case for calling it from an extension.  Hence, drop it from
    all branches.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index bf451d4..a142868 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1630,7 +1630,7 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	 *
 	 * Relcache init file invalidation requires processing both before and
 	 * after we send the SI messages, only when committing.  See
-	 * AtEOXact_Inval().
+	 * PreCommit_Inval() and AtCommit_Inval().
 	 */
 	if (isCommit)
 	{
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index ee389fc..6f0f80c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1358,14 +1358,25 @@ 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.  Inplace updates do so, and standbys 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.  Standbys don't need those invals, but
+		 * we've not done the work to withhold them.  ON COMMIT DELETE ROWS
+		 * can't cope with an error in index_build(), which is more likely
+		 * than an error here.  ON COMMIT DELETE ROWS would need a deeper
+		 * redesign to become safe against arbitrary errors.  Meanwhile, the
+		 * damage from this is limited to temp tables of one session.
 		 */
 		if (nmsgs != 0)
 		{
@@ -1373,6 +1384,9 @@ RecordTransactionCommit(void)
 									RelcacheInitFileInval);
 			wrote_xlog = true;	/* not strictly necessary */
 		}
+		START_CRIT_SECTION();
+		AtCommit_Inval();
+		END_CRIT_SECTION();
 
 		/*
 		 * If we didn't create XLOG entries, we're done here; otherwise we
@@ -1510,13 +1524,15 @@ RecordTransactionCommit(void)
 			TransactionIdAsyncCommitTree(xid, nchildren, children, XactLastRecEnd);
 	}
 
-	/*
-	 * If we entered a commit critical section, leave it now, and let
-	 * checkpoints proceed.
-	 */
+	/* If we entered a commit critical section, finish it. */
 	if (markXidCommitted)
 	{
+		/* Let checkpoints proceed. */
 		MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+
+		/* Make catalog changes visible to all backends. */
+		AtCommit_Inval();
+
 		END_CRIT_SECTION();
 	}
 
@@ -2271,6 +2287,15 @@ CommitTransaction(void)
 	AtEOXact_LargeObject(true);
 
 	/*
+	 * Unlink relcache init files as needed.  If unlinking, acquire
+	 * RelCacheInitLock until after commit.  Do this before
+	 * PreCommit_Notify(), since notify concurrency is more important than
+	 * DDL/connection establishment concurrency.  Due to that LWLock, don't
+	 * move this before I/O-heavy smgrDoPendingSyncs().
+	 */
+	PreCommit_Inval();
+
+	/*
 	 * Insert notifications sent by NOTIFY commands into the queue.  This
 	 * should be late in the pre-commit sequence to minimize time spent
 	 * holding the notify-insertion lock.  However, this could result in
@@ -2368,14 +2393,6 @@ CommitTransaction(void)
 	/* Clean up the relation cache */
 	AtEOXact_RelationCache(true);
 
-	/*
-	 * Make catalog changes visible to all backends.  This has to happen
-	 * before locks are released (if anyone is waiting for lock on a relation
-	 * we've modified, we want them to know about the catalog change before
-	 * they start using the relation).
-	 */
-	AtEOXact_Inval(true);
-
 	AtEOXact_MultiXact();
 
 	ResourceOwnerRelease(TopTransactionResourceOwner,
@@ -2906,7 +2923,7 @@ AbortTransaction(void)
 							 false, true);
 		AtEOXact_Buffers(false);
 		AtEOXact_RelationCache(false);
-		AtEOXact_Inval(false);
+		AtAbort_Inval();
 		AtEOXact_MultiXact();
 		ResourceOwnerRelease(TopTransactionResourceOwner,
 							 RESOURCE_RELEASE_LOCKS,
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index 603aa41..0772314 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -612,6 +612,8 @@ PrepareInvalidationState(void)
 {
 	TransInvalidationInfo *myInfo;
 
+	Assert(IsTransactionState());
+
 	if (transInvalInfo != NULL &&
 		transInvalInfo->my_level == GetCurrentTransactionNestLevel())
 		return;
@@ -862,14 +864,14 @@ AcceptInvalidationMessages(void)
 void
 PostPrepare_Inval(void)
 {
-	AtEOXact_Inval(false);
+	AtAbort_Inval();
 }
 
 /*
  * xactGetCommittedInvalidationMessages() is called by
  * RecordTransactionCommit() to collect invalidation messages to add to the
  * commit record. This applies only to commit message types, never to
- * abort records. Must always run before AtEOXact_Inval(), since that
+ * abort records. Must always run before AtCommit_Inval(), since that
  * removes the data we need to see.
  *
  * Remember that this runs before we have officially committed, so we
@@ -908,7 +910,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 	 * Collect all the pending messages into a single contiguous array of
 	 * invalidation messages, to simplify what needs to happen while building
 	 * the commit WAL message.  Maintain the order that they would be
-	 * processed in by AtEOXact_Inval(), to ensure emulated behaviour in redo
+	 * processed in by AtCommit_Inval(), to ensure emulated behaviour in redo
 	 * is as similar as possible to original.  We want the same bugs, if any,
 	 * not new ones.
 	 */
@@ -955,7 +957,8 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
  * only at end-of-xact.
  *
  * Relcache init file invalidation requires processing both
- * before and after we send the SI messages. See AtEOXact_Inval()
+ * before and after we send the SI messages. See PreCommit_Inval()
+ * and AtCommit_Inval().
  */
 void
 ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
@@ -998,63 +1001,99 @@ ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 }
 
 /*
- * AtEOXact_Inval
- *		Process queued-up invalidation messages at end of main transaction.
- *
- * If isCommit, we must send out the messages in our PriorCmdInvalidMsgs 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).  This means that
- * we can skip immediate local processing of anything that's still in
- * CurrentCmdInvalidMsgs, and just send that list out too.
- *
- * If not isCommit, we are aborting, and must locally process the messages
- * in PriorCmdInvalidMsgs.  No messages need be sent to other backends,
- * since they'll not have seen our changed tuples anyway.  We can forget
- * about CurrentCmdInvalidMsgs too, since those changes haven't touched
- * the caches yet.
+ * PreCommit_Inval
+ *		Process queued-up invalidation before commit of main transaction.
+ *
+ * Call this after the last pre-commit action that could queue transactional
+ * invalidations.  (Direct SendSharedInvalidMessages() remains fine.)  Call
+ * this after any expensive processing, because we may hold an LWLock from
+ * here till end of xact.
+ *
+ * Tasks belong here if they are safe even if the xact later aborts.
+ * Currently, this just unlinks a file, failure of which does abort the xact.
+ */
+void
+PreCommit_Inval(void)
+{
+	/* This is a separate function just to run before the critical section. */
+	Assert(CritSectionCount == 0);
+
+	if (transInvalInfo == NULL)
+		return;
+
+	/* Must be at top of stack */
+	Assert(transInvalInfo->my_level == 1 && transInvalInfo->parent == NULL);
+
+	/*
+	 * Relcache init file invalidation requires processing both before and
+	 * after we send the SI messages.
+	 */
+	if (transInvalInfo->RelcacheInitFileInval)
+		RelationCacheInitFilePreInvalidate();
+}
+
+/*
+ * AtCommit_Inval
+ *		Process queued-up invalidations after commit of main transaction.
+ *
+ * Call this after TransactionIdCommitTree(), so consumers of the
+ * invalidations find any new rows.  Call it before locks are released (if
+ * anyone is waiting for lock on a relation we've modified, we want them to
+ * know about the catalog change before they start using the relation).
+ *
+ * We must send out the messages in our PriorCmdInvalidMsgs 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).  This means that we can skip immediate local
+ * processing of anything that's still in CurrentCmdInvalidMsgs, and just send
+ * that list out too.
  *
  * In any case, reset our state to empty.  We need not physically
  * free memory here, since TopTransactionContext is about to be emptied
  * anyway.
+ */
+void
+AtCommit_Inval(void)
+{
+	/* PANIC rather than let other backends use stale cache entries. */
+	Assert(CritSectionCount > 0);
+
+	if (transInvalInfo == NULL)
+		return;
+
+	AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
+							   &transInvalInfo->CurrentCmdInvalidMsgs);
+
+	ProcessInvalidationMessagesMulti(&transInvalInfo->PriorCmdInvalidMsgs,
+									 SendSharedInvalidMessages);
+
+	if (transInvalInfo->RelcacheInitFileInval)
+		RelationCacheInitFilePostInvalidate();
+
+	/* Need not free anything explicitly */
+	transInvalInfo = NULL;
+}
+
+/*
+ * AtAbort_Inval
+ *		Process queued-up invalidation messages at abort of main transaction.
  *
- * Note:
- *		This should be called as the last step in processing a transaction.
+ * We must locally process the messages in PriorCmdInvalidMsgs.  No messages
+ * need be sent to other backends, since they'll not have seen our changed
+ * tuples anyway.  We can forget about CurrentCmdInvalidMsgs too, since those
+ * changes haven't touched the caches yet.
  */
 void
-AtEOXact_Inval(bool isCommit)
+AtAbort_Inval(void)
 {
-	/* Quick exit if no messages */
 	if (transInvalInfo == NULL)
 		return;
 
 	/* Must be at top of stack */
 	Assert(transInvalInfo->my_level == 1 && transInvalInfo->parent == NULL);
 
-	if (isCommit)
-	{
-		/*
-		 * Relcache init file invalidation requires processing both before and
-		 * after we send the SI messages.  However, we need not do anything
-		 * unless we committed.
-		 */
-		if (transInvalInfo->RelcacheInitFileInval)
-			RelationCacheInitFilePreInvalidate();
-
-		AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
-								   &transInvalInfo->CurrentCmdInvalidMsgs);
-
-		ProcessInvalidationMessagesMulti(&transInvalInfo->PriorCmdInvalidMsgs,
-										 SendSharedInvalidMessages);
-
-		if (transInvalInfo->RelcacheInitFileInval)
-			RelationCacheInitFilePostInvalidate();
-	}
-	else
-	{
-		ProcessInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
-									LocalExecuteInvalidationMessage);
-	}
+	ProcessInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
+								LocalExecuteInvalidationMessage);
 
 	/* Need not free anything explicitly */
 	transInvalInfo = NULL;
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index 50c9440..8147cf6 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -352,7 +352,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.
+		 * have reached AtCommit_Inval().  See test at inplace-inval.spec.
 		 *
 		 * If a heap_update() call just released its LOCKTAG_TUPLE, we'll
 		 * probably find the old tuple and reach "tuple concurrently updated".
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..94d8b44 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -26,7 +26,9 @@ typedef void (*RelcacheCallbackFunction) (Datum arg, Oid relid);
 
 extern void AcceptInvalidationMessages(void);
 
-extern void AtEOXact_Inval(bool isCommit);
+extern void PreCommit_Inval(void);
+extern void AtCommit_Inval(void);
+extern void AtAbort_Inval(void);
 
 extern void AtEOSubXact_Inval(bool isCommit);
 
#3Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#2)
Re: Inval reliability, especially for inplace updates

On Sat, Jun 15, 2024 at 03:37:18PM -0700, Noah Misch wrote:

I'm attaching the implementation.

I'm withdrawing inplace150-inval-durability-atcommit-v1.patch, having found
two major problems so far:

1. It sends transactional invalidation messages before
ProcArrayEndTransaction(), so other backends can read stale data.

2. It didn't make the equivalent changes for COMMIT PREPARED.

#4Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#2)
2 attachment(s)
Re: Inval reliability, especially for inplace updates

On Sat, Jun 15, 2024 at 03:37:18PM -0700, Noah Misch wrote:

On Wed, May 22, 2024 at 05:05:48PM -0700, Noah Misch wrote:

/messages/by-id/20240512232923.aa.nmisch@google.com wrote:

Separable, nontrivial things not fixed in the attached patch stack:

- Inplace update uses transactional CacheInvalidateHeapTuple(). ROLLBACK of
CREATE INDEX wrongly discards the inval, leading to the relhasindex=t loss
still seen in inplace-inval.spec. CacheInvalidateRelmap() does this right.

I plan to fix that like CacheInvalidateRelmap(): send the inval immediately,
inside the critical section. Send it in heap_xlog_inplace(), too.

a. Within logical decoding, cease processing invalidations for inplace

I'm attaching the implementation. This applies atop the v3 patch stack from
/messages/by-id/20240614003549.c2.nmisch@google.com, but the threads are
mostly orthogonal and intended for independent review. Translating a tuple
into inval messages uses more infrastructure than relmapper, which needs just
a database ID. Hence, this ended up more like a miniature of inval.c's
participation in the transaction commit sequence.

I waffled on whether to back-patch inplace150-inval-durability-atcommit

That inplace150 patch turned out to be unnecessary. Contrary to the
"noncritical resource releasing" comment some lines above
AtEOXact_Inval(true), the actual behavior is already to promote ERROR to
PANIC. An ERROR just before or after sending invals becomes PANIC, "cannot
abort transaction %u, it was already committed". Since
inplace130-AtEOXact_RelationCache-comments existed to clear the way for
inplace150, inplace130 also becomes unnecessary. I've removed both from the
attached v2 patch stack.

The patch changes xl_heap_inplace of XLOG_HEAP_INPLACE. For back branches, we
could choose between:

- Same change, no WAL version bump. Standby must update before primary. This
is best long-term, but the transition is more disruptive. I'm leaning
toward this one, but the second option isn't bad:

- heap_xlog_inplace() could set the shared-inval-queue overflow signal on
every backend. This is more wasteful, but inplace updates might be rare
enough (~once per VACUUM) to make it tolerable.

- Use LogStandbyInvalidations() just after XLOG_HEAP_INPLACE. This isn't
correct if one ends recovery between the two records, but you'd need to be
unlucky to notice. Noticing would need a procedure like the following. A
hot standby backend populates a relcache entry, then does DDL on the rel
after recovery ends.

That still holds.

Attachments:

inplace140-heapam_xlog-comment-v2.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Remove comment about xl_heap_inplace "AT END OF STRUCT".
    
    Commit 2c03216d831160bedd72d45f712601b6f7d03f1c moved the tuple data
    from there to the buffer-0 data.  Back-patch to v12 (all supported
    versions), the plan for the next change to this struct.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 22a1747..42736f3 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -425,7 +425,6 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
-	/* TUPLE DATA FOLLOWS AT END OF STRUCT */
 } xl_heap_inplace;
 
 #define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
inplace160-inval-durability-inplace-v2.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Core code no longer needs XLOG_INVALIDATIONS, but this leaves removing
    it for future work.  Extensions could be relying on that mechanism, so
    that removal would not be back-patch material.  Back-patch to v12 (all
    supported versions).
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 797bddf..d7e417f 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6305,6 +6305,9 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 	Relation	relation = scan->heap_rel;
 	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;
@@ -6312,6 +6315,29 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 	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();
 
@@ -6341,9 +6367,16 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 		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);
@@ -6355,22 +6388,23 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
-	END_CRIT_SECTION();
-
 	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);
 	systable_endscan(scan);
 
-	/*
-	 * 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.
-	 */
-	if (!IsBootstrapProcessingMode())
-		CacheInvalidateHeapTuple(relation, tuple, NULL);
+	AcceptInvalidationMessages();	/* local processing of just-sent inval */
 }
 
 /*
@@ -10268,6 +10302,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 5f5673e..f31cc3a 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 25f870b..32e509a 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 9bda1aa..30285bd 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1358,14 +1358,23 @@ 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 formerly did so, they now send
+		 * immediate invalidations.  Extensions might still do so, 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 b4b68b1..ba576c6 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2884,19 +2884,21 @@ index_update_stats(Relation rel,
 	}
 
 	/*
-	 * If anything changed, write out the tuple
+	 * If anything changed, write out the tuple and immediate invals
 	 */
 	if (dirty)
-	{
 		heap_inplace_update_finish(state, tuple);
-		/* the above sends a cache inval message */
-	}
 	else
-	{
 		heap_inplace_update_cancel(state);
-		/* no need to change tuple, but force relcache inval anyway */
-		CacheInvalidateRelcacheByTuple(tuple);
-	}
+
+	/*
+	 * Queue a transactional relcache inval.  CREATE INDEX needs an immediate
+	 * inval for the relhasindex change, but it also needs a transactional
+	 * inval for when the new index's rows become visible.  Other CREATE INDEX
+	 * and REINDEX code happens to also queue a transactional inval, but keep
+	 * this in case rare callers rely on this part of our API contract.
+	 */
+	CacheInvalidateRelcacheByTuple(tuple);
 
 	heap_freetuple(tuple);
 
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 36d82bd..5d4173a 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.
 				 */
 				heap_inplace_update_finish(state, tuple);
 			}
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 8ec5adf..b2cf14e 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -508,23 +508,18 @@ 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.
+			 * Older WAL may contain commit-time invals for inplace updates.
+			 * 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 98aa527..b40e38c 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2287,7 +2287,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;
@@ -2328,7 +2329,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)
 		{
@@ -2337,7 +2338,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 603aa41..b3d3adb 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;
 
-	/* Events emitted by previous commands of this (sub)transaction */
-	InvalidationMsgsGroup PriorCmdInvalidMsgs;
-
 	/* 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;
+
+	/* 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,
@@ -1038,16 +1139,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
@@ -1058,6 +1159,45 @@ AtEOXact_Inval(bool isCommit)
 
 	/* Need not free anything explicitly */
 	transInvalInfo = NULL;
+	inplaceInvalInfo = NULL;
+}
+
+/*
+ * 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;
 }
 
 /*
@@ -1125,18 +1265,21 @@ 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;
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Successful inplace update must clear this. */
+		Assert(inplaceInvalInfo == NULL);
 	}
 	else
 	{
@@ -1148,6 +1291,9 @@ AtEOSubXact_Inval(bool isCommit)
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Reset from aborted inplace update. */
+		inplaceInvalInfo = NULL;
 	}
 }
 
@@ -1177,7 +1323,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 +1331,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 +1369,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 +1379,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 +1457,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 +1513,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1362,15 +1537,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 +1557,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1399,14 +1572,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 +1592,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 +1781,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 50c9440..f41b1c2 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".
diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 42736f3..4591e9a 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -20,6 +20,7 @@
 #include "storage/buf.h"
 #include "storage/bufpage.h"
 #include "storage/relfilelocator.h"
+#include "storage/sinval.h"
 #include "utils/relcache.h"
 
 
@@ -425,9 +426,14 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
+	Oid			dbId;			/* MyDatabaseId */
+	Oid			tsId;			/* MyDatabaseTableSpace */
+	bool		relcacheInitFileInval;	/* invalidate relcache init files */
+	int			nmsgs;			/* number of shared inval msgs */
+	SharedInvalidationMessage msgs[FLEXIBLE_ARRAY_MEMBER];
 } xl_heap_inplace;
 
-#define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
+#define MinSizeOfHeapInplace	(offsetof(xl_heap_inplace, nmsgs) + sizeof(int))
 
 /*
  * This is what we need to know about setting a visibility map bit
diff --git a/src/include/storage/sinval.h b/src/include/storage/sinval.h
index 8f5744b..c812237 100644
--- a/src/include/storage/sinval.h
+++ b/src/include/storage/sinval.h
@@ -144,6 +144,8 @@ extern void ProcessCatchupInterrupt(void);
 
 extern int	xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 												 bool *RelcacheInitFileInval);
+extern int	inplaceGetInvalidationMessages(SharedInvalidationMessage **msgs,
+										   bool *RelcacheInitFileInval);
 extern void ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 												 int nmsgs, bool RelcacheInitFileInval,
 												 Oid dbid, Oid tsid);
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..3390e7a 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index 67b34ad..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -14,7 +14,7 @@ step read1:
 
 relhasindex
 -----------
-f          
+t          
 (1 row)
 
 
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index d8e1c98..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -32,7 +32,7 @@ permutation
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
-	read1	# observe damage XXX is an extant bug
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 75433b3..0a69b58 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1252,6 +1252,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
#5Andres Freund
andres@anarazel.de
In reply to: Noah Misch (#4)
Re: Inval reliability, especially for inplace updates

Hi,

On 2024-06-17 16:58:54 -0700, Noah Misch wrote:

On Sat, Jun 15, 2024 at 03:37:18PM -0700, Noah Misch wrote:

On Wed, May 22, 2024 at 05:05:48PM -0700, Noah Misch wrote:

/messages/by-id/20240512232923.aa.nmisch@google.com wrote:

Separable, nontrivial things not fixed in the attached patch stack:

- Inplace update uses transactional CacheInvalidateHeapTuple(). ROLLBACK of
CREATE INDEX wrongly discards the inval, leading to the relhasindex=t loss
still seen in inplace-inval.spec. CacheInvalidateRelmap() does this right.

I plan to fix that like CacheInvalidateRelmap(): send the inval immediately,
inside the critical section. Send it in heap_xlog_inplace(), too.

I'm worried this might cause its own set of bugs, e.g. if there are any places
that, possibly accidentally, rely on the invalidation from the inplace update
to also cover separate changes.

Have you considered instead submitting these invalidations during abort as
well?

a. Within logical decoding, cease processing invalidations for inplace

I'm attaching the implementation. This applies atop the v3 patch stack from
/messages/by-id/20240614003549.c2.nmisch@google.com, but the threads are
mostly orthogonal and intended for independent review. Translating a tuple
into inval messages uses more infrastructure than relmapper, which needs just
a database ID. Hence, this ended up more like a miniature of inval.c's
participation in the transaction commit sequence.

I waffled on whether to back-patch inplace150-inval-durability-atcommit

That inplace150 patch turned out to be unnecessary. Contrary to the
"noncritical resource releasing" comment some lines above
AtEOXact_Inval(true), the actual behavior is already to promote ERROR to
PANIC. An ERROR just before or after sending invals becomes PANIC, "cannot
abort transaction %u, it was already committed".

Relying on that, instead of explicit critical sections, seems fragile to me.
IIRC some of the behaviour around errors around transaction commit/abort has
changed a bunch of times. Tying correctness into something that could be
changed for unrelated reasons doesn't seem great.

I'm not sure it holds true even today - what if the transaction didn't have an
xid? Then RecordTransactionAbort() wouldn't trigger
"cannot abort transaction %u, it was already committed"
I think?

- Same change, no WAL version bump. Standby must update before primary. This
is best long-term, but the transition is more disruptive. I'm leaning
toward this one, but the second option isn't bad:

Hm. The inplace record doesn't use the length of the "main data" record
segment for anything, from what I can tell. If records by an updated primary
were replayed by an old standby, it'd just ignore the additional data, afaict?

I think with the code as-is, the situation with an updated standby replaying
an old primary's record would actually be worse - it'd afaict just assume the
now-longer record contained valid fields, despite those just pointing into
uninitialized memory. I think the replay routine would have to check the
length of the main data and execute the invalidation conditionally.

- heap_xlog_inplace() could set the shared-inval-queue overflow signal on
every backend. This is more wasteful, but inplace updates might be rare
enough (~once per VACUUM) to make it tolerable.

We already set that surprisingly frequently, as
a) The size of the sinval queue is small
b) If a backend is busy, it does not process catchup interrupts
(i.e. executing queries, waiting for a lock prevents processing)
c) There's no deduplication of invals, we often end up sending the same inval
over and over.

So I suspect this might not be too bad, compared to the current badness.

At least for core code. I guess there could be extension code triggering
inplace updates more frequently? But I'd hope they'd do it not on catalog
tables... Except that we wouldn't know that that's the case during replay,
it's not contained in the record.

- Use LogStandbyInvalidations() just after XLOG_HEAP_INPLACE. This isn't
correct if one ends recovery between the two records, but you'd need to be
unlucky to notice. Noticing would need a procedure like the following. A
hot standby backend populates a relcache entry, then does DDL on the rel
after recovery ends.

Hm. The problematic cases presumably involves an access exclusive lock? If so,
could we do LogStandbyInvalidations() *before* logging the WAL record for the
inplace update? The invalidations can't be processed by other backends until
the exclusive lock has been released, which should avoid the race?

Greetings,

Andres Freund

#6Noah Misch
noah@leadboat.com
In reply to: Andres Freund (#5)
Re: Inval reliability, especially for inplace updates

On Mon, Jun 17, 2024 at 06:57:30PM -0700, Andres Freund wrote:

On 2024-06-17 16:58:54 -0700, Noah Misch wrote:

On Sat, Jun 15, 2024 at 03:37:18PM -0700, Noah Misch wrote:

On Wed, May 22, 2024 at 05:05:48PM -0700, Noah Misch wrote:

/messages/by-id/20240512232923.aa.nmisch@google.com wrote:

Separable, nontrivial things not fixed in the attached patch stack:

- Inplace update uses transactional CacheInvalidateHeapTuple(). ROLLBACK of
CREATE INDEX wrongly discards the inval, leading to the relhasindex=t loss
still seen in inplace-inval.spec. CacheInvalidateRelmap() does this right.

I plan to fix that like CacheInvalidateRelmap(): send the inval immediately,
inside the critical section. Send it in heap_xlog_inplace(), too.

I'm worried this might cause its own set of bugs, e.g. if there are any places
that, possibly accidentally, rely on the invalidation from the inplace update
to also cover separate changes.

Good point. I do have index_update_stats() still doing an ideally-superfluous
relcache update for that reason. Taking that further, it would be cheap
insurance to have the inplace update do a transactional inval in addition to
its immediate inval. Future master-only work could remove the transactional
one. How about that?

Have you considered instead submitting these invalidations during abort as
well?

I had not. Hmmm. If the lock protocol in README.tuplock (after patch
inplace120) told SearchSysCacheLocked1() to do systable scans instead of
syscache reads, that could work. Would need to ensure a PANIC if transaction
abort doesn't reach the inval submission. Overall, it would be harder to
reason about the state of caches, but I suspect the patch would be smaller.
How should we choose between those strategies?

a. Within logical decoding, cease processing invalidations for inplace

I'm attaching the implementation. This applies atop the v3 patch stack from
/messages/by-id/20240614003549.c2.nmisch@google.com, but the threads are
mostly orthogonal and intended for independent review. Translating a tuple
into inval messages uses more infrastructure than relmapper, which needs just
a database ID. Hence, this ended up more like a miniature of inval.c's
participation in the transaction commit sequence.

I waffled on whether to back-patch inplace150-inval-durability-atcommit

That inplace150 patch turned out to be unnecessary. Contrary to the
"noncritical resource releasing" comment some lines above
AtEOXact_Inval(true), the actual behavior is already to promote ERROR to
PANIC. An ERROR just before or after sending invals becomes PANIC, "cannot
abort transaction %u, it was already committed".

Relying on that, instead of explicit critical sections, seems fragile to me.
IIRC some of the behaviour around errors around transaction commit/abort has
changed a bunch of times. Tying correctness into something that could be
changed for unrelated reasons doesn't seem great.

Fair enough. It could still be a good idea for master, but given I missed a
bug in inplace150-inval-durability-atcommit-v1.patch far worse than the ones
$SUBJECT fixes, let's not risk it in back branches.

I'm not sure it holds true even today - what if the transaction didn't have an
xid? Then RecordTransactionAbort() wouldn't trigger
"cannot abort transaction %u, it was already committed"
I think?

I think that's right. As the inplace160-inval-durability-inplace-v2.patch
edits to xact.c say, the concept of invals in XID-less transactions is buggy
at its core. Fortunately, after that patch, we use them only for two things
that could themselves stop with something roughly as simple as the attached.

- Same change, no WAL version bump. Standby must update before primary. This
is best long-term, but the transition is more disruptive. I'm leaning
toward this one, but the second option isn't bad:

Hm. The inplace record doesn't use the length of the "main data" record
segment for anything, from what I can tell. If records by an updated primary
were replayed by an old standby, it'd just ignore the additional data, afaict?

Agreed, but ...

I think with the code as-is, the situation with an updated standby replaying
an old primary's record would actually be worse - it'd afaict just assume the
now-longer record contained valid fields, despite those just pointing into
uninitialized memory. I think the replay routine would have to check the
length of the main data and execute the invalidation conditionally.

I anticipated back branches supporting a new XLOG_HEAP_INPLACE_WITH_INVAL
alongside the old XLOG_HEAP_INPLACE. Updated standbys would run both fine,
and old binaries consuming new WAL would PANIC, "heap_redo: unknown op code".

- heap_xlog_inplace() could set the shared-inval-queue overflow signal on
every backend. This is more wasteful, but inplace updates might be rare
enough (~once per VACUUM) to make it tolerable.

We already set that surprisingly frequently, as
a) The size of the sinval queue is small
b) If a backend is busy, it does not process catchup interrupts
(i.e. executing queries, waiting for a lock prevents processing)
c) There's no deduplication of invals, we often end up sending the same inval
over and over.

So I suspect this might not be too bad, compared to the current badness.

That is good. We might be able to do the overflow signal once at end of
recovery, like RelationCacheInitFileRemove() does for the init file. That's
mildly harder to reason about, but it would be cheaper. Hmmm.

At least for core code. I guess there could be extension code triggering
inplace updates more frequently? But I'd hope they'd do it not on catalog
tables... Except that we wouldn't know that that's the case during replay,
it's not contained in the record.

For what it's worth, from a grep of PGXN, only citus does inplace updates.

- Use LogStandbyInvalidations() just after XLOG_HEAP_INPLACE. This isn't
correct if one ends recovery between the two records, but you'd need to be
unlucky to notice. Noticing would need a procedure like the following. A
hot standby backend populates a relcache entry, then does DDL on the rel
after recovery ends.

Hm. The problematic cases presumably involves an access exclusive lock? If so,
could we do LogStandbyInvalidations() *before* logging the WAL record for the
inplace update? The invalidations can't be processed by other backends until
the exclusive lock has been released, which should avoid the race?

A lock forces a backend to drain the inval queue before using the locked
object, but it doesn't stop the backend from draining the queue and
repopulating cache entries earlier. For example, pg_describe_object() can
query many syscaches without locking underlying objects. Hence, the inval
system relies on the buffer change getting fully visible to catcache queries
before the sinval message enters the shared queue.

Thanks,
nm

#7Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#6)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

On Tue, Jun 18, 2024 at 08:23:49AM -0700, Noah Misch wrote:

On Mon, Jun 17, 2024 at 06:57:30PM -0700, Andres Freund wrote:

On 2024-06-17 16:58:54 -0700, Noah Misch wrote:

On Sat, Jun 15, 2024 at 03:37:18PM -0700, Noah Misch wrote:

On Wed, May 22, 2024 at 05:05:48PM -0700, Noah Misch wrote:

/messages/by-id/20240512232923.aa.nmisch@google.com wrote:

Separable, nontrivial things not fixed in the attached patch stack:

- Inplace update uses transactional CacheInvalidateHeapTuple(). ROLLBACK of
CREATE INDEX wrongly discards the inval, leading to the relhasindex=t loss
still seen in inplace-inval.spec. CacheInvalidateRelmap() does this right.

I plan to fix that like CacheInvalidateRelmap(): send the inval immediately,
inside the critical section. Send it in heap_xlog_inplace(), too.

I'm worried this might cause its own set of bugs, e.g. if there are any places
that, possibly accidentally, rely on the invalidation from the inplace update
to also cover separate changes.

Good point. I do have index_update_stats() still doing an ideally-superfluous
relcache update for that reason. Taking that further, it would be cheap
insurance to have the inplace update do a transactional inval in addition to
its immediate inval. Future master-only work could remove the transactional
one. How about that?

Have you considered instead submitting these invalidations during abort as
well?

I had not. Hmmm. If the lock protocol in README.tuplock (after patch
inplace120) told SearchSysCacheLocked1() to do systable scans instead of
syscache reads, that could work. Would need to ensure a PANIC if transaction
abort doesn't reach the inval submission. Overall, it would be harder to
reason about the state of caches, but I suspect the patch would be smaller.
How should we choose between those strategies?

a. Within logical decoding, cease processing invalidations for inplace

I'm attaching the implementation. This applies atop the v3 patch stack from
/messages/by-id/20240614003549.c2.nmisch@google.com, but the threads are
mostly orthogonal and intended for independent review. Translating a tuple
into inval messages uses more infrastructure than relmapper, which needs just
a database ID. Hence, this ended up more like a miniature of inval.c's
participation in the transaction commit sequence.

I waffled on whether to back-patch inplace150-inval-durability-atcommit

That inplace150 patch turned out to be unnecessary. Contrary to the
"noncritical resource releasing" comment some lines above
AtEOXact_Inval(true), the actual behavior is already to promote ERROR to
PANIC. An ERROR just before or after sending invals becomes PANIC, "cannot
abort transaction %u, it was already committed".

Relying on that, instead of explicit critical sections, seems fragile to me.
IIRC some of the behaviour around errors around transaction commit/abort has
changed a bunch of times. Tying correctness into something that could be
changed for unrelated reasons doesn't seem great.

Fair enough. It could still be a good idea for master, but given I missed a
bug in inplace150-inval-durability-atcommit-v1.patch far worse than the ones
$SUBJECT fixes, let's not risk it in back branches.

I'm not sure it holds true even today - what if the transaction didn't have an
xid? Then RecordTransactionAbort() wouldn't trigger
"cannot abort transaction %u, it was already committed"
I think?

I think that's right. As the inplace160-inval-durability-inplace-v2.patch
edits to xact.c say, the concept of invals in XID-less transactions is buggy
at its core. Fortunately, after that patch, we use them only for two things
that could themselves stop with something roughly as simple as the attached.

Now actually attached.

Show quoted text

- Same change, no WAL version bump. Standby must update before primary. This
is best long-term, but the transition is more disruptive. I'm leaning
toward this one, but the second option isn't bad:

Hm. The inplace record doesn't use the length of the "main data" record
segment for anything, from what I can tell. If records by an updated primary
were replayed by an old standby, it'd just ignore the additional data, afaict?

Agreed, but ...

I think with the code as-is, the situation with an updated standby replaying
an old primary's record would actually be worse - it'd afaict just assume the
now-longer record contained valid fields, despite those just pointing into
uninitialized memory. I think the replay routine would have to check the
length of the main data and execute the invalidation conditionally.

I anticipated back branches supporting a new XLOG_HEAP_INPLACE_WITH_INVAL
alongside the old XLOG_HEAP_INPLACE. Updated standbys would run both fine,
and old binaries consuming new WAL would PANIC, "heap_redo: unknown op code".

- heap_xlog_inplace() could set the shared-inval-queue overflow signal on
every backend. This is more wasteful, but inplace updates might be rare
enough (~once per VACUUM) to make it tolerable.

We already set that surprisingly frequently, as
a) The size of the sinval queue is small
b) If a backend is busy, it does not process catchup interrupts
(i.e. executing queries, waiting for a lock prevents processing)
c) There's no deduplication of invals, we often end up sending the same inval
over and over.

So I suspect this might not be too bad, compared to the current badness.

That is good. We might be able to do the overflow signal once at end of
recovery, like RelationCacheInitFileRemove() does for the init file. That's
mildly harder to reason about, but it would be cheaper. Hmmm.

At least for core code. I guess there could be extension code triggering
inplace updates more frequently? But I'd hope they'd do it not on catalog
tables... Except that we wouldn't know that that's the case during replay,
it's not contained in the record.

For what it's worth, from a grep of PGXN, only citus does inplace updates.

- Use LogStandbyInvalidations() just after XLOG_HEAP_INPLACE. This isn't
correct if one ends recovery between the two records, but you'd need to be
unlucky to notice. Noticing would need a procedure like the following. A
hot standby backend populates a relcache entry, then does DDL on the rel
after recovery ends.

Hm. The problematic cases presumably involves an access exclusive lock? If so,
could we do LogStandbyInvalidations() *before* logging the WAL record for the
inplace update? The invalidations can't be processed by other backends until
the exclusive lock has been released, which should avoid the race?

A lock forces a backend to drain the inval queue before using the locked
object, but it doesn't stop the backend from draining the queue and
repopulating cache entries earlier. For example, pg_describe_object() can
query many syscaches without locking underlying objects. Hence, the inval
system relies on the buffer change getting fully visible to catcache queries
before the sinval message enters the shared queue.

Thanks,
nm

Attachments:

inval-requires-xid-v0.patchtext/plain; charset=us-asciiDownload
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 30285bd..0803e0a 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1377,11 +1377,7 @@ RecordTransactionCommit(void)
 		 * DELETE ROWS invals, but we've not done the work to withhold them.
 		 */
 		if (nmsgs != 0)
-		{
-			LogStandbyInvalidations(nmsgs, invalMessages,
-									RelcacheInitFileInval);
-			wrote_xlog = true;	/* not strictly necessary */
-		}
+			elog(ERROR, "cannot commit a transaction with invalidation messages but no xid");
 
 		/*
 		 * If we didn't create XLOG entries, we're done here; otherwise we
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 7fa80a5..3ef5b15 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -17439,6 +17439,10 @@ PreCommit_on_commit_actions(void)
 	List	   *oids_to_truncate = NIL;
 	List	   *oids_to_drop = NIL;
 
+	/* If no XID, we didn't create material to truncate or drop. */
+	if (!TransactionIdIsValid(GetCurrentTransactionIdIfAny()))
+		return;
+
 	foreach(l, on_commits)
 	{
 		OnCommitItem *oc = (OnCommitItem *) lfirst(l);
diff --git a/src/backend/commands/typecmds.c b/src/backend/commands/typecmds.c
index 2a1e713..d87cc85 100644
--- a/src/backend/commands/typecmds.c
+++ b/src/backend/commands/typecmds.c
@@ -2878,7 +2878,8 @@ AlterDomainDropConstraint(List *names, const char *constrName,
 	 * dependent plans get rebuilt.  Since this command doesn't change the
 	 * domain's pg_type row, that won't happen automatically; do it manually.
 	 */
-	CacheInvalidateHeapTuple(rel, tup, NULL);
+	if (found)
+		CacheInvalidateHeapTuple(rel, tup, NULL);
 
 	ObjectAddressSet(address, TypeRelationId, domainoid);
 
#8Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#4)
4 attachment(s)
Re: Inval reliability, especially for inplace updates

On Mon, Jun 17, 2024 at 04:58:54PM -0700, Noah Misch wrote:

attached v2 patch stack.

Rebased. This applies on top of three patches from
/messages/by-id/20240629024251.03.nmisch@google.com. I'm attaching those
to placate cfbot, but this thread is for review of the last patch only.

Attachments:

inplace090-LOCKTAG_TUPLE-eoxact-v5.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Warn if LOCKTAG_TUPLE is held at commit, under debug_assertions.
    
    The current use always releases this locktag.  A planned use will
    continue that intent.  It will involve more areas of code, making unlock
    omissions easier.  Warn under debug_assertions, like we do for various
    resource leaks.  Back-patch to v12 (all supported versions), the plan
    for the commit of the new use.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240512232923.aa.nmisch@google.com

diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 0400a50..461d925 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -2256,6 +2256,11 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
 				locallock->numLockOwners = 0;
 		}
 
+#ifdef USE_ASSERT_CHECKING
+		if (LOCALLOCK_LOCKTAG(*locallock) == LOCKTAG_TUPLE && !allLocks)
+			elog(WARNING, "tuple lock held at commit");
+#endif
+
 		/*
 		 * If the lock or proclock pointers are NULL, this lock was taken via
 		 * the relation fast-path (and is not known to have been transferred).
inplace110-successors-v5.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Fix data loss at inplace update after heap_update().
    
    As previously-added tests demonstrated, heap_inplace_update() could
    instead update an unrelated tuple of the same catalog.  It could lose
    the update.  Losing relhasindex=t was a source of index corruption.
    Inplace-updating commands like VACUUM will now wait for heap_update()
    commands like GRANT TABLE and GRANT DATABASE.  That isn't ideal, but a
    long-running GRANT already hurts VACUUM progress more just by keeping an
    XID running.  The VACUUM will behave like a DELETE or UPDATE waiting for
    the uncommitted change.
    
    For implementation details, start at the heap_inplace_update_scan()
    header comment and README.tuplock.  Back-patch to v12 (all supported
    versions).  In back branches, retain a deprecated heap_inplace_update(),
    for extensions.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/CAMp+ueZQz3yDk7qg42hk6-9gxniYbp-=bG2mgqecErqR5gGGOA@mail.gmail.com

diff --git a/src/backend/access/heap/README.tuplock b/src/backend/access/heap/README.tuplock
index 6441e8b..dbfa2b7 100644
--- a/src/backend/access/heap/README.tuplock
+++ b/src/backend/access/heap/README.tuplock
@@ -153,3 +153,56 @@ The following infomask bits are applicable:
 
 We currently never set the HEAP_XMAX_COMMITTED when the HEAP_XMAX_IS_MULTI bit
 is set.
+
+Locking to write inplace-updated tables
+---------------------------------------
+
+[This is the plan, but LOCKTAG_TUPLE acquisition is not yet here.]
+
+If IsInplaceUpdateRelation() returns true for a table, the table is a system
+catalog that receives heap_inplace_update_scan() calls.  Preparing a
+heap_update() of these tables follows additional locking rules, to ensure we
+don't lose the effects of an inplace update.  In particular, consider a moment
+when a backend has fetched the old tuple to modify, not yet having called
+heap_update().  Another backend's inplace update starting then can't conclude
+until the heap_update() places its new tuple in a buffer.  We enforce that
+using locktags as follows.  While DDL code is the main audience, the executor
+follows these rules to make e.g. "MERGE INTO pg_class" safer.  Locking rules
+are per-catalog:
+
+  pg_class heap_inplace_update_scan() callers: before the call, acquire
+  LOCKTAG_RELATION in mode ShareLock (CREATE INDEX), ShareUpdateExclusiveLock
+  (VACUUM), or a mode with strictly more conflicts.  If the update targets a
+  row of RELKIND_INDEX (but not RELKIND_PARTITIONED_INDEX), that lock must be
+  on the table.  Locking the index rel is optional.  (This allows VACUUM to
+  overwrite per-index pg_class while holding a lock on the table alone.)  We
+  could allow weaker locks, in which case the next paragraph would simply call
+  for stronger locks for its class of commands.  heap_inplace_update_scan()
+  acquires and releases LOCKTAG_TUPLE in InplaceUpdateTupleLock, an alias for
+  ExclusiveLock, on each tuple it overwrites.
+
+  pg_class heap_update() callers: before copying the tuple to modify, take a
+  lock that conflicts with at least one of those from the preceding paragraph.
+  SearchSysCacheLocked1() is one convenient way to acquire LOCKTAG_TUPLE.
+  After heap_update(), release any LOCKTAG_TUPLE.  Most of these callers opt
+  to acquire just the LOCKTAG_RELATION.
+
+  pg_database: before copying the tuple to modify, all updaters of pg_database
+  rows acquire LOCKTAG_TUPLE.  (Few updaters acquire LOCKTAG_OBJECT on the
+  database OID, so it wasn't worth extending that as a second option.)
+
+Ideally, DDL might want to perform permissions checks before LockTuple(), as
+we do with RangeVarGetRelidExtended() callbacks.  We typically don't bother.
+LOCKTAG_TUPLE acquirers release it after each row, so the potential
+inconvenience is lower.
+
+Reading inplace-updated columns
+-------------------------------
+
+Inplace updates create an exception to the rule that tuple data won't change
+under a reader holding a pin.  A reader of a heap_fetch() result tuple may
+witness a torn read.  Current inplace-updated fields are aligned and are no
+wider than four bytes, and current readers don't need consistency across
+fields.  Hence, they get by with just fetching each field once.  XXX such a
+caller may also read a value that has not reached WAL; see
+heap_inplace_update_finish().
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 91b2014..107507e 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -76,6 +76,9 @@ static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
 								  Buffer newbuf, HeapTuple oldtup,
 								  HeapTuple newtup, HeapTuple old_key_tuple,
 								  bool all_visible_cleared, bool new_all_visible_cleared);
+#ifdef USE_ASSERT_CHECKING
+static void check_inplace_rel_lock(HeapTuple oldtup);
+#endif
 static Bitmapset *HeapDetermineColumnsInfo(Relation relation,
 										   Bitmapset *interesting_cols,
 										   Bitmapset *external_cols,
@@ -97,6 +100,7 @@ static void compute_new_xmax_infomask(TransactionId xmax, uint16 old_infomask,
 static TM_Result heap_lock_updated_tuple(Relation rel, HeapTuple tuple,
 										 ItemPointer ctid, TransactionId xid,
 										 LockTupleMode mode);
+static bool inplace_xmax_lock(SysScanDesc scan);
 static void GetMultiXactIdHintBits(MultiXactId multi, uint16 *new_infomask,
 								   uint16 *new_infomask2);
 static TransactionId MultiXactIdGetUpdateXid(TransactionId xmax,
@@ -4072,6 +4076,45 @@ l2:
 	return TM_Ok;
 }
 
+#ifdef USE_ASSERT_CHECKING
+/*
+ * Confirm adequate relation lock held, per rules from README.tuplock section
+ * "Locking to write inplace-updated tables".
+ */
+static void
+check_inplace_rel_lock(HeapTuple oldtup)
+{
+	Form_pg_class classForm = (Form_pg_class) GETSTRUCT(oldtup);
+	Oid			relid = classForm->oid;
+	Oid			dbid;
+	LOCKTAG		tag;
+
+	if (IsSharedRelation(relid))
+		dbid = InvalidOid;
+	else
+		dbid = MyDatabaseId;
+
+	if (classForm->relkind == RELKIND_INDEX)
+	{
+		Relation	irel = index_open(relid, AccessShareLock);
+
+		SET_LOCKTAG_RELATION(tag, dbid, irel->rd_index->indrelid);
+		index_close(irel, AccessShareLock);
+	}
+	else
+		SET_LOCKTAG_RELATION(tag, dbid, relid);
+
+	if (!LockHeldByMe(&tag, ShareUpdateExclusiveLock, true))
+		elog(WARNING,
+			 "missing lock for relation \"%s\" (OID %u, relkind %c) @ TID (%u,%u)",
+			 NameStr(classForm->relname),
+			 relid,
+			 classForm->relkind,
+			 ItemPointerGetBlockNumber(&oldtup->t_self),
+			 ItemPointerGetOffsetNumber(&oldtup->t_self));
+}
+#endif
+
 /*
  * Check if the specified attribute's values are the same.  Subroutine for
  * HeapDetermineColumnsInfo.
@@ -6041,34 +6084,45 @@ heap_abort_speculative(Relation relation, ItemPointer tid)
 }
 
 /*
- * heap_inplace_update - update a tuple "in place" (ie, overwrite it)
+ * heap_inplace_update_scan - update a row "in place" (ie, overwrite it)
  *
- * Overwriting violates both MVCC and transactional safety, so the uses
- * of this function in Postgres are extremely limited.  Nonetheless we
- * find some places to use it.
+ * Overwriting violates both MVCC and transactional safety, so the uses of
+ * this function in Postgres are extremely limited.  Nonetheless we find some
+ * places to use it.  See README.tuplock section "Locking to write
+ * inplace-updated tables" and later sections for expectations of readers and
+ * writers of a table that gets inplace updates.  Standard flow:
  *
- * The tuple cannot change size, and therefore it's reasonable to assume
- * that its null bitmap (if any) doesn't change either.  So we just
- * overwrite the data portion of the tuple without touching the null
- * bitmap or any of the header fields.
+ * ... [any slow preparation not requiring oldtup] ...
+ * heap_inplace_update_scan([...], &tup, &inplace_state);
+ * if (!HeapTupleIsValid(tup))
+ *	elog(ERROR, [...]);
+ * ... [buffer is exclusive-locked; mutate "tup"] ...
+ * if (dirty)
+ *	heap_inplace_update_finish(inplace_state, tup);
+ * else
+ *	heap_inplace_update_cancel(inplace_state);
  *
- * tuple is an in-memory tuple structure containing the data to be written
- * over the target tuple.  Also, tuple->t_self identifies the target tuple.
+ * Since this is intended for system catalogs and SERIALIZABLE doesn't cover
+ * DDL, this skips some predicate locks.
  *
- * Note that the tuple updated here had better not come directly from the
- * syscache if the relation has a toast relation as this tuple could
- * include toast values that have been expanded, causing a failure here.
+ * The first several params duplicate the systable_beginscan() param list.
+ * "oldtupcopy" is an output parameter, assigned NULL if the key ceases to
+ * find a live tuple.  (In PROC_IN_VACUUM, that is a low-probability transient
+ * condition.)  If "oldtupcopy" gets non-NULL, you must pass output parameter
+ * "state" to heap_inplace_update_finish() or heap_inplace_update_cancel().
  */
 void
-heap_inplace_update(Relation relation, HeapTuple tuple)
+heap_inplace_update_scan(Relation relation,
+						 Oid indexId,
+						 bool indexOK,
+						 Snapshot snapshot,
+						 int nkeys, const ScanKeyData *key,
+						 HeapTuple *oldtupcopy, void **state)
 {
-	Buffer		buffer;
-	Page		page;
-	OffsetNumber offnum;
-	ItemId		lp = NULL;
-	HeapTupleHeader htup;
-	uint32		oldlen;
-	uint32		newlen;
+	ScanKey		mutable_key = palloc(sizeof(ScanKeyData) * nkeys);
+	int			retries = 0;
+	SysScanDesc scan;
+	HeapTuple	oldtup;
 
 	/*
 	 * For now, we don't allow parallel updates.  Unlike a regular update,
@@ -6081,21 +6135,70 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
 				(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
 				 errmsg("cannot update tuples during a parallel operation")));
 
-	INJECTION_POINT("inplace-before-pin");
-	buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
-	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
-	page = (Page) BufferGetPage(buffer);
+	/*
+	 * Accept a snapshot argument, for symmetry, but this function advances
+	 * its snapshot as needed to reach the tail of the updated tuple chain.
+	 */
+	Assert(snapshot == NULL);
 
-	offnum = ItemPointerGetOffsetNumber(&(tuple->t_self));
-	if (PageGetMaxOffsetNumber(page) >= offnum)
-		lp = PageGetItemId(page, offnum);
+	Assert(IsInplaceUpdateRelation(relation) || !IsSystemRelation(relation));
 
-	if (PageGetMaxOffsetNumber(page) < offnum || !ItemIdIsNormal(lp))
-		elog(ERROR, "invalid lp");
+	/* Loop for an exclusive-locked buffer of a non-updated tuple. */
+	do
+	{
+		CHECK_FOR_INTERRUPTS();
 
-	htup = (HeapTupleHeader) PageGetItem(page, lp);
+		/*
+		 * Processes issuing heap_update (e.g. GRANT) at maximum speed could
+		 * drive us to this error.  A hostile table owner has stronger ways to
+		 * damage their own table, so that's minor.
+		 */
+		if (retries++ > 10000)
+			elog(ERROR, "giving up after too many tries to overwrite row");
 
-	oldlen = ItemIdGetLength(lp) - htup->t_hoff;
+		memcpy(mutable_key, key, sizeof(ScanKeyData) * nkeys);
+		INJECTION_POINT("inplace-before-pin");
+		scan = systable_beginscan(relation, indexId, indexOK, snapshot,
+								  nkeys, mutable_key);
+		oldtup = systable_getnext(scan);
+		if (!HeapTupleIsValid(oldtup))
+		{
+			systable_endscan(scan);
+			*oldtupcopy = NULL;
+			return;
+		}
+
+#ifdef USE_ASSERT_CHECKING
+		if (RelationGetRelid(relation) == RelationRelationId)
+			check_inplace_rel_lock(oldtup);
+#endif
+	} while (!inplace_xmax_lock(scan));
+
+	*oldtupcopy = heap_copytuple(oldtup);
+	*state = scan;
+}
+
+/*
+ * heap_inplace_update_finish - second phase of heap_inplace_update_scan()
+ *
+ * The tuple cannot change size, and therefore its header fields and null
+ * bitmap (if any) don't change either.
+ */
+void
+heap_inplace_update_finish(void *state, HeapTuple tuple)
+{
+	SysScanDesc scan = (SysScanDesc) state;
+	TupleTableSlot *slot = scan->slot;
+	BufferHeapTupleTableSlot *bslot = (BufferHeapTupleTableSlot *) slot;
+	HeapTuple	oldtup = bslot->base.tuple;
+	HeapTupleHeader htup = oldtup->t_data;
+	Buffer		buffer = bslot->buffer;
+	Relation	relation = scan->heap_rel;
+	uint32		oldlen;
+	uint32		newlen;
+
+	Assert(ItemPointerEquals(&oldtup->t_self, &tuple->t_self));
+	oldlen = oldtup->t_len - htup->t_hoff;
 	newlen = tuple->t_len - tuple->t_data->t_hoff;
 	if (oldlen != newlen || htup->t_hoff != tuple->t_data->t_hoff)
 		elog(ERROR, "wrong tuple length");
@@ -6107,6 +6210,19 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
 		   (char *) tuple->t_data + tuple->t_data->t_hoff,
 		   newlen);
 
+	/*----------
+	 * XXX A crash here can allow datfrozenxid() to get ahead of relfrozenxid:
+	 *
+	 * ["D" is a VACUUM (ONLY_DATABASE_STATS)]
+	 * ["R" is a VACUUM tbl]
+	 * D: vac_update_datfrozenid() -> systable_beginscan(pg_class)
+	 * D: systable_getnext() returns pg_class tuple of tbl
+	 * R: memcpy() into pg_class tuple of tbl
+	 * D: raise pg_database.datfrozenxid, XLogInsert(), finish
+	 * [crash]
+	 * [recovery restores datfrozenxid w/o relfrozenxid]
+	 */
+
 	MarkBufferDirty(buffer);
 
 	/* XLOG stuff */
@@ -6127,23 +6243,188 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
 
 		recptr = XLogInsert(RM_HEAP_ID, XLOG_HEAP_INPLACE);
 
-		PageSetLSN(page, recptr);
+		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
 	END_CRIT_SECTION();
 
-	UnlockReleaseBuffer(buffer);
+	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+	systable_endscan(scan);
 
 	/*
 	 * 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.
 	 */
 	if (!IsBootstrapProcessingMode())
 		CacheInvalidateHeapTuple(relation, tuple, NULL);
 }
 
+/*
+ * heap_inplace_update_cancel - abandon a heap_inplace_update_scan()
+ *
+ * This is an alternative to making a no-op update.
+ */
+void
+heap_inplace_update_cancel(void *state)
+{
+	SysScanDesc scan = (SysScanDesc) state;
+	TupleTableSlot *slot = scan->slot;
+	BufferHeapTupleTableSlot *bslot = (BufferHeapTupleTableSlot *) slot;
+	Buffer		buffer = bslot->buffer;
+
+	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+	systable_endscan(scan);
+}
+
+/*
+ * inplace_xmax_lock - protect inplace update from concurrent heap_update()
+ *
+ * This operates on the last tuple that systable_getnext() returned.  Evaluate
+ * whether the tuple's state is compatible with a no-key update.  Current
+ * transaction rowmarks are fine, as is KEY SHARE from any transaction.  If
+ * compatible, return true with the buffer exclusive-locked.  Otherwise,
+ * return false after blocking transactions, if any, have ended.
+ *
+ * One could modify this to return true for tuples with delete in progress,
+ * All inplace updaters take lock that conflicts with DROP.  If it does happen
+ * somehow, we'll wait for it like we would an update.
+ *
+ * Readers of inplace-updated fields expect changes to those fields are
+ * durable.  For example, vac_truncate_clog() reads datfrozenxid from
+ * pg_database tuples via catalog snapshots.  A future snapshot must not
+ * return a lower datfrozenxid for the same database OID (lower in the
+ * FullTransactionIdPrecedes() sense).  We achieve that since no update of a
+ * tuple can start while we hold a lock on its buffer.  In cases like
+ * BEGIN;GRANT;CREATE INDEX;COMMIT we're inplace-updating a tuple visible only
+ * to this transaction.  ROLLBACK then is one case where it's okay to lose
+ * inplace updates.  (Restoring relhasindex=false on ROLLBACK is fine, since
+ * any concurrent CREATE INDEX would have blocked, then inplace-updated the
+ * committed tuple.)
+ *
+ * In principle, we could avoid waiting by overwriting every tuple in the
+ * updated tuple chain.  Reader expectations permit updating a tuple only if
+ * it's aborted, is the tail of the chain, or we already updated the tuple
+ * referenced in its t_ctid.  Hence, we would need to overwrite the tuples in
+ * order from tail to head.  That would tolerate either (a) mutating all
+ * tuples in one critical section or (b) accepting a chance of partial
+ * completion.  Partial completion of a relfrozenxid update would have the
+ * weird consequence that the table's next VACUUM could see the table's
+ * relfrozenxid move forward between vacuum_get_cutoffs() and finishing.
+ */
+static bool
+inplace_xmax_lock(SysScanDesc scan)
+{
+	TupleTableSlot *slot = scan->slot;
+	BufferHeapTupleTableSlot *bslot = (BufferHeapTupleTableSlot *) slot;
+	HeapTupleData oldtup = *bslot->base.tuple;
+	Buffer		buffer = bslot->buffer;
+	TM_Result	result;
+	bool		ret;
+
+	Assert(TTS_IS_BUFFERTUPLE(slot));
+	Assert(BufferIsValid(buffer));
+
+	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
+
+	/*----------
+	 * Interpret HeapTupleSatisfiesUpdate() like heap_update() does, except:
+	 *
+	 * - wait unconditionally
+	 * - no tuple locks
+	 * - don't recheck header after wait: simpler to defer to next iteration
+	 * - don't try to continue even if the updater aborts: likewise
+	 * - no crosscheck
+	 */
+	result = HeapTupleSatisfiesUpdate(&oldtup, GetCurrentCommandId(false),
+									  buffer);
+
+	if (result == TM_Invisible)
+	{
+		/* no known way this can happen */
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg_internal("attempted to overwrite invisible tuple")));
+	}
+	else if (result == TM_SelfModified)
+	{
+		/*
+		 * CREATE INDEX might reach this if an expression is silly enough to
+		 * call e.g. SELECT ... FROM pg_class FOR SHARE.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("tuple to be updated was already modified by an operation triggered by the current command")));
+	}
+	else if (result == TM_BeingModified)
+	{
+		TransactionId xwait;
+		uint16		infomask;
+		Relation	relation;
+
+		xwait = HeapTupleHeaderGetRawXmax(oldtup.t_data);
+		infomask = oldtup.t_data->t_infomask;
+		relation = scan->heap_rel;
+
+		if (infomask & HEAP_XMAX_IS_MULTI)
+		{
+			LockTupleMode lockmode = LockTupleNoKeyExclusive;
+			MultiXactStatus mxact_status = MultiXactStatusNoKeyUpdate;
+			int			remain;
+			bool		current_is_member;
+
+			if (DoesMultiXactIdConflict((MultiXactId) xwait, infomask,
+										lockmode, &current_is_member))
+			{
+				LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+				systable_endscan(scan);
+				ret = false;
+				MultiXactIdWait((MultiXactId) xwait, mxact_status, infomask,
+								relation, &oldtup.t_self, XLTW_Update,
+								&remain);
+			}
+			else
+				ret = true;
+		}
+		else if (TransactionIdIsCurrentTransactionId(xwait))
+			ret = true;
+		else if (HEAP_XMAX_IS_KEYSHR_LOCKED(infomask))
+			ret = true;
+		else
+		{
+			LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+			systable_endscan(scan);
+			ret = false;
+			XactLockTableWait(xwait, relation, &oldtup.t_self,
+							  XLTW_Update);
+		}
+	}
+	else
+	{
+		ret = (result == TM_Ok);
+		if (!ret)
+		{
+			LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+			systable_endscan(scan);
+		}
+	}
+
+	/*
+	 * GetCatalogSnapshot() relies on invalidation messages to know when to
+	 * take a new snapshot.  COMMIT of xwait is responsible for sending the
+	 * invalidation.  We're not acquiring heavyweight locks sufficient to
+	 * block if not yet sent, so we must take a new snapshot to avoid spinning
+	 * that ends with a "too many tries" error.  While we don't need this if
+	 * xwait aborted, don't bother optimizing that.
+	 */
+	if (!ret)
+		InvalidateCatalogSnapshot();
+	return ret;
+}
+
 #define		FRM_NOOP				0x0001
 #define		FRM_INVALIDATE_XMAX		0x0002
 #define		FRM_RETURN_IS_XID		0x0004
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index a819b41..b4b68b1 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2784,7 +2784,9 @@ index_update_stats(Relation rel,
 {
 	Oid			relid = RelationGetRelid(rel);
 	Relation	pg_class;
+	ScanKeyData key[1];
 	HeapTuple	tuple;
+	void	   *state;
 	Form_pg_class rd_rel;
 	bool		dirty;
 
@@ -2818,33 +2820,12 @@ index_update_stats(Relation rel,
 
 	pg_class = table_open(RelationRelationId, RowExclusiveLock);
 
-	/*
-	 * Make a copy of the tuple to update.  Normally we use the syscache, but
-	 * we can't rely on that during bootstrap or while reindexing pg_class
-	 * itself.
-	 */
-	if (IsBootstrapProcessingMode() ||
-		ReindexIsProcessingHeap(RelationRelationId))
-	{
-		/* don't assume syscache will work */
-		TableScanDesc pg_class_scan;
-		ScanKeyData key[1];
-
-		ScanKeyInit(&key[0],
-					Anum_pg_class_oid,
-					BTEqualStrategyNumber, F_OIDEQ,
-					ObjectIdGetDatum(relid));
-
-		pg_class_scan = table_beginscan_catalog(pg_class, 1, key);
-		tuple = heap_getnext(pg_class_scan, ForwardScanDirection);
-		tuple = heap_copytuple(tuple);
-		table_endscan(pg_class_scan);
-	}
-	else
-	{
-		/* normal case, use syscache */
-		tuple = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relid));
-	}
+	ScanKeyInit(&key[0],
+				Anum_pg_class_oid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(relid));
+	heap_inplace_update_scan(pg_class, ClassOidIndexId, true, NULL, 1, key,
+							 &tuple, &state);
 
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "could not find tuple for relation %u", relid);
@@ -2907,11 +2888,12 @@ index_update_stats(Relation rel,
 	 */
 	if (dirty)
 	{
-		heap_inplace_update(pg_class, tuple);
+		heap_inplace_update_finish(state, tuple);
 		/* the above sends a cache inval message */
 	}
 	else
 	{
+		heap_inplace_update_cancel(state);
 		/* no need to change tuple, but force relcache inval anyway */
 		CacheInvalidateRelcacheByTuple(tuple);
 	}
diff --git a/src/backend/catalog/toasting.c b/src/backend/catalog/toasting.c
index 738bc46..c882f3c 100644
--- a/src/backend/catalog/toasting.c
+++ b/src/backend/catalog/toasting.c
@@ -29,6 +29,7 @@
 #include "catalog/toasting.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
+#include "utils/fmgroids.h"
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
@@ -333,21 +334,36 @@ create_toast_table(Relation rel, Oid toastOid, Oid toastIndexOid,
 	 */
 	class_rel = table_open(RelationRelationId, RowExclusiveLock);
 
-	reltup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relOid));
-	if (!HeapTupleIsValid(reltup))
-		elog(ERROR, "cache lookup failed for relation %u", relOid);
-
-	((Form_pg_class) GETSTRUCT(reltup))->reltoastrelid = toast_relid;
-
 	if (!IsBootstrapProcessingMode())
 	{
 		/* normal case, use a transactional update */
+		reltup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relOid));
+		if (!HeapTupleIsValid(reltup))
+			elog(ERROR, "cache lookup failed for relation %u", relOid);
+
+		((Form_pg_class) GETSTRUCT(reltup))->reltoastrelid = toast_relid;
+
 		CatalogTupleUpdate(class_rel, &reltup->t_self, reltup);
 	}
 	else
 	{
 		/* While bootstrapping, we cannot UPDATE, so overwrite in-place */
-		heap_inplace_update(class_rel, reltup);
+
+		ScanKeyData key[1];
+		void	   *state;
+
+		ScanKeyInit(&key[0],
+					Anum_pg_class_oid,
+					BTEqualStrategyNumber, F_OIDEQ,
+					ObjectIdGetDatum(relOid));
+		heap_inplace_update_scan(class_rel, ClassOidIndexId, true,
+								 NULL, 1, key, &reltup, &state);
+		if (!HeapTupleIsValid(reltup))
+			elog(ERROR, "cache lookup failed for relation %u", relOid);
+
+		((Form_pg_class) GETSTRUCT(reltup))->reltoastrelid = toast_relid;
+
+		heap_inplace_update_finish(state, reltup);
 	}
 
 	heap_freetuple(reltup);
diff --git a/src/backend/commands/dbcommands.c b/src/backend/commands/dbcommands.c
index be629ea..da4d2b7 100644
--- a/src/backend/commands/dbcommands.c
+++ b/src/backend/commands/dbcommands.c
@@ -1637,6 +1637,8 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	bool		db_istemplate;
 	Relation	pgdbrel;
 	HeapTuple	tup;
+	ScanKeyData key[1];
+	void	   *inplace_state;
 	Form_pg_database datform;
 	int			notherbackends;
 	int			npreparedxacts;
@@ -1774,11 +1776,6 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	 */
 	pgstat_drop_database(db_id);
 
-	tup = SearchSysCacheCopy1(DATABASEOID, ObjectIdGetDatum(db_id));
-	if (!HeapTupleIsValid(tup))
-		elog(ERROR, "cache lookup failed for database %u", db_id);
-	datform = (Form_pg_database) GETSTRUCT(tup);
-
 	/*
 	 * Except for the deletion of the catalog row, subsequent actions are not
 	 * transactional (consider DropDatabaseBuffers() discarding modified
@@ -1790,8 +1787,17 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	 * modification is durable before performing irreversible filesystem
 	 * operations.
 	 */
+	ScanKeyInit(&key[0],
+				Anum_pg_database_oid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(db_id));
+	heap_inplace_update_scan(pgdbrel, DatabaseOidIndexId, true,
+							 NULL, 1, key, &tup, &inplace_state);
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for database %u", db_id);
+	datform = (Form_pg_database) GETSTRUCT(tup);
 	datform->datconnlimit = DATCONNLIMIT_INVALID_DB;
-	heap_inplace_update(pgdbrel, tup);
+	heap_inplace_update_finish(inplace_state, tup);
 	XLogFlush(XactLastRecEnd);
 
 	/*
@@ -1799,6 +1805,7 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	 * the row will be gone, but if we fail, dropdb() can be invoked again.
 	 */
 	CatalogTupleDelete(pgdbrel, &tup->t_self);
+	heap_freetuple(tup);
 
 	/*
 	 * Drop db-specific replication slots.
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 7a5ed6b..22d0ce7 100644
--- a/src/backend/commands/event_trigger.c
+++ b/src/backend/commands/event_trigger.c
@@ -946,25 +946,18 @@ EventTriggerOnLogin(void)
 		{
 			Relation	pg_db = table_open(DatabaseRelationId, RowExclusiveLock);
 			HeapTuple	tuple;
+			void	   *state;
 			Form_pg_database db;
 			ScanKeyData key[1];
-			SysScanDesc scan;
 
-			/*
-			 * Get the pg_database tuple to scribble on.  Note that this does
-			 * not directly rely on the syscache to avoid issues with
-			 * flattened toast values for the in-place update.
-			 */
+			/* Fetch a copy of the tuple to scribble on */
 			ScanKeyInit(&key[0],
 						Anum_pg_database_oid,
 						BTEqualStrategyNumber, F_OIDEQ,
 						ObjectIdGetDatum(MyDatabaseId));
 
-			scan = systable_beginscan(pg_db, DatabaseOidIndexId, true,
-									  NULL, 1, key);
-			tuple = systable_getnext(scan);
-			tuple = heap_copytuple(tuple);
-			systable_endscan(scan);
+			heap_inplace_update_scan(pg_db, DatabaseOidIndexId, true,
+									 NULL, 1, key, &tuple, &state);
 
 			if (!HeapTupleIsValid(tuple))
 				elog(ERROR, "could not find tuple for database %u", MyDatabaseId);
@@ -980,13 +973,15 @@ EventTriggerOnLogin(void)
 				 * that avoids possible waiting on the row-level lock. Second,
 				 * that avoids dealing with TOAST.
 				 *
-				 * It's known that changes made by heap_inplace_update() may
-				 * be lost due to concurrent normal updates.  However, we are
-				 * OK with that.  The subsequent connections will still have a
-				 * chance to set "dathasloginevt" to false.
+				 * 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.
 				 */
-				heap_inplace_update(pg_db, tuple);
+				heap_inplace_update_finish(state, tuple);
 			}
+			else
+				heap_inplace_update_cancel(state);
 			table_close(pg_db, RowExclusiveLock);
 			heap_freetuple(tuple);
 		}
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 48f8eab..d299a25 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -1405,7 +1405,9 @@ vac_update_relstats(Relation relation,
 {
 	Oid			relid = RelationGetRelid(relation);
 	Relation	rd;
+	ScanKeyData key[1];
 	HeapTuple	ctup;
+	void	   *inplace_state;
 	Form_pg_class pgcform;
 	bool		dirty,
 				futurexid,
@@ -1416,7 +1418,12 @@ vac_update_relstats(Relation relation,
 	rd = table_open(RelationRelationId, RowExclusiveLock);
 
 	/* Fetch a copy of the tuple to scribble on */
-	ctup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relid));
+	ScanKeyInit(&key[0],
+				Anum_pg_class_oid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(relid));
+	heap_inplace_update_scan(rd, ClassOidIndexId, true,
+							 NULL, 1, key, &ctup, &inplace_state);
 	if (!HeapTupleIsValid(ctup))
 		elog(ERROR, "pg_class entry for relid %u vanished during vacuuming",
 			 relid);
@@ -1524,7 +1531,9 @@ vac_update_relstats(Relation relation,
 
 	/* If anything changed, write out the tuple. */
 	if (dirty)
-		heap_inplace_update(rd, ctup);
+		heap_inplace_update_finish(inplace_state, ctup);
+	else
+		heap_inplace_update_cancel(inplace_state);
 
 	table_close(rd, RowExclusiveLock);
 
@@ -1576,6 +1585,7 @@ vac_update_datfrozenxid(void)
 	bool		bogus = false;
 	bool		dirty = false;
 	ScanKeyData key[1];
+	void	   *inplace_state;
 
 	/*
 	 * Restrict this task to one backend per database.  This avoids race
@@ -1699,20 +1709,18 @@ vac_update_datfrozenxid(void)
 	relation = table_open(DatabaseRelationId, RowExclusiveLock);
 
 	/*
-	 * Get the pg_database tuple to scribble on.  Note that this does not
-	 * directly rely on the syscache to avoid issues with flattened toast
-	 * values for the in-place update.
+	 * Fetch a copy of the tuple to scribble on.  We could check the syscache
+	 * tuple first.  If that concluded !dirty, we'd avoid waiting on
+	 * concurrent heap_update() and would avoid exclusive-locking the buffer.
+	 * For now, don't optimize that.
 	 */
 	ScanKeyInit(&key[0],
 				Anum_pg_database_oid,
 				BTEqualStrategyNumber, F_OIDEQ,
 				ObjectIdGetDatum(MyDatabaseId));
 
-	scan = systable_beginscan(relation, DatabaseOidIndexId, true,
-							  NULL, 1, key);
-	tuple = systable_getnext(scan);
-	tuple = heap_copytuple(tuple);
-	systable_endscan(scan);
+	heap_inplace_update_scan(relation, DatabaseOidIndexId, true,
+							 NULL, 1, key, &tuple, &inplace_state);
 
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "could not find tuple for database %u", MyDatabaseId);
@@ -1746,7 +1754,9 @@ vac_update_datfrozenxid(void)
 		newMinMulti = dbform->datminmxid;
 
 	if (dirty)
-		heap_inplace_update(relation, tuple);
+		heap_inplace_update_finish(inplace_state, tuple);
+	else
+		heap_inplace_update_cancel(inplace_state);
 
 	heap_freetuple(tuple);
 	table_close(relation, RowExclusiveLock);
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 9e9aec8..2e13fb9 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -336,7 +336,14 @@ extern TM_Result heap_lock_tuple(Relation relation, HeapTuple tuple,
 								 bool follow_updates,
 								 Buffer *buffer, struct TM_FailureData *tmfd);
 
-extern void heap_inplace_update(Relation relation, HeapTuple tuple);
+extern void heap_inplace_update_scan(Relation relation,
+									 Oid indexId,
+									 bool indexOK,
+									 Snapshot snapshot,
+									 int nkeys, const ScanKeyData *key,
+									 HeapTuple *oldtupcopy, void **state);
+extern void heap_inplace_update_finish(void *state, HeapTuple tuple);
+extern void heap_inplace_update_cancel(void *state);
 extern bool heap_prepare_freeze_tuple(HeapTupleHeader tuple,
 									  const struct VacuumCutoffs *cutoffs,
 									  HeapPageFreeze *pagefrz,
diff --git a/src/test/isolation/expected/intra-grant-inplace-db.out b/src/test/isolation/expected/intra-grant-inplace-db.out
index 432ece5..a91402c 100644
--- a/src/test/isolation/expected/intra-grant-inplace-db.out
+++ b/src/test/isolation/expected/intra-grant-inplace-db.out
@@ -9,20 +9,20 @@ step b1: BEGIN;
 step grant1: 
 	GRANT TEMP ON DATABASE isolation_regression TO regress_temp_grantee;
 
-step vac2: VACUUM (FREEZE);
+step vac2: VACUUM (FREEZE); <waiting ...>
 step snap3: 
 	INSERT INTO frozen_witness
 	SELECT datfrozenxid FROM pg_database WHERE datname = current_catalog;
 
 step c1: COMMIT;
+step vac2: <... completed>
 step cmp3: 
 	SELECT 'datfrozenxid retreated'
 	FROM pg_database
 	WHERE datname = current_catalog
 		AND age(datfrozenxid) > (SELECT min(age(x)) FROM frozen_witness);
 
-?column?              
-----------------------
-datfrozenxid retreated
-(1 row)
+?column?
+--------
+(0 rows)
 
diff --git a/src/test/isolation/expected/intra-grant-inplace.out b/src/test/isolation/expected/intra-grant-inplace.out
index cc1e47a..c2a9841 100644
--- a/src/test/isolation/expected/intra-grant-inplace.out
+++ b/src/test/isolation/expected/intra-grant-inplace.out
@@ -14,15 +14,16 @@ relhasindex
 f          
 (1 row)
 
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
 step c1: COMMIT;
+step addk2: <... completed>
 step read2: 
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass;
 
 relhasindex
 -----------
-f          
+t          
 (1 row)
 
 
@@ -58,8 +59,9 @@ relhasindex
 f          
 (1 row)
 
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
 step r3: ROLLBACK;
+step addk2: <... completed>
 
 starting permutation: b2 sfnku2 addk2 c2
 step b2: BEGIN;
@@ -122,17 +124,18 @@ relhasindex
 f          
 (1 row)
 
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
 step r3: ROLLBACK;
 step grant1: <... completed>
 step c1: COMMIT;
+step addk2: <... completed>
 step read2: 
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass;
 
 relhasindex
 -----------
-f          
+t          
 (1 row)
 
 
diff --git a/src/test/isolation/specs/intra-grant-inplace-db.spec b/src/test/isolation/specs/intra-grant-inplace-db.spec
index bbecd5d..9de40ec 100644
--- a/src/test/isolation/specs/intra-grant-inplace-db.spec
+++ b/src/test/isolation/specs/intra-grant-inplace-db.spec
@@ -42,5 +42,4 @@ step cmp3	{
 }
 
 
-# XXX extant bug
 permutation snap3 b1 grant1 vac2(c1) snap3 c1 cmp3
diff --git a/src/test/isolation/specs/intra-grant-inplace.spec b/src/test/isolation/specs/intra-grant-inplace.spec
index 3cd696b..eed0b52 100644
--- a/src/test/isolation/specs/intra-grant-inplace.spec
+++ b/src/test/isolation/specs/intra-grant-inplace.spec
@@ -73,7 +73,7 @@ step keyshr5	{
 teardown	{ ROLLBACK; }
 
 
-# XXX extant bugs: permutation comments refer to planned post-bugfix behavior
+# XXX extant bugs: permutation comments refer to planned future LockTuple()
 
 permutation
 	b1
diff --git a/src/test/modules/injection_points/expected/inplace.out b/src/test/modules/injection_points/expected/inplace.out
index 123f45a..db7dab6 100644
--- a/src/test/modules/injection_points/expected/inplace.out
+++ b/src/test/modules/injection_points/expected/inplace.out
@@ -40,4 +40,301 @@ step read1:
 	SELECT reltuples = -1 AS reltuples_unknown
 	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
 
-ERROR:  could not create unique index "pg_class_oid_index"
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 c2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step c2: COMMIT;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 r2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step r2: ROLLBACK;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 c2 revoke2 grant2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step c2: COMMIT;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: vac1 begin2 grant2 revoke2 mkrels3 c2 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step c2: COMMIT;
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 r2 grant2 revoke2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step r2: ROLLBACK;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 c2 revoke2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step c2: COMMIT;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
diff --git a/src/test/modules/injection_points/specs/inplace.spec b/src/test/modules/injection_points/specs/inplace.spec
index e957713..86539a5 100644
--- a/src/test/modules/injection_points/specs/inplace.spec
+++ b/src/test/modules/injection_points/specs/inplace.spec
@@ -32,12 +32,9 @@ setup
 	CREATE TABLE vactest.orig50 ();
 	SELECT vactest.mkrels('orig', 51, 100);
 }
-
-# XXX DROP causes an assertion failure; adopt DROP once fixed
 teardown
 {
-	--DROP SCHEMA vactest CASCADE;
-	DO $$BEGIN EXECUTE 'ALTER SCHEMA vactest RENAME TO schema' || oid FROM pg_namespace where nspname = 'vactest'; END$$;
+	DROP SCHEMA vactest CASCADE;
 	DROP EXTENSION injection_points;
 }
 
@@ -56,11 +53,13 @@ step read1	{
 	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
 }
 
-
 # Transactional updates of the tuple vac1 is waiting to inplace-update.
 session s2
 step grant2		{ GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC; }
-
+step revoke2	{ REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC; }
+step begin2		{ BEGIN; }
+step c2			{ COMMIT; }
+step r2			{ ROLLBACK; }
 
 # Non-blocking actions.
 session s3
@@ -74,10 +73,69 @@ step mkrels3	{
 }
 
 
-# XXX extant bug
+# target gains a successor at the last moment
 permutation
 	vac1(mkrels3)	# reads pg_class tuple T0 for vactest.orig50, xmax invalid
 	grant2			# T0 becomes eligible for pruning, T1 is successor
 	vac3			# T0 becomes LP_UNUSED
-	mkrels3			# T0 reused; vac1 wakes and overwrites the reused T0
+	mkrels3			# vac1 wakes, scans to T1
 	read1
+
+# target already has a successor, which commits
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1
+	vac1(mkrels3)	# reads T0 for vactest.orig50
+	c2				# T0 becomes eligible for pruning
+	vac3			# T0 becomes LP_UNUSED
+	mkrels3			# vac1 wakes, scans to T1
+	read1
+
+# target already has a successor, which becomes LP_UNUSED at the last moment
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1
+	vac1(mkrels3)	# reads T0 for vactest.orig50
+	r2				# T1 becomes eligible for pruning
+	vac3			# T1 becomes LP_UNUSED
+	mkrels3			# reuse T1; vac1 scans to T0
+	read1
+
+# target already has a successor, which becomes LP_REDIRECT at the last moment
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1, non-HOT due to filled page
+	vac1(mkrels3)	# reads T0
+	c2
+	revoke2			# HOT update to T2
+	grant2			# HOT update to T3
+	vac3			# T1 becomes LP_REDIRECT
+	mkrels3			# reuse T2; vac1 scans to T3
+	read1
+
+# waiting for updater to end
+permutation
+	vac1(c2)		# reads pg_class tuple T0 for vactest.orig50, xmax invalid
+	begin2
+	grant2			# T0.t_ctid = T1, non-HOT due to filled page
+	revoke2			# HOT update to T2
+	mkrels3			# vac1 awakes briefly, then waits for s2
+	c2
+	read1
+
+# Another LP_UNUSED.  This time, do change the live tuple.  Final live tuple
+# body is identical to original, at a different TID.
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1, non-HOT due to filled page
+	vac1(mkrels3)	# reads T0
+	r2				# T1 becomes eligible for pruning
+	grant2			# T0.t_ctid = T2; T0 becomes eligible for pruning
+	revoke2			# T2.t_ctid = T3; T2 becomes eligible for pruning
+	vac3			# T0, T1 & T2 become LP_UNUSED
+	mkrels3			# reuse T0, T1 & T2; vac1 scans to T3
+	read1
+
+# Another LP_REDIRECT.  Compared to the earlier test, omit the last grant2.
+# Hence, final live tuple body is identical to original, at a different TID.
+permutation begin2 grant2 vac1(mkrels3) c2 revoke2 vac3 mkrels3 read1
inplace120-locktag-v5.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Make heap_update() callers wait for inplace update.
    
    The previous commit fixed some ways of losing an inplace update.  It
    remained possible to lose one when a backend working toward a
    heap_update() copied a tuple into memory just before inplace update of
    that tuple.  In catalogs eligible for inplace update, use LOCKTAG_TUPLE
    to govern admission to the steps of copying an old tuple, modifying it,
    and issuing heap_update().  This includes UPDATE and MERGE commands.  To
    avoid changing most of the pg_class DDL, don't require LOCKTAG_TUPLE
    when holding a relation lock sufficient to exclude inplace updaters.
    Back-patch to v12 (all supported versions).
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20231027214946.79.nmisch@google.com

diff --git a/src/backend/access/heap/README.tuplock b/src/backend/access/heap/README.tuplock
index dbfa2b7..fb06ff2 100644
--- a/src/backend/access/heap/README.tuplock
+++ b/src/backend/access/heap/README.tuplock
@@ -157,8 +157,6 @@ is set.
 Locking to write inplace-updated tables
 ---------------------------------------
 
-[This is the plan, but LOCKTAG_TUPLE acquisition is not yet here.]
-
 If IsInplaceUpdateRelation() returns true for a table, the table is a system
 catalog that receives heap_inplace_update_scan() calls.  Preparing a
 heap_update() of these tables follows additional locking rules, to ensure we
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 107507e..797bddf 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -51,6 +51,8 @@
 #include "access/xloginsert.h"
 #include "access/xlogutils.h"
 #include "catalog/catalog.h"
+#include "catalog/pg_database.h"
+#include "catalog/pg_database_d.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
@@ -77,6 +79,9 @@ static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
 								  HeapTuple newtup, HeapTuple old_key_tuple,
 								  bool all_visible_cleared, bool new_all_visible_cleared);
 #ifdef USE_ASSERT_CHECKING
+static void check_lock_if_inplace_updateable_rel(Relation relation,
+												 ItemPointer otid,
+												 HeapTuple newtup);
 static void check_inplace_rel_lock(HeapTuple oldtup);
 #endif
 static Bitmapset *HeapDetermineColumnsInfo(Relation relation,
@@ -126,6 +131,8 @@ static HeapTuple ExtractReplicaIdentity(Relation relation, HeapTuple tp, bool ke
  * heavyweight lock mode and MultiXactStatus values to use for any particular
  * tuple lock strength.
  *
+ * These interact with InplaceUpdateTupleLock, an alias for ExclusiveLock.
+ *
  * Don't look at lockstatus/updstatus directly!  Use get_mxact_status_for_lock
  * instead.
  */
@@ -3212,6 +3219,10 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
 				(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
 				 errmsg("cannot update tuples during a parallel operation")));
 
+#ifdef USE_ASSERT_CHECKING
+	check_lock_if_inplace_updateable_rel(relation, otid, newtup);
+#endif
+
 	/*
 	 * Fetch the list of attributes to be checked for various operations.
 	 *
@@ -4078,6 +4089,89 @@ l2:
 
 #ifdef USE_ASSERT_CHECKING
 /*
+ * Confirm adequate lock held during heap_update(), per rules from
+ * README.tuplock section "Locking to write inplace-updated tables".
+ */
+static void
+check_lock_if_inplace_updateable_rel(Relation relation,
+									 ItemPointer otid,
+									 HeapTuple newtup)
+{
+	/* LOCKTAG_TUPLE acceptable for any catalog */
+	switch (RelationGetRelid(relation))
+	{
+		case RelationRelationId:
+		case DatabaseRelationId:
+			{
+				LOCKTAG		tuptag;
+
+				SET_LOCKTAG_TUPLE(tuptag,
+								  relation->rd_lockInfo.lockRelId.dbId,
+								  relation->rd_lockInfo.lockRelId.relId,
+								  ItemPointerGetBlockNumber(otid),
+								  ItemPointerGetOffsetNumber(otid));
+				if (LockHeldByMe(&tuptag, InplaceUpdateTupleLock, false))
+					return;
+			}
+			break;
+		default:
+			Assert(!IsInplaceUpdateRelation(relation));
+			return;
+	}
+
+	switch (RelationGetRelid(relation))
+	{
+		case RelationRelationId:
+			{
+				/* LOCKTAG_TUPLE or LOCKTAG_RELATION ok */
+				Form_pg_class classForm = (Form_pg_class) GETSTRUCT(newtup);
+				Oid			relid = classForm->oid;
+				Oid			dbid;
+				LOCKTAG		tag;
+
+				if (IsSharedRelation(relid))
+					dbid = InvalidOid;
+				else
+					dbid = MyDatabaseId;
+
+				if (classForm->relkind == RELKIND_INDEX)
+				{
+					Relation	irel = index_open(relid, AccessShareLock);
+
+					SET_LOCKTAG_RELATION(tag, dbid, irel->rd_index->indrelid);
+					index_close(irel, AccessShareLock);
+				}
+				else
+					SET_LOCKTAG_RELATION(tag, dbid, relid);
+
+				if (!LockHeldByMe(&tag, ShareUpdateExclusiveLock, false) &&
+					!LockHeldByMe(&tag, ShareRowExclusiveLock, true))
+					elog(WARNING,
+						 "missing lock for relation \"%s\" (OID %u, relkind %c) @ TID (%u,%u)",
+						 NameStr(classForm->relname),
+						 relid,
+						 classForm->relkind,
+						 ItemPointerGetBlockNumber(otid),
+						 ItemPointerGetOffsetNumber(otid));
+			}
+			break;
+		case DatabaseRelationId:
+			{
+				/* LOCKTAG_TUPLE required */
+				Form_pg_database dbForm = (Form_pg_database) GETSTRUCT(newtup);
+
+				elog(WARNING,
+					 "missing lock on database \"%s\" (OID %u) @ TID (%u,%u)",
+					 NameStr(dbForm->datname),
+					 dbForm->oid,
+					 ItemPointerGetBlockNumber(otid),
+					 ItemPointerGetOffsetNumber(otid));
+			}
+			break;
+	}
+}
+
+/*
  * Confirm adequate relation lock held, per rules from README.tuplock section
  * "Locking to write inplace-updated tables".
  */
@@ -6123,6 +6217,7 @@ heap_inplace_update_scan(Relation relation,
 	int			retries = 0;
 	SysScanDesc scan;
 	HeapTuple	oldtup;
+	ItemPointerData locked;
 
 	/*
 	 * For now, we don't allow parallel updates.  Unlike a regular update,
@@ -6144,6 +6239,7 @@ heap_inplace_update_scan(Relation relation,
 	Assert(IsInplaceUpdateRelation(relation) || !IsSystemRelation(relation));
 
 	/* Loop for an exclusive-locked buffer of a non-updated tuple. */
+	ItemPointerSetInvalid(&locked);
 	do
 	{
 		CHECK_FOR_INTERRUPTS();
@@ -6163,6 +6259,8 @@ heap_inplace_update_scan(Relation relation,
 		oldtup = systable_getnext(scan);
 		if (!HeapTupleIsValid(oldtup))
 		{
+			if (ItemPointerIsValid(&locked))
+				UnlockTuple(relation, &locked, InplaceUpdateTupleLock);
 			systable_endscan(scan);
 			*oldtupcopy = NULL;
 			return;
@@ -6172,6 +6270,15 @@ heap_inplace_update_scan(Relation relation,
 		if (RelationGetRelid(relation) == RelationRelationId)
 			check_inplace_rel_lock(oldtup);
 #endif
+
+		if (!(ItemPointerIsValid(&locked) &&
+			  ItemPointerEquals(&locked, &oldtup->t_self)))
+		{
+			if (ItemPointerIsValid(&locked))
+				UnlockTuple(relation, &locked, InplaceUpdateTupleLock);
+			LockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
+		}
+		locked = oldtup->t_self;
 	} while (!inplace_xmax_lock(scan));
 
 	*oldtupcopy = heap_copytuple(oldtup);
@@ -6183,6 +6290,8 @@ heap_inplace_update_scan(Relation relation,
  *
  * The tuple cannot change size, and therefore its header fields and null
  * bitmap (if any) don't change either.
+ *
+ * Since we hold LOCKTAG_TUPLE, no updater has a local copy of this tuple.
  */
 void
 heap_inplace_update_finish(void *state, HeapTuple tuple)
@@ -6249,6 +6358,7 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 	END_CRIT_SECTION();
 
 	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+	UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
 	systable_endscan(scan);
 
 	/*
@@ -6274,9 +6384,12 @@ heap_inplace_update_cancel(void *state)
 	SysScanDesc scan = (SysScanDesc) state;
 	TupleTableSlot *slot = scan->slot;
 	BufferHeapTupleTableSlot *bslot = (BufferHeapTupleTableSlot *) slot;
+	HeapTuple	oldtup = bslot->base.tuple;
 	Buffer		buffer = bslot->buffer;
+	Relation	relation = scan->heap_rel;
 
 	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+	UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
 	systable_endscan(scan);
 }
 
@@ -6334,7 +6447,7 @@ inplace_xmax_lock(SysScanDesc scan)
 	 * Interpret HeapTupleSatisfiesUpdate() like heap_update() does, except:
 	 *
 	 * - wait unconditionally
-	 * - no tuple locks
+	 * - caller handles tuple lock, since inplace needs it unconditionally
 	 * - don't recheck header after wait: simpler to defer to next iteration
 	 * - don't try to continue even if the updater aborts: likewise
 	 * - no crosscheck
diff --git a/src/backend/catalog/aclchk.c b/src/backend/catalog/aclchk.c
index a44ccee..bc0e259 100644
--- a/src/backend/catalog/aclchk.c
+++ b/src/backend/catalog/aclchk.c
@@ -75,6 +75,7 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_func.h"
 #include "parser/parse_type.h"
+#include "storage/lmgr.h"
 #include "utils/acl.h"
 #include "utils/aclchk_internal.h"
 #include "utils/builtins.h"
@@ -1848,7 +1849,7 @@ ExecGrant_Relation(InternalGrant *istmt)
 		HeapTuple	tuple;
 		ListCell   *cell_colprivs;
 
-		tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(relOid));
+		tuple = SearchSysCacheLocked1(RELOID, ObjectIdGetDatum(relOid));
 		if (!HeapTupleIsValid(tuple))
 			elog(ERROR, "cache lookup failed for relation %u", relOid);
 		pg_class_tuple = (Form_pg_class) GETSTRUCT(tuple);
@@ -2060,6 +2061,7 @@ ExecGrant_Relation(InternalGrant *istmt)
 										 values, nulls, replaces);
 
 			CatalogTupleUpdate(relation, &newtuple->t_self, newtuple);
+			UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
 
 			/* Update initial privileges for extensions */
 			recordExtensionInitPriv(relOid, RelationRelationId, 0, new_acl);
@@ -2072,6 +2074,8 @@ ExecGrant_Relation(InternalGrant *istmt)
 
 			pfree(new_acl);
 		}
+		else
+			UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
 
 		/*
 		 * Handle column-level privileges, if any were specified or implied.
@@ -2185,7 +2189,7 @@ ExecGrant_common(InternalGrant *istmt, Oid classid, AclMode default_privs,
 		Oid		   *oldmembers;
 		Oid		   *newmembers;
 
-		tuple = SearchSysCache1(cacheid, ObjectIdGetDatum(objectid));
+		tuple = SearchSysCacheLocked1(cacheid, ObjectIdGetDatum(objectid));
 		if (!HeapTupleIsValid(tuple))
 			elog(ERROR, "cache lookup failed for %s %u", get_object_class_descr(classid), objectid);
 
@@ -2261,6 +2265,7 @@ ExecGrant_common(InternalGrant *istmt, Oid classid, AclMode default_privs,
 									 nulls, replaces);
 
 		CatalogTupleUpdate(relation, &newtuple->t_self, newtuple);
+		UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
 
 		/* Update initial privileges for extensions */
 		recordExtensionInitPriv(objectid, classid, 0, new_acl);
diff --git a/src/backend/catalog/catalog.c b/src/backend/catalog/catalog.c
index 6c39434..8aefbcd 100644
--- a/src/backend/catalog/catalog.c
+++ b/src/backend/catalog/catalog.c
@@ -138,6 +138,15 @@ IsCatalogRelationOid(Oid relid)
 /*
  * IsInplaceUpdateRelation
  *		True iff core code performs inplace updates on the relation.
+ *
+ *		This is used for assertions and for making the executor follow the
+ *		locking protocol described at README.tuplock section "Locking to write
+ *		inplace-updated tables".  Extensions may inplace-update other heap
+ *		tables, but concurrent SQL UPDATE on the same table may overwrite
+ *		those modifications.
+ *
+ *		The executor can assume these are not partitions or partitioned and
+ *		have no triggers.
  */
 bool
 IsInplaceUpdateRelation(Relation relation)
diff --git a/src/backend/commands/dbcommands.c b/src/backend/commands/dbcommands.c
index da4d2b7..fd48022 100644
--- a/src/backend/commands/dbcommands.c
+++ b/src/backend/commands/dbcommands.c
@@ -1864,6 +1864,7 @@ RenameDatabase(const char *oldname, const char *newname)
 {
 	Oid			db_id;
 	HeapTuple	newtup;
+	ItemPointerData otid;
 	Relation	rel;
 	int			notherbackends;
 	int			npreparedxacts;
@@ -1935,11 +1936,13 @@ RenameDatabase(const char *oldname, const char *newname)
 				 errdetail_busy_db(notherbackends, npreparedxacts)));
 
 	/* rename */
-	newtup = SearchSysCacheCopy1(DATABASEOID, ObjectIdGetDatum(db_id));
+	newtup = SearchSysCacheLockedCopy1(DATABASEOID, ObjectIdGetDatum(db_id));
 	if (!HeapTupleIsValid(newtup))
 		elog(ERROR, "cache lookup failed for database %u", db_id);
+	otid = newtup->t_self;
 	namestrcpy(&(((Form_pg_database) GETSTRUCT(newtup))->datname), newname);
-	CatalogTupleUpdate(rel, &newtup->t_self, newtup);
+	CatalogTupleUpdate(rel, &otid, newtup);
+	UnlockTuple(rel, &otid, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(DatabaseRelationId, db_id, 0);
 
@@ -2188,6 +2191,7 @@ movedb(const char *dbname, const char *tblspcname)
 			ereport(ERROR,
 					(errcode(ERRCODE_UNDEFINED_DATABASE),
 					 errmsg("database \"%s\" does not exist", dbname)));
+		LockTuple(pgdbrel, &oldtuple->t_self, InplaceUpdateTupleLock);
 
 		new_record[Anum_pg_database_dattablespace - 1] = ObjectIdGetDatum(dst_tblspcoid);
 		new_record_repl[Anum_pg_database_dattablespace - 1] = true;
@@ -2196,6 +2200,7 @@ movedb(const char *dbname, const char *tblspcname)
 									 new_record,
 									 new_record_nulls, new_record_repl);
 		CatalogTupleUpdate(pgdbrel, &oldtuple->t_self, newtuple);
+		UnlockTuple(pgdbrel, &oldtuple->t_self, InplaceUpdateTupleLock);
 
 		InvokeObjectPostAlterHook(DatabaseRelationId, db_id, 0);
 
@@ -2426,6 +2431,7 @@ AlterDatabase(ParseState *pstate, AlterDatabaseStmt *stmt, bool isTopLevel)
 		ereport(ERROR,
 				(errcode(ERRCODE_UNDEFINED_DATABASE),
 				 errmsg("database \"%s\" does not exist", stmt->dbname)));
+	LockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	datform = (Form_pg_database) GETSTRUCT(tuple);
 	dboid = datform->oid;
@@ -2475,6 +2481,7 @@ AlterDatabase(ParseState *pstate, AlterDatabaseStmt *stmt, bool isTopLevel)
 	newtuple = heap_modify_tuple(tuple, RelationGetDescr(rel), new_record,
 								 new_record_nulls, new_record_repl);
 	CatalogTupleUpdate(rel, &tuple->t_self, newtuple);
+	UnlockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(DatabaseRelationId, dboid, 0);
 
@@ -2524,6 +2531,7 @@ AlterDatabaseRefreshColl(AlterDatabaseRefreshCollStmt *stmt)
 	if (!object_ownercheck(DatabaseRelationId, db_id, GetUserId()))
 		aclcheck_error(ACLCHECK_NOT_OWNER, OBJECT_DATABASE,
 					   stmt->dbname);
+	LockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	datum = heap_getattr(tuple, Anum_pg_database_datcollversion, RelationGetDescr(rel), &isnull);
 	oldversion = isnull ? NULL : TextDatumGetCString(datum);
@@ -2552,6 +2560,7 @@ AlterDatabaseRefreshColl(AlterDatabaseRefreshCollStmt *stmt)
 		bool		nulls[Natts_pg_database] = {0};
 		bool		replaces[Natts_pg_database] = {0};
 		Datum		values[Natts_pg_database] = {0};
+		HeapTuple	newtuple;
 
 		ereport(NOTICE,
 				(errmsg("changing version from %s to %s",
@@ -2560,14 +2569,15 @@ AlterDatabaseRefreshColl(AlterDatabaseRefreshCollStmt *stmt)
 		values[Anum_pg_database_datcollversion - 1] = CStringGetTextDatum(newversion);
 		replaces[Anum_pg_database_datcollversion - 1] = true;
 
-		tuple = heap_modify_tuple(tuple, RelationGetDescr(rel),
-								  values, nulls, replaces);
-		CatalogTupleUpdate(rel, &tuple->t_self, tuple);
-		heap_freetuple(tuple);
+		newtuple = heap_modify_tuple(tuple, RelationGetDescr(rel),
+									 values, nulls, replaces);
+		CatalogTupleUpdate(rel, &tuple->t_self, newtuple);
+		heap_freetuple(newtuple);
 	}
 	else
 		ereport(NOTICE,
 				(errmsg("version has not changed")));
+	UnlockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(DatabaseRelationId, db_id, 0);
 
@@ -2679,6 +2689,8 @@ AlterDatabaseOwner(const char *dbname, Oid newOwnerId)
 					(errcode(ERRCODE_INSUFFICIENT_PRIVILEGE),
 					 errmsg("permission denied to change owner of database")));
 
+		LockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
+
 		repl_repl[Anum_pg_database_datdba - 1] = true;
 		repl_val[Anum_pg_database_datdba - 1] = ObjectIdGetDatum(newOwnerId);
 
@@ -2700,6 +2712,7 @@ AlterDatabaseOwner(const char *dbname, Oid newOwnerId)
 
 		newtuple = heap_modify_tuple(tuple, RelationGetDescr(rel), repl_val, repl_null, repl_repl);
 		CatalogTupleUpdate(rel, &newtuple->t_self, newtuple);
+		UnlockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 		heap_freetuple(newtuple);
 
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 22d0ce7..36d82bd 100644
--- a/src/backend/commands/event_trigger.c
+++ b/src/backend/commands/event_trigger.c
@@ -388,6 +388,7 @@ SetDatabaseHasLoginEventTriggers(void)
 	/* Set dathasloginevt flag in pg_database */
 	Form_pg_database db;
 	Relation	pg_db = table_open(DatabaseRelationId, RowExclusiveLock);
+	ItemPointerData otid;
 	HeapTuple	tuple;
 
 	/*
@@ -399,16 +400,18 @@ SetDatabaseHasLoginEventTriggers(void)
 	 */
 	LockSharedObject(DatabaseRelationId, MyDatabaseId, 0, AccessExclusiveLock);
 
-	tuple = SearchSysCacheCopy1(DATABASEOID, ObjectIdGetDatum(MyDatabaseId));
+	tuple = SearchSysCacheLockedCopy1(DATABASEOID, ObjectIdGetDatum(MyDatabaseId));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "cache lookup failed for database %u", MyDatabaseId);
+	otid = tuple->t_self;
 	db = (Form_pg_database) GETSTRUCT(tuple);
 	if (!db->dathasloginevt)
 	{
 		db->dathasloginevt = true;
-		CatalogTupleUpdate(pg_db, &tuple->t_self, tuple);
+		CatalogTupleUpdate(pg_db, &otid, tuple);
 		CommandCounterIncrement();
 	}
+	UnlockTuple(pg_db, &otid, InplaceUpdateTupleLock);
 	table_close(pg_db, RowExclusiveLock);
 	heap_freetuple(tuple);
 }
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 2caab88..8d04ca0 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -4409,14 +4409,17 @@ update_relispartition(Oid relationId, bool newval)
 {
 	HeapTuple	tup;
 	Relation	classRel;
+	ItemPointerData otid;
 
 	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relationId));
+	tup = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(relationId));
 	if (!HeapTupleIsValid(tup))
 		elog(ERROR, "cache lookup failed for relation %u", relationId);
+	otid = tup->t_self;
 	Assert(((Form_pg_class) GETSTRUCT(tup))->relispartition != newval);
 	((Form_pg_class) GETSTRUCT(tup))->relispartition = newval;
-	CatalogTupleUpdate(classRel, &tup->t_self, tup);
+	CatalogTupleUpdate(classRel, &otid, tup);
+	UnlockTuple(classRel, &otid, InplaceUpdateTupleLock);
 	heap_freetuple(tup);
 	table_close(classRel, RowExclusiveLock);
 }
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 8fcb188..7fa80a5 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3609,6 +3609,7 @@ SetRelationTableSpace(Relation rel,
 {
 	Relation	pg_class;
 	HeapTuple	tuple;
+	ItemPointerData otid;
 	Form_pg_class rd_rel;
 	Oid			reloid = RelationGetRelid(rel);
 
@@ -3617,9 +3618,10 @@ SetRelationTableSpace(Relation rel,
 	/* Get a modifiable copy of the relation's pg_class row. */
 	pg_class = table_open(RelationRelationId, RowExclusiveLock);
 
-	tuple = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(reloid));
+	tuple = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(reloid));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "cache lookup failed for relation %u", reloid);
+	otid = tuple->t_self;
 	rd_rel = (Form_pg_class) GETSTRUCT(tuple);
 
 	/* Update the pg_class row. */
@@ -3627,7 +3629,8 @@ SetRelationTableSpace(Relation rel,
 		InvalidOid : newTableSpaceId;
 	if (RelFileNumberIsValid(newRelFilenumber))
 		rd_rel->relfilenode = newRelFilenumber;
-	CatalogTupleUpdate(pg_class, &tuple->t_self, tuple);
+	CatalogTupleUpdate(pg_class, &otid, tuple);
+	UnlockTuple(pg_class, &otid, InplaceUpdateTupleLock);
 
 	/*
 	 * Record dependency on tablespace.  This is only required for relations
@@ -4121,6 +4124,7 @@ RenameRelationInternal(Oid myrelid, const char *newrelname, bool is_internal, bo
 {
 	Relation	targetrelation;
 	Relation	relrelation;	/* for RELATION relation */
+	ItemPointerData otid;
 	HeapTuple	reltup;
 	Form_pg_class relform;
 	Oid			namespaceId;
@@ -4143,7 +4147,8 @@ RenameRelationInternal(Oid myrelid, const char *newrelname, bool is_internal, bo
 	 */
 	relrelation = table_open(RelationRelationId, RowExclusiveLock);
 
-	reltup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(myrelid));
+	reltup = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(myrelid));
+	otid = reltup->t_self;
 	if (!HeapTupleIsValid(reltup))	/* shouldn't happen */
 		elog(ERROR, "cache lookup failed for relation %u", myrelid);
 	relform = (Form_pg_class) GETSTRUCT(reltup);
@@ -4170,7 +4175,8 @@ RenameRelationInternal(Oid myrelid, const char *newrelname, bool is_internal, bo
 	 */
 	namestrcpy(&(relform->relname), newrelname);
 
-	CatalogTupleUpdate(relrelation, &reltup->t_self, reltup);
+	CatalogTupleUpdate(relrelation, &otid, reltup);
+	UnlockTuple(relrelation, &otid, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHookArg(RelationRelationId, myrelid, 0,
 								 InvalidOid, is_internal);
@@ -14917,7 +14923,7 @@ ATExecSetRelOptions(Relation rel, List *defList, AlterTableType operation,
 
 	/* Fetch heap tuple */
 	relid = RelationGetRelid(rel);
-	tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
+	tuple = SearchSysCacheLocked1(RELOID, ObjectIdGetDatum(relid));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "cache lookup failed for relation %u", relid);
 
@@ -15021,6 +15027,7 @@ ATExecSetRelOptions(Relation rel, List *defList, AlterTableType operation,
 								 repl_val, repl_null, repl_repl);
 
 	CatalogTupleUpdate(pgclass, &newtuple->t_self, newtuple);
+	UnlockTuple(pgclass, &tuple->t_self, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(RelationRelationId, RelationGetRelid(rel), 0);
 
@@ -17170,7 +17177,8 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 	ObjectAddress thisobj;
 	bool		already_done = false;
 
-	classTup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relOid));
+	/* no rel lock for relkind=c so use LOCKTAG_TUPLE */
+	classTup = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(relOid));
 	if (!HeapTupleIsValid(classTup))
 		elog(ERROR, "cache lookup failed for relation %u", relOid);
 	classForm = (Form_pg_class) GETSTRUCT(classTup);
@@ -17189,6 +17197,8 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 	already_done = object_address_present(&thisobj, objsMoved);
 	if (!already_done && oldNspOid != newNspOid)
 	{
+		ItemPointerData otid = classTup->t_self;
+
 		/* check for duplicate name (more friendly than unique-index failure) */
 		if (get_relname_relid(NameStr(classForm->relname),
 							  newNspOid) != InvalidOid)
@@ -17201,7 +17211,9 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 		/* classTup is a copy, so OK to scribble on */
 		classForm->relnamespace = newNspOid;
 
-		CatalogTupleUpdate(classRel, &classTup->t_self, classTup);
+		CatalogTupleUpdate(classRel, &otid, classTup);
+		UnlockTuple(classRel, &otid, InplaceUpdateTupleLock);
+
 
 		/* Update dependency on schema if caller said so */
 		if (hasDependEntry &&
@@ -17213,6 +17225,8 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 			elog(ERROR, "could not change schema dependency for relation \"%s\"",
 				 NameStr(classForm->relname));
 	}
+	else
+		UnlockTuple(classRel, &classTup->t_self, InplaceUpdateTupleLock);
 	if (!already_done)
 	{
 		add_exact_object_address(&thisobj, objsMoved);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 4d7c92d..321ad47 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1209,6 +1209,8 @@ InitResultRelInfo(ResultRelInfo *resultRelInfo,
 	resultRelInfo->ri_NumIndices = 0;
 	resultRelInfo->ri_IndexRelationDescs = NULL;
 	resultRelInfo->ri_IndexRelationInfo = NULL;
+	resultRelInfo->ri_needLockTagTuple =
+		IsInplaceUpdateRelation(resultRelationDesc);
 	/* make a copy so as not to depend on relcache info not changing... */
 	resultRelInfo->ri_TrigDesc = CopyTriggerDesc(resultRelationDesc->trigdesc);
 	if (resultRelInfo->ri_TrigDesc)
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index d0a89cd..f18efdb 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -559,8 +559,12 @@ ExecSimpleRelationUpdate(ResultRelInfo *resultRelInfo,
 	Relation	rel = resultRelInfo->ri_RelationDesc;
 	ItemPointer tid = &(searchslot->tts_tid);
 
-	/* For now we support only tables. */
+	/*
+	 * We support only non-system tables, with
+	 * check_publication_add_relation() accountable.
+	 */
 	Assert(rel->rd_rel->relkind == RELKIND_RELATION);
+	Assert(!IsCatalogRelation(rel));
 
 	CheckCmdReplicaIdentity(rel, CMD_UPDATE);
 
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index a2442b7..b70d2f6 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -2320,6 +2320,8 @@ ExecUpdate(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 	}
 	else
 	{
+		ItemPointerData lockedtid;
+
 		/*
 		 * If we generate a new candidate tuple after EvalPlanQual testing, we
 		 * must loop back here to try again.  (We don't need to redo triggers,
@@ -2328,6 +2330,7 @@ ExecUpdate(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 		 * to do them again.)
 		 */
 redo_act:
+		lockedtid = *tupleid;
 		result = ExecUpdateAct(context, resultRelInfo, tupleid, oldtuple, slot,
 							   canSetTag, &updateCxt);
 
@@ -2421,6 +2424,14 @@ redo_act:
 								ExecInitUpdateProjection(context->mtstate,
 														 resultRelInfo);
 
+							if (resultRelInfo->ri_needLockTagTuple)
+							{
+								UnlockTuple(resultRelationDesc,
+											&lockedtid, InplaceUpdateTupleLock);
+								LockTuple(resultRelationDesc,
+										  tupleid, InplaceUpdateTupleLock);
+							}
+
 							/* Fetch the most recent version of old tuple. */
 							oldSlot = resultRelInfo->ri_oldTupleSlot;
 							if (!table_tuple_fetch_row_version(resultRelationDesc,
@@ -2525,6 +2536,14 @@ ExecOnConflictUpdate(ModifyTableContext *context,
 	TransactionId xmin;
 	bool		isnull;
 
+	/*
+	 * Parse analysis should have blocked ON CONFLICT for all system
+	 * relations, which includes these.  There's no fundamental obstacle to
+	 * supporting this; we'd just need to handle LOCKTAG_TUPLE like the other
+	 * ExecUpdate() caller.
+	 */
+	Assert(!resultRelInfo->ri_needLockTagTuple);
+
 	/* Determine lock mode to use */
 	lockmode = ExecUpdateLockMode(context->estate, resultRelInfo);
 
@@ -2850,6 +2869,7 @@ ExecMergeMatched(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 {
 	ModifyTableState *mtstate = context->mtstate;
 	List	  **mergeActions = resultRelInfo->ri_MergeActions;
+	ItemPointerData lockedtid;
 	List	   *actionStates;
 	TupleTableSlot *newslot = NULL;
 	TupleTableSlot *rslot = NULL;
@@ -2886,17 +2906,33 @@ ExecMergeMatched(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 	 * target wholerow junk attr.
 	 */
 	Assert(tupleid != NULL || oldtuple != NULL);
+	ItemPointerSetInvalid(&lockedtid);
 	if (oldtuple != NULL)
 	{
 		Assert(resultRelInfo->ri_TrigDesc);
+		Assert(!resultRelInfo->ri_needLockTagTuple);
 		ExecForceStoreHeapTuple(oldtuple, resultRelInfo->ri_oldTupleSlot,
 								false);
 	}
-	else if (!table_tuple_fetch_row_version(resultRelInfo->ri_RelationDesc,
-											tupleid,
-											SnapshotAny,
-											resultRelInfo->ri_oldTupleSlot))
-		elog(ERROR, "failed to fetch the target tuple");
+	else
+	{
+		if (resultRelInfo->ri_needLockTagTuple)
+		{
+			/*
+			 * This locks even tuples that don't match mas_whenqual, which
+			 * isn't ideal.  MERGE on system catalogs is a minor use case, so
+			 * don't bother doing better.
+			 */
+			LockTuple(resultRelInfo->ri_RelationDesc, tupleid,
+					  InplaceUpdateTupleLock);
+			lockedtid = *tupleid;
+		}
+		if (!table_tuple_fetch_row_version(resultRelInfo->ri_RelationDesc,
+										   tupleid,
+										   SnapshotAny,
+										   resultRelInfo->ri_oldTupleSlot))
+			elog(ERROR, "failed to fetch the target tuple");
+	}
 
 	/*
 	 * Test the join condition.  If it's satisfied, perform a MATCHED action.
@@ -2968,7 +3004,7 @@ lmerge_matched:
 										tupleid, NULL, newslot, &result))
 				{
 					if (result == TM_Ok)
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 
 					break;		/* concurrent update/delete */
 				}
@@ -2979,7 +3015,7 @@ lmerge_matched:
 				{
 					if (!ExecIRUpdateTriggers(estate, resultRelInfo,
 											  oldtuple, newslot))
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 				}
 				else
 				{
@@ -2999,7 +3035,8 @@ lmerge_matched:
 					if (updateCxt.crossPartUpdate)
 					{
 						mtstate->mt_merge_updated += 1;
-						return context->cpUpdateReturningSlot;
+						rslot = context->cpUpdateReturningSlot;
+						goto out;
 					}
 				}
 
@@ -3017,7 +3054,7 @@ lmerge_matched:
 										NULL, NULL, &result))
 				{
 					if (result == TM_Ok)
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 
 					break;		/* concurrent update/delete */
 				}
@@ -3028,7 +3065,7 @@ lmerge_matched:
 				{
 					if (!ExecIRDeleteTriggers(estate, resultRelInfo,
 											  oldtuple))
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 				}
 				else
 					result = ExecDeleteAct(context, resultRelInfo, tupleid,
@@ -3109,7 +3146,7 @@ lmerge_matched:
 				 * let caller handle it under NOT MATCHED [BY TARGET] clauses.
 				 */
 				*matched = false;
-				return NULL;
+				goto out;
 
 			case TM_Updated:
 				{
@@ -3183,7 +3220,7 @@ lmerge_matched:
 								 * more to do.
 								 */
 								if (TupIsNull(epqslot))
-									return NULL;
+									goto out;
 
 								/*
 								 * If we got a NULL ctid from the subplan, the
@@ -3201,6 +3238,15 @@ lmerge_matched:
 								 * we need to switch to the NOT MATCHED BY
 								 * SOURCE case.
 								 */
+								if (resultRelInfo->ri_needLockTagTuple)
+								{
+									if (ItemPointerIsValid(&lockedtid))
+										UnlockTuple(resultRelInfo->ri_RelationDesc, &lockedtid,
+													InplaceUpdateTupleLock);
+									LockTuple(resultRelInfo->ri_RelationDesc, &context->tmfd.ctid,
+											  InplaceUpdateTupleLock);
+									lockedtid = context->tmfd.ctid;
+								}
 								if (!table_tuple_fetch_row_version(resultRelationDesc,
 																   &context->tmfd.ctid,
 																   SnapshotAny,
@@ -3229,7 +3275,7 @@ lmerge_matched:
 							 * MATCHED [BY TARGET] actions
 							 */
 							*matched = false;
-							return NULL;
+							goto out;
 
 						case TM_SelfModified:
 
@@ -3257,13 +3303,13 @@ lmerge_matched:
 
 							/* This shouldn't happen */
 							elog(ERROR, "attempted to update or delete invisible tuple");
-							return NULL;
+							goto out;
 
 						default:
 							/* see table_tuple_lock call in ExecDelete() */
 							elog(ERROR, "unexpected table_tuple_lock status: %u",
 								 result);
-							return NULL;
+							goto out;
 					}
 				}
 
@@ -3310,6 +3356,10 @@ lmerge_matched:
 	/*
 	 * Successfully executed an action or no qualifying action was found.
 	 */
+out:
+	if (ItemPointerIsValid(&lockedtid))
+		UnlockTuple(resultRelInfo->ri_RelationDesc, &lockedtid,
+					InplaceUpdateTupleLock);
 	return rslot;
 }
 
@@ -3761,6 +3811,7 @@ ExecModifyTable(PlanState *pstate)
 	HeapTupleData oldtupdata;
 	HeapTuple	oldtuple;
 	ItemPointer tupleid;
+	bool		tuplock;
 
 	CHECK_FOR_INTERRUPTS();
 
@@ -4073,6 +4124,8 @@ ExecModifyTable(PlanState *pstate)
 				break;
 
 			case CMD_UPDATE:
+				tuplock = false;
+
 				/* Initialize projection info if first time for this table */
 				if (unlikely(!resultRelInfo->ri_projectNewInfoValid))
 					ExecInitUpdateProjection(node, resultRelInfo);
@@ -4084,6 +4137,7 @@ ExecModifyTable(PlanState *pstate)
 				oldSlot = resultRelInfo->ri_oldTupleSlot;
 				if (oldtuple != NULL)
 				{
+					Assert(!resultRelInfo->ri_needLockTagTuple);
 					/* Use the wholerow junk attr as the old tuple. */
 					ExecForceStoreHeapTuple(oldtuple, oldSlot, false);
 				}
@@ -4092,6 +4146,11 @@ ExecModifyTable(PlanState *pstate)
 					/* Fetch the most recent version of old tuple. */
 					Relation	relation = resultRelInfo->ri_RelationDesc;
 
+					if (resultRelInfo->ri_needLockTagTuple)
+					{
+						LockTuple(relation, tupleid, InplaceUpdateTupleLock);
+						tuplock = true;
+					}
 					if (!table_tuple_fetch_row_version(relation, tupleid,
 													   SnapshotAny,
 													   oldSlot))
@@ -4103,6 +4162,9 @@ ExecModifyTable(PlanState *pstate)
 				/* Now apply the update. */
 				slot = ExecUpdate(&context, resultRelInfo, tupleid, oldtuple,
 								  slot, node->canSetTag);
+				if (tuplock)
+					UnlockTuple(resultRelInfo->ri_RelationDesc, tupleid,
+								InplaceUpdateTupleLock);
 				break;
 
 			case CMD_DELETE:
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 930cc03..3f1e8ce 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -3770,6 +3770,7 @@ RelationSetNewRelfilenumber(Relation relation, char persistence)
 {
 	RelFileNumber newrelfilenumber;
 	Relation	pg_class;
+	ItemPointerData otid;
 	HeapTuple	tuple;
 	Form_pg_class classform;
 	MultiXactId minmulti = InvalidMultiXactId;
@@ -3812,11 +3813,12 @@ RelationSetNewRelfilenumber(Relation relation, char persistence)
 	 */
 	pg_class = table_open(RelationRelationId, RowExclusiveLock);
 
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(relation)));
+	tuple = SearchSysCacheLockedCopy1(RELOID,
+									  ObjectIdGetDatum(RelationGetRelid(relation)));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "could not find tuple for relation %u",
 			 RelationGetRelid(relation));
+	otid = tuple->t_self;
 	classform = (Form_pg_class) GETSTRUCT(tuple);
 
 	/*
@@ -3936,9 +3938,10 @@ RelationSetNewRelfilenumber(Relation relation, char persistence)
 		classform->relminmxid = minmulti;
 		classform->relpersistence = persistence;
 
-		CatalogTupleUpdate(pg_class, &tuple->t_self, tuple);
+		CatalogTupleUpdate(pg_class, &otid, tuple);
 	}
 
+	UnlockTuple(pg_class, &otid, InplaceUpdateTupleLock);
 	heap_freetuple(tuple);
 
 	table_close(pg_class, RowExclusiveLock);
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index 3e03dfc..50c9440 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -30,7 +30,10 @@
 #include "catalog/pg_shseclabel_d.h"
 #include "common/int.h"
 #include "lib/qunique.h"
+#include "miscadmin.h"
+#include "storage/lmgr.h"
 #include "utils/catcache.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/rel.h"
 #include "utils/syscache.h"
@@ -269,6 +272,98 @@ ReleaseSysCache(HeapTuple tuple)
 }
 
 /*
+ * SearchSysCacheLocked1
+ *
+ * Combine SearchSysCache1() with acquiring a LOCKTAG_TUPLE at mode
+ * InplaceUpdateTupleLock.  This is a tool for complying with the
+ * README.tuplock section "Locking to write inplace-updated tables".  After
+ * the caller's heap_update(), it should UnlockTuple(InplaceUpdateTupleLock)
+ * and ReleaseSysCache().
+ *
+ * The returned tuple may be the subject of an uncommitted update, so this
+ * doesn't prevent the "tuple concurrently updated" error.
+ */
+HeapTuple
+SearchSysCacheLocked1(int cacheId,
+					  Datum key1)
+{
+	ItemPointerData tid;
+	LOCKTAG		tag;
+	Oid			dboid =
+		SysCache[cacheId]->cc_relisshared ? InvalidOid : MyDatabaseId;
+	Oid			reloid = cacheinfo[cacheId].reloid;
+
+	/*----------
+	 * Since inplace updates may happen just before our LockTuple(), we must
+	 * return content acquired after LockTuple() of the TID we return.  If we
+	 * just fetched twice instead of looping, the following sequence would
+	 * defeat our locking:
+	 *
+	 * GRANT:   SearchSysCache1() = TID (1,5)
+	 * GRANT:   LockTuple(pg_class, (1,5))
+	 * [no more inplace update of (1,5) until we release the lock]
+	 * CLUSTER: SearchSysCache1() = TID (1,5)
+	 * CLUSTER: heap_update() = TID (1,8)
+	 * CLUSTER: COMMIT
+	 * GRANT:   SearchSysCache1() = TID (1,8)
+	 * GRANT:   return (1,8) from SearchSysCacheLocked1()
+	 * VACUUM:  SearchSysCache1() = TID (1,8)
+	 * VACUUM:  LockTuple(pg_class, (1,8))  # two TIDs now locked for one rel
+	 * VACUUM:  inplace update
+	 * GRANT:   heap_update() = (1,9)  # lose inplace update
+	 *
+	 * In the happy case, this takes two fetches, one to determine the TID to
+	 * lock and another to get the content and confirm the TID didn't change.
+	 *
+	 * This is valid even if the row gets updated to a new TID, the old TID
+	 * becomes LP_UNUSED, and the row gets updated back to its old TID.  We'd
+	 * still hold the right LOCKTAG_TUPLE and a copy of the row captured after
+	 * the LOCKTAG_TUPLE.
+	 */
+	ItemPointerSetInvalid(&tid);
+	for (;;)
+	{
+		HeapTuple	tuple;
+		LOCKMODE	lockmode = InplaceUpdateTupleLock;
+
+		tuple = SearchSysCache1(cacheId, key1);
+		if (ItemPointerIsValid(&tid))
+		{
+			if (!HeapTupleIsValid(tuple))
+			{
+				LockRelease(&tag, lockmode, false);
+				return tuple;
+			}
+			if (ItemPointerEquals(&tid, &tuple->t_self))
+				return tuple;
+			LockRelease(&tag, lockmode, false);
+		}
+		else if (!HeapTupleIsValid(tuple))
+			return tuple;
+
+		tid = tuple->t_self;
+		ReleaseSysCache(tuple);
+		/* like: LockTuple(rel, &tid, lockmode) */
+		SET_LOCKTAG_TUPLE(tag, dboid, reloid,
+						  ItemPointerGetBlockNumber(&tid),
+						  ItemPointerGetOffsetNumber(&tid));
+		(void) LockAcquire(&tag, lockmode, false, false);
+
+		/*
+		 * 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.
+		 *
+		 * If a heap_update() call just released its LOCKTAG_TUPLE, we'll
+		 * probably find the old tuple and reach "tuple concurrently updated".
+		 * If that heap_update() aborts, our LOCKTAG_TUPLE blocks inplace
+		 * updates while our caller works.
+		 */
+		AcceptInvalidationMessages();
+	}
+}
+
+/*
  * SearchSysCacheCopy
  *
  * A convenience routine that does SearchSysCache and (if successful)
@@ -295,6 +390,28 @@ SearchSysCacheCopy(int cacheId,
 }
 
 /*
+ * SearchSysCacheLockedCopy1
+ *
+ * Meld SearchSysCacheLockedCopy1 with SearchSysCacheCopy().  After the
+ * caller's heap_update(), it should UnlockTuple(InplaceUpdateTupleLock) and
+ * heap_freetuple().
+ */
+HeapTuple
+SearchSysCacheLockedCopy1(int cacheId,
+						  Datum key1)
+{
+	HeapTuple	tuple,
+				newtuple;
+
+	tuple = SearchSysCacheLocked1(cacheId, key1);
+	if (!HeapTupleIsValid(tuple))
+		return tuple;
+	newtuple = heap_copytuple(tuple);
+	ReleaseSysCache(tuple);
+	return newtuple;
+}
+
+/*
  * SearchSysCacheExists
  *
  * A convenience routine that just probes to see if a tuple can be found.
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index b62c96f..eab0add 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -482,6 +482,9 @@ typedef struct ResultRelInfo
 	/* Have the projection and the slots above been initialized? */
 	bool		ri_projectNewInfoValid;
 
+	/* updates do LockTuple() before oldtup read; see README.tuplock */
+	bool		ri_needLockTagTuple;
+
 	/* triggers to be fired, if any */
 	TriggerDesc *ri_TrigDesc;
 
diff --git a/src/include/storage/lockdefs.h b/src/include/storage/lockdefs.h
index 934ba84..810b297 100644
--- a/src/include/storage/lockdefs.h
+++ b/src/include/storage/lockdefs.h
@@ -47,6 +47,8 @@ typedef int LOCKMODE;
 
 #define MaxLockMode				8	/* highest standard lock mode */
 
+/* See README.tuplock section "Locking to write inplace-updated tables" */
+#define InplaceUpdateTupleLock ExclusiveLock
 
 /* WAL representation of an AccessExclusiveLock on a table */
 typedef struct xl_standby_lock
diff --git a/src/include/utils/syscache.h b/src/include/utils/syscache.h
index 03a27dd..b541911 100644
--- a/src/include/utils/syscache.h
+++ b/src/include/utils/syscache.h
@@ -43,9 +43,14 @@ extern HeapTuple SearchSysCache4(int cacheId,
 
 extern void ReleaseSysCache(HeapTuple tuple);
 
+extern HeapTuple SearchSysCacheLocked1(int cacheId,
+									   Datum key1);
+
 /* convenience routines */
 extern HeapTuple SearchSysCacheCopy(int cacheId,
 									Datum key1, Datum key2, Datum key3, Datum key4);
+extern HeapTuple SearchSysCacheLockedCopy1(int cacheId,
+										   Datum key1);
 extern bool SearchSysCacheExists(int cacheId,
 								 Datum key1, Datum key2, Datum key3, Datum key4);
 extern Oid	GetSysCacheOid(int cacheId, AttrNumber oidcol,
diff --git a/src/test/isolation/expected/intra-grant-inplace.out b/src/test/isolation/expected/intra-grant-inplace.out
index c2a9841..b5fe8b0 100644
--- a/src/test/isolation/expected/intra-grant-inplace.out
+++ b/src/test/isolation/expected/intra-grant-inplace.out
@@ -154,9 +154,11 @@ step b1: BEGIN;
 step grant1: 
 	GRANT SELECT ON intra_grant_inplace TO PUBLIC;
  <waiting ...>
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
-step c2: COMMIT;
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
+step addk2: <... completed>
+ERROR:  deadlock detected
 step grant1: <... completed>
+step c2: COMMIT;
 step c1: COMMIT;
 step read2: 
 	SELECT relhasindex FROM pg_class
@@ -194,9 +196,8 @@ relhasindex
 f          
 (1 row)
 
-s4: WARNING:  got: tuple concurrently updated
-step revoke4: <... completed>
 step r3: ROLLBACK;
+step revoke4: <... completed>
 
 starting permutation: b1 drop1 b3 sfu3 revoke4 c1 r3
 step b1: BEGIN;
@@ -223,6 +224,6 @@ relhasindex
 -----------
 (0 rows)
 
-s4: WARNING:  got: tuple concurrently deleted
+s4: WARNING:  got: cache lookup failed for relation REDACTED
 step revoke4: <... completed>
 step r3: ROLLBACK;
diff --git a/src/test/isolation/specs/eval-plan-qual.spec b/src/test/isolation/specs/eval-plan-qual.spec
index 3a74406..07307e6 100644
--- a/src/test/isolation/specs/eval-plan-qual.spec
+++ b/src/test/isolation/specs/eval-plan-qual.spec
@@ -194,7 +194,7 @@ step simplepartupdate_noroute {
 	update parttbl set b = 2 where c = 1 returning *;
 }
 
-# test system class updates
+# test system class LockTuple()
 
 step sys1	{
 	UPDATE pg_class SET reltuples = 123 WHERE oid = 'accounts'::regclass;
diff --git a/src/test/isolation/specs/intra-grant-inplace.spec b/src/test/isolation/specs/intra-grant-inplace.spec
index eed0b52..2992c85 100644
--- a/src/test/isolation/specs/intra-grant-inplace.spec
+++ b/src/test/isolation/specs/intra-grant-inplace.spec
@@ -14,6 +14,7 @@ teardown
 
 # heap_update()
 session s1
+setup	{ SET deadlock_timeout = '100s'; }
 step b1	{ BEGIN; }
 step grant1	{
 	GRANT SELECT ON intra_grant_inplace TO PUBLIC;
@@ -25,6 +26,7 @@ step c1	{ COMMIT; }
 
 # inplace update
 session s2
+setup	{ SET deadlock_timeout = '10ms'; }
 step read2	{
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass;
@@ -73,8 +75,6 @@ step keyshr5	{
 teardown	{ ROLLBACK; }
 
 
-# XXX extant bugs: permutation comments refer to planned future LockTuple()
-
 permutation
 	b1
 	grant1
@@ -126,8 +126,8 @@ permutation
 	b2
 	sfnku2
 	b1
-	grant1(c2)		# acquire LockTuple(), await sfnku2 xmax
-	addk2			# block in LockTuple() behind grant1 = deadlock
+	grant1(addk2)	# acquire LockTuple(), await sfnku2 xmax
+	addk2(*)		# block in LockTuple() behind grant1 = deadlock
 	c2
 	c1
 	read2
@@ -138,7 +138,7 @@ permutation
 	grant1
 	b3
 	sfu3(c1)	# acquire LockTuple(), await grant1 xmax
-	revoke4(sfu3)	# block in LockTuple() behind sfu3
+	revoke4(r3)	# block in LockTuple() behind sfu3
 	c1
 	r3			# revoke4 unlocks old tuple and finds new
 
inplace160-inval-durability-inplace-v3.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Core code no longer needs XLOG_INVALIDATIONS, but this leaves removing
    it for future work.  Extensions could be relying on that mechanism, so
    that removal would not be back-patch material.  Back-patch to v12 (all
    supported versions).
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 797bddf..d7e417f 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6305,6 +6305,9 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 	Relation	relation = scan->heap_rel;
 	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;
@@ -6312,6 +6315,29 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 	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();
 
@@ -6341,9 +6367,16 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 		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);
@@ -6355,22 +6388,23 @@ heap_inplace_update_finish(void *state, HeapTuple tuple)
 		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
-	END_CRIT_SECTION();
-
 	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);
 	systable_endscan(scan);
 
-	/*
-	 * 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.
-	 */
-	if (!IsBootstrapProcessingMode())
-		CacheInvalidateHeapTuple(relation, tuple, NULL);
+	AcceptInvalidationMessages();	/* local processing of just-sent inval */
 }
 
 /*
@@ -10268,6 +10302,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 5f5673e..f31cc3a 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 25f870b..32e509a 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 9bda1aa..30285bd 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1358,14 +1358,23 @@ 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 formerly did so, they now send
+		 * immediate invalidations.  Extensions might still do so, 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 b4b68b1..ba576c6 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2884,19 +2884,21 @@ index_update_stats(Relation rel,
 	}
 
 	/*
-	 * If anything changed, write out the tuple
+	 * If anything changed, write out the tuple and immediate invals
 	 */
 	if (dirty)
-	{
 		heap_inplace_update_finish(state, tuple);
-		/* the above sends a cache inval message */
-	}
 	else
-	{
 		heap_inplace_update_cancel(state);
-		/* no need to change tuple, but force relcache inval anyway */
-		CacheInvalidateRelcacheByTuple(tuple);
-	}
+
+	/*
+	 * Queue a transactional relcache inval.  CREATE INDEX needs an immediate
+	 * inval for the relhasindex change, but it also needs a transactional
+	 * inval for when the new index's rows become visible.  Other CREATE INDEX
+	 * and REINDEX code happens to also queue a transactional inval, but keep
+	 * this in case rare callers rely on this part of our API contract.
+	 */
+	CacheInvalidateRelcacheByTuple(tuple);
 
 	heap_freetuple(tuple);
 
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 36d82bd..5d4173a 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.
 				 */
 				heap_inplace_update_finish(state, tuple);
 			}
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 8ec5adf..b2cf14e 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -508,23 +508,18 @@ 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.
+			 * Older WAL may contain commit-time invals for inplace updates.
+			 * 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 111d8a2..ea8ca0e 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2288,7 +2288,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;
@@ -2329,7 +2330,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)
 		{
@@ -2338,7 +2339,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 603aa41..b3d3adb 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;
 
-	/* Events emitted by previous commands of this (sub)transaction */
-	InvalidationMsgsGroup PriorCmdInvalidMsgs;
-
 	/* 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;
+
+	/* 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,
@@ -1038,16 +1139,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
@@ -1058,6 +1159,45 @@ AtEOXact_Inval(bool isCommit)
 
 	/* Need not free anything explicitly */
 	transInvalInfo = NULL;
+	inplaceInvalInfo = NULL;
+}
+
+/*
+ * 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;
 }
 
 /*
@@ -1125,18 +1265,21 @@ 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;
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Successful inplace update must clear this. */
+		Assert(inplaceInvalInfo == NULL);
 	}
 	else
 	{
@@ -1148,6 +1291,9 @@ AtEOSubXact_Inval(bool isCommit)
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Reset from aborted inplace update. */
+		inplaceInvalInfo = NULL;
 	}
 }
 
@@ -1177,7 +1323,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 +1331,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 +1369,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 +1379,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 +1457,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 +1513,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1362,15 +1537,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 +1557,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1399,14 +1572,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 +1592,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 +1781,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 50c9440..f41b1c2 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".
diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 42736f3..4591e9a 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -20,6 +20,7 @@
 #include "storage/buf.h"
 #include "storage/bufpage.h"
 #include "storage/relfilelocator.h"
+#include "storage/sinval.h"
 #include "utils/relcache.h"
 
 
@@ -425,9 +426,14 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
+	Oid			dbId;			/* MyDatabaseId */
+	Oid			tsId;			/* MyDatabaseTableSpace */
+	bool		relcacheInitFileInval;	/* invalidate relcache init files */
+	int			nmsgs;			/* number of shared inval msgs */
+	SharedInvalidationMessage msgs[FLEXIBLE_ARRAY_MEMBER];
 } xl_heap_inplace;
 
-#define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
+#define MinSizeOfHeapInplace	(offsetof(xl_heap_inplace, nmsgs) + sizeof(int))
 
 /*
  * This is what we need to know about setting a visibility map bit
diff --git a/src/include/storage/sinval.h b/src/include/storage/sinval.h
index 8f5744b..c812237 100644
--- a/src/include/storage/sinval.h
+++ b/src/include/storage/sinval.h
@@ -144,6 +144,8 @@ extern void ProcessCatchupInterrupt(void);
 
 extern int	xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 												 bool *RelcacheInitFileInval);
+extern int	inplaceGetInvalidationMessages(SharedInvalidationMessage **msgs,
+										   bool *RelcacheInitFileInval);
 extern void ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 												 int nmsgs, bool RelcacheInitFileInval,
 												 Oid dbid, Oid tsid);
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..3390e7a 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 82b3b41..c3c5d97 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1252,6 +1252,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
#9Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#6)
Re: Inval reliability, especially for inplace updates

On Tue, Jun 18, 2024 at 08:23:49AM -0700, Noah Misch wrote:

On Mon, Jun 17, 2024 at 06:57:30PM -0700, Andres Freund wrote:

On 2024-06-17 16:58:54 -0700, Noah Misch wrote:

That inplace150 patch turned out to be unnecessary. Contrary to the
"noncritical resource releasing" comment some lines above
AtEOXact_Inval(true), the actual behavior is already to promote ERROR to
PANIC. An ERROR just before or after sending invals becomes PANIC, "cannot
abort transaction %u, it was already committed".

Relying on that, instead of explicit critical sections, seems fragile to me.
IIRC some of the behaviour around errors around transaction commit/abort has
changed a bunch of times. Tying correctness into something that could be
changed for unrelated reasons doesn't seem great.

Fair enough. It could still be a good idea for master, but given I missed a
bug in inplace150-inval-durability-atcommit-v1.patch far worse than the ones
$SUBJECT fixes, let's not risk it in back branches.

What are your thoughts on whether a change to explicit critical sections
should be master-only vs. back-patched? I have a feeling your comment pointed
to something I'm still missing, but I don't know where to look next.

#10Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#6)
4 attachment(s)
Re: Inval reliability, especially for inplace updates

On Tue, Jun 18, 2024 at 08:23:49AM -0700, Noah Misch wrote:

On Mon, Jun 17, 2024 at 06:57:30PM -0700, Andres Freund wrote:

On 2024-06-17 16:58:54 -0700, Noah Misch wrote:

On Sat, Jun 15, 2024 at 03:37:18PM -0700, Noah Misch wrote:

On Wed, May 22, 2024 at 05:05:48PM -0700, Noah Misch wrote:

/messages/by-id/20240512232923.aa.nmisch@google.com wrote:

Separable, nontrivial things not fixed in the attached patch stack:

- Inplace update uses transactional CacheInvalidateHeapTuple(). ROLLBACK of
CREATE INDEX wrongly discards the inval, leading to the relhasindex=t loss
still seen in inplace-inval.spec. CacheInvalidateRelmap() does this right.

I plan to fix that like CacheInvalidateRelmap(): send the inval immediately,
inside the critical section. Send it in heap_xlog_inplace(), too.

I'm worried this might cause its own set of bugs, e.g. if there are any places
that, possibly accidentally, rely on the invalidation from the inplace update
to also cover separate changes.

Good point. I do have index_update_stats() still doing an ideally-superfluous
relcache update for that reason. Taking that further, it would be cheap
insurance to have the inplace update do a transactional inval in addition to
its immediate inval. Future master-only work could remove the transactional
one. How about that?

Restoring the transactional inval seemed good to me, so I've rebased and
included that. This applies on top of three patches from
/messages/by-id/20240822073200.4f.nmisch@google.com. I'm attaching those
to placate cfbot, but this thread is for review of the last patch only.

Attachments:

inplace090-LOCKTAG_TUPLE-eoxact-v9.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Warn if LOCKTAG_TUPLE is held at commit, under debug_assertions.
    
    The current use always releases this locktag.  A planned use will
    continue that intent.  It will involve more areas of code, making unlock
    omissions easier.  Warn under debug_assertions, like we do for various
    resource leaks.  Back-patch to v12 (all supported versions), the plan
    for the commit of the new use.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240512232923.aa.nmisch@google.com

diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 0400a50..e5e7ab5 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -2256,6 +2256,16 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
 				locallock->numLockOwners = 0;
 		}
 
+#ifdef USE_ASSERT_CHECKING
+
+		/*
+		 * Tuple locks are currently held only for short durations within a
+		 * transaction. Check that we didn't forget to release one.
+		 */
+		if (LOCALLOCK_LOCKTAG(*locallock) == LOCKTAG_TUPLE && !allLocks)
+			elog(WARNING, "tuple lock held at commit");
+#endif
+
 		/*
 		 * If the lock or proclock pointers are NULL, this lock was taken via
 		 * the relation fast-path (and is not known to have been transferred).
inplace110-successors-v9.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Fix data loss at inplace update after heap_update().
    
    As previously-added tests demonstrated, heap_inplace_update() could
    instead update an unrelated tuple of the same catalog.  It could lose
    the update.  Losing relhasindex=t was a source of index corruption.
    Inplace-updating commands like VACUUM will now wait for heap_update()
    commands like GRANT TABLE and GRANT DATABASE.  That isn't ideal, but a
    long-running GRANT already hurts VACUUM progress more just by keeping an
    XID running.  The VACUUM will behave like a DELETE or UPDATE waiting for
    the uncommitted change.
    
    For implementation details, start at the systable_inplace_update_begin()
    header comment and README.tuplock.  Back-patch to v12 (all supported
    versions).  In back branches, retain a deprecated heap_inplace_update(),
    for extensions.
    
    Reviewed by Heikki Linnakangas and Alexander Lakhin.
    
    Discussion: https://postgr.es/m/CAMp+ueZQz3yDk7qg42hk6-9gxniYbp-=bG2mgqecErqR5gGGOA@mail.gmail.com

diff --git a/src/backend/access/heap/README.tuplock b/src/backend/access/heap/README.tuplock
index 6441e8b..ddb2def 100644
--- a/src/backend/access/heap/README.tuplock
+++ b/src/backend/access/heap/README.tuplock
@@ -153,3 +153,14 @@ The following infomask bits are applicable:
 
 We currently never set the HEAP_XMAX_COMMITTED when the HEAP_XMAX_IS_MULTI bit
 is set.
+
+Reading inplace-updated columns
+-------------------------------
+
+Inplace updates create an exception to the rule that tuple data won't change
+under a reader holding a pin.  A reader of a heap_fetch() result tuple may
+witness a torn read.  Current inplace-updated fields are aligned and are no
+wider than four bytes, and current readers don't need consistency across
+fields.  Hence, they get by with just fetching each field once.  XXX such a
+caller may also read a value that has not reached WAL; see
+systable_inplace_update_finish().
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 91b2014..24f7e62 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -63,7 +63,6 @@
 #include "storage/procarray.h"
 #include "storage/standby.h"
 #include "utils/datum.h"
-#include "utils/injection_point.h"
 #include "utils/inval.h"
 #include "utils/relcache.h"
 #include "utils/snapmgr.h"
@@ -6041,61 +6040,166 @@ heap_abort_speculative(Relation relation, ItemPointer tid)
 }
 
 /*
- * heap_inplace_update - update a tuple "in place" (ie, overwrite it)
+ * heap_inplace_lock - protect inplace update from concurrent heap_update()
  *
- * Overwriting violates both MVCC and transactional safety, so the uses
- * of this function in Postgres are extremely limited.  Nonetheless we
- * find some places to use it.
+ * Evaluate whether the tuple's state is compatible with a no-key update.
+ * Current transaction rowmarks are fine, as is KEY SHARE from any
+ * transaction.  If compatible, return true with the buffer exclusive-locked,
+ * and the caller must release that by calling heap_inplace_update(), calling
+ * heap_inplace_unlock(), or raising an error.  Otherwise, return false after
+ * blocking transactions, if any, have ended.
  *
- * The tuple cannot change size, and therefore it's reasonable to assume
- * that its null bitmap (if any) doesn't change either.  So we just
- * overwrite the data portion of the tuple without touching the null
- * bitmap or any of the header fields.
+ * Since this is intended for system catalogs and SERIALIZABLE doesn't cover
+ * DDL, this doesn't guarantee any particular predicate locking.
  *
- * tuple is an in-memory tuple structure containing the data to be written
- * over the target tuple.  Also, tuple->t_self identifies the target tuple.
+ * One could modify this to return true for tuples with delete in progress,
+ * All inplace updaters take a lock that conflicts with DROP.  If explicit
+ * "DELETE FROM pg_class" is in progress, we'll wait for it like we would an
+ * update.
  *
- * Note that the tuple updated here had better not come directly from the
- * syscache if the relation has a toast relation as this tuple could
- * include toast values that have been expanded, causing a failure here.
+ * Readers of inplace-updated fields expect changes to those fields are
+ * durable.  For example, vac_truncate_clog() reads datfrozenxid from
+ * pg_database tuples via catalog snapshots.  A future snapshot must not
+ * return a lower datfrozenxid for the same database OID (lower in the
+ * FullTransactionIdPrecedes() sense).  We achieve that since no update of a
+ * tuple can start while we hold a lock on its buffer.  In cases like
+ * BEGIN;GRANT;CREATE INDEX;COMMIT we're inplace-updating a tuple visible only
+ * to this transaction.  ROLLBACK then is one case where it's okay to lose
+ * inplace updates.  (Restoring relhasindex=false on ROLLBACK is fine, since
+ * any concurrent CREATE INDEX would have blocked, then inplace-updated the
+ * committed tuple.)
+ *
+ * In principle, we could avoid waiting by overwriting every tuple in the
+ * updated tuple chain.  Reader expectations permit updating a tuple only if
+ * it's aborted, is the tail of the chain, or we already updated the tuple
+ * referenced in its t_ctid.  Hence, we would need to overwrite the tuples in
+ * order from tail to head.  That would imply either (a) mutating all tuples
+ * in one critical section or (b) accepting a chance of partial completion.
+ * Partial completion of a relfrozenxid update would have the weird
+ * consequence that the table's next VACUUM could see the table's relfrozenxid
+ * move forward between vacuum_get_cutoffs() and finishing.
+ */
+bool
+heap_inplace_lock(Relation relation,
+				  HeapTuple oldtup_ptr, Buffer buffer)
+{
+	HeapTupleData oldtup = *oldtup_ptr; /* minimize diff vs. heap_update() */
+	TM_Result	result;
+	bool		ret;
+
+	Assert(BufferIsValid(buffer));
+
+	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
+
+	/*----------
+	 * Interpret HeapTupleSatisfiesUpdate() like heap_update() does, except:
+	 *
+	 * - wait unconditionally
+	 * - no tuple locks
+	 * - don't recheck header after wait: simpler to defer to next iteration
+	 * - don't try to continue even if the updater aborts: likewise
+	 * - no crosscheck
+	 */
+	result = HeapTupleSatisfiesUpdate(&oldtup, GetCurrentCommandId(false),
+									  buffer);
+
+	if (result == TM_Invisible)
+	{
+		/* no known way this can happen */
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg_internal("attempted to overwrite invisible tuple")));
+	}
+	else if (result == TM_SelfModified)
+	{
+		/*
+		 * CREATE INDEX might reach this if an expression is silly enough to
+		 * call e.g. SELECT ... FROM pg_class FOR SHARE.  C code of other SQL
+		 * statements might get here after a heap_update() of the same row, in
+		 * the absence of an intervening CommandCounterIncrement().
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("tuple to be updated was already modified by an operation triggered by the current command")));
+	}
+	else if (result == TM_BeingModified)
+	{
+		TransactionId xwait;
+		uint16		infomask;
+
+		xwait = HeapTupleHeaderGetRawXmax(oldtup.t_data);
+		infomask = oldtup.t_data->t_infomask;
+
+		if (infomask & HEAP_XMAX_IS_MULTI)
+		{
+			LockTupleMode lockmode = LockTupleNoKeyExclusive;
+			MultiXactStatus mxact_status = MultiXactStatusNoKeyUpdate;
+			int			remain;
+			bool		current_is_member;
+
+			if (DoesMultiXactIdConflict((MultiXactId) xwait, infomask,
+										lockmode, &current_is_member))
+			{
+				LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+				ret = false;
+				MultiXactIdWait((MultiXactId) xwait, mxact_status, infomask,
+								relation, &oldtup.t_self, XLTW_Update,
+								&remain);
+			}
+			else
+				ret = true;
+		}
+		else if (TransactionIdIsCurrentTransactionId(xwait))
+			ret = true;
+		else if (HEAP_XMAX_IS_KEYSHR_LOCKED(infomask))
+			ret = true;
+		else
+		{
+			LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+			ret = false;
+			XactLockTableWait(xwait, relation, &oldtup.t_self,
+							  XLTW_Update);
+		}
+	}
+	else
+	{
+		ret = (result == TM_Ok);
+		if (!ret)
+		{
+			LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+		}
+	}
+
+	/*
+	 * GetCatalogSnapshot() relies on invalidation messages to know when to
+	 * take a new snapshot.  COMMIT of xwait is responsible for sending the
+	 * invalidation.  We're not acquiring heavyweight locks sufficient to
+	 * block if not yet sent, so we must take a new snapshot to ensure a later
+	 * attempt has a fair chance.  While we don't need this if xwait aborted,
+	 * don't bother optimizing that.
+	 */
+	if (!ret)
+		InvalidateCatalogSnapshot();
+	return ret;
+}
+
+/*
+ * heap_inplace_update - subroutine of systable_inplace_update_finish
+ *
+ * The tuple cannot change size, and therefore its header fields and null
+ * bitmap (if any) don't change either.
  */
 void
-heap_inplace_update(Relation relation, HeapTuple tuple)
+heap_inplace_update(Relation relation,
+					HeapTuple oldtup, HeapTuple tuple,
+					Buffer buffer)
 {
-	Buffer		buffer;
-	Page		page;
-	OffsetNumber offnum;
-	ItemId		lp = NULL;
-	HeapTupleHeader htup;
+	HeapTupleHeader htup = oldtup->t_data;
 	uint32		oldlen;
 	uint32		newlen;
 
-	/*
-	 * For now, we don't allow parallel updates.  Unlike a regular update,
-	 * this should never create a combo CID, so it might be possible to relax
-	 * this restriction, but not without more thought and testing.  It's not
-	 * clear that it would be useful, anyway.
-	 */
-	if (IsInParallelMode())
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
-				 errmsg("cannot update tuples during a parallel operation")));
-
-	INJECTION_POINT("inplace-before-pin");
-	buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
-	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
-	page = (Page) BufferGetPage(buffer);
-
-	offnum = ItemPointerGetOffsetNumber(&(tuple->t_self));
-	if (PageGetMaxOffsetNumber(page) >= offnum)
-		lp = PageGetItemId(page, offnum);
-
-	if (PageGetMaxOffsetNumber(page) < offnum || !ItemIdIsNormal(lp))
-		elog(ERROR, "invalid lp");
-
-	htup = (HeapTupleHeader) PageGetItem(page, lp);
-
-	oldlen = ItemIdGetLength(lp) - htup->t_hoff;
+	Assert(ItemPointerEquals(&oldtup->t_self, &tuple->t_self));
+	oldlen = oldtup->t_len - htup->t_hoff;
 	newlen = tuple->t_len - tuple->t_data->t_hoff;
 	if (oldlen != newlen || htup->t_hoff != tuple->t_data->t_hoff)
 		elog(ERROR, "wrong tuple length");
@@ -6107,6 +6211,19 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
 		   (char *) tuple->t_data + tuple->t_data->t_hoff,
 		   newlen);
 
+	/*----------
+	 * XXX A crash here can allow datfrozenxid() to get ahead of relfrozenxid:
+	 *
+	 * ["D" is a VACUUM (ONLY_DATABASE_STATS)]
+	 * ["R" is a VACUUM tbl]
+	 * D: vac_update_datfrozenid() -> systable_beginscan(pg_class)
+	 * D: systable_getnext() returns pg_class tuple of tbl
+	 * R: memcpy() into pg_class tuple of tbl
+	 * D: raise pg_database.datfrozenxid, XLogInsert(), finish
+	 * [crash]
+	 * [recovery restores datfrozenxid w/o relfrozenxid]
+	 */
+
 	MarkBufferDirty(buffer);
 
 	/* XLOG stuff */
@@ -6127,23 +6244,35 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
 
 		recptr = XLogInsert(RM_HEAP_ID, XLOG_HEAP_INPLACE);
 
-		PageSetLSN(page, recptr);
+		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
 	END_CRIT_SECTION();
 
-	UnlockReleaseBuffer(buffer);
+	heap_inplace_unlock(relation, oldtup, buffer);
 
 	/*
 	 * 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.
 	 */
 	if (!IsBootstrapProcessingMode())
 		CacheInvalidateHeapTuple(relation, tuple, NULL);
 }
 
+/*
+ * heap_inplace_unlock - reverse of heap_inplace_lock
+ */
+void
+heap_inplace_unlock(Relation relation,
+					HeapTuple oldtup, Buffer buffer)
+{
+	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+}
+
 #define		FRM_NOOP				0x0001
 #define		FRM_INVALIDATE_XMAX		0x0002
 #define		FRM_RETURN_IS_XID		0x0004
diff --git a/src/backend/access/index/genam.c b/src/backend/access/index/genam.c
index 43c95d6..5f55e8c 100644
--- a/src/backend/access/index/genam.c
+++ b/src/backend/access/index/genam.c
@@ -20,6 +20,7 @@
 #include "postgres.h"
 
 #include "access/genam.h"
+#include "access/heapam.h"
 #include "access/relscan.h"
 #include "access/tableam.h"
 #include "access/transam.h"
@@ -29,6 +30,7 @@
 #include "storage/bufmgr.h"
 #include "storage/procarray.h"
 #include "utils/acl.h"
+#include "utils/injection_point.h"
 #include "utils/lsyscache.h"
 #include "utils/rel.h"
 #include "utils/rls.h"
@@ -747,3 +749,140 @@ systable_endscan_ordered(SysScanDesc sysscan)
 		UnregisterSnapshot(sysscan->snapshot);
 	pfree(sysscan);
 }
+
+/*
+ * systable_inplace_update_begin --- update a row "in place" (overwrite it)
+ *
+ * Overwriting violates both MVCC and transactional safety, so the uses of
+ * this function in Postgres are extremely limited.  Nonetheless we find some
+ * places to use it.  Standard flow:
+ *
+ * ... [any slow preparation not requiring oldtup] ...
+ * systable_inplace_update_begin([...], &tup, &inplace_state);
+ * if (!HeapTupleIsValid(tup))
+ *	elog(ERROR, [...]);
+ * ... [buffer is exclusive-locked; mutate "tup"] ...
+ * if (dirty)
+ *	systable_inplace_update_finish(inplace_state, tup);
+ * else
+ *	systable_inplace_update_cancel(inplace_state);
+ *
+ * The first several params duplicate the systable_beginscan() param list.
+ * "oldtupcopy" is an output parameter, assigned NULL if the key ceases to
+ * find a live tuple.  (In PROC_IN_VACUUM, that is a low-probability transient
+ * condition.)  If "oldtupcopy" gets non-NULL, you must pass output parameter
+ * "state" to systable_inplace_update_finish() or
+ * systable_inplace_update_cancel().
+ */
+void
+systable_inplace_update_begin(Relation relation,
+							  Oid indexId,
+							  bool indexOK,
+							  Snapshot snapshot,
+							  int nkeys, const ScanKeyData *key,
+							  HeapTuple *oldtupcopy,
+							  void **state)
+{
+	ScanKey		mutable_key = palloc(sizeof(ScanKeyData) * nkeys);
+	int			retries = 0;
+	SysScanDesc scan;
+	HeapTuple	oldtup;
+
+	/*
+	 * For now, we don't allow parallel updates.  Unlike a regular update,
+	 * this should never create a combo CID, so it might be possible to relax
+	 * this restriction, but not without more thought and testing.  It's not
+	 * clear that it would be useful, anyway.
+	 */
+	if (IsInParallelMode())
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+				 errmsg("cannot update tuples during a parallel operation")));
+
+	/*
+	 * Accept a snapshot argument, for symmetry, but this function advances
+	 * its snapshot as needed to reach the tail of the updated tuple chain.
+	 */
+	Assert(snapshot == NULL);
+
+	Assert(IsInplaceUpdateRelation(relation) || !IsSystemRelation(relation));
+
+	/* Loop for an exclusive-locked buffer of a non-updated tuple. */
+	for (;;)
+	{
+		TupleTableSlot *slot;
+		BufferHeapTupleTableSlot *bslot;
+
+		CHECK_FOR_INTERRUPTS();
+
+		/*
+		 * Processes issuing heap_update (e.g. GRANT) at maximum speed could
+		 * drive us to this error.  A hostile table owner has stronger ways to
+		 * damage their own table, so that's minor.
+		 */
+		if (retries++ > 10000)
+			elog(ERROR, "giving up after too many tries to overwrite row");
+
+		memcpy(mutable_key, key, sizeof(ScanKeyData) * nkeys);
+		INJECTION_POINT("inplace-before-pin");
+		scan = systable_beginscan(relation, indexId, indexOK, snapshot,
+								  nkeys, mutable_key);
+		oldtup = systable_getnext(scan);
+		if (!HeapTupleIsValid(oldtup))
+		{
+			systable_endscan(scan);
+			*oldtupcopy = NULL;
+			return;
+		}
+
+		slot = scan->slot;
+		Assert(TTS_IS_BUFFERTUPLE(slot));
+		bslot = (BufferHeapTupleTableSlot *) slot;
+		if (heap_inplace_lock(scan->heap_rel,
+							  bslot->base.tuple, bslot->buffer))
+			break;
+		systable_endscan(scan);
+	};
+
+	*oldtupcopy = heap_copytuple(oldtup);
+	*state = scan;
+}
+
+/*
+ * systable_inplace_update_finish --- second phase of inplace update
+ *
+ * The tuple cannot change size, and therefore its header fields and null
+ * bitmap (if any) don't change either.
+ */
+void
+systable_inplace_update_finish(void *state, HeapTuple tuple)
+{
+	SysScanDesc scan = (SysScanDesc) state;
+	Relation	relation = scan->heap_rel;
+	TupleTableSlot *slot = scan->slot;
+	BufferHeapTupleTableSlot *bslot = (BufferHeapTupleTableSlot *) slot;
+	HeapTuple	oldtup = bslot->base.tuple;
+	Buffer		buffer = bslot->buffer;
+
+	heap_inplace_update(relation, oldtup, tuple, buffer);
+	systable_endscan(scan);
+}
+
+/*
+ * systable_inplace_update_cancel --- abandon inplace update
+ *
+ * This is an alternative to making a no-op update.
+ */
+void
+systable_inplace_update_cancel(void *state)
+{
+	SysScanDesc scan = (SysScanDesc) state;
+	Relation	relation = scan->heap_rel;
+	TupleTableSlot *slot = scan->slot;
+	BufferHeapTupleTableSlot *bslot = (BufferHeapTupleTableSlot *) slot;
+	HeapTuple	oldtup = bslot->base.tuple;
+	Buffer		buffer = bslot->buffer;
+
+	heap_inplace_unlock(relation, oldtup, buffer);
+	systable_endscan(scan);
+}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 3375905..e4608b9 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2785,7 +2785,9 @@ index_update_stats(Relation rel,
 {
 	Oid			relid = RelationGetRelid(rel);
 	Relation	pg_class;
+	ScanKeyData key[1];
 	HeapTuple	tuple;
+	void	   *state;
 	Form_pg_class rd_rel;
 	bool		dirty;
 
@@ -2819,33 +2821,12 @@ index_update_stats(Relation rel,
 
 	pg_class = table_open(RelationRelationId, RowExclusiveLock);
 
-	/*
-	 * Make a copy of the tuple to update.  Normally we use the syscache, but
-	 * we can't rely on that during bootstrap or while reindexing pg_class
-	 * itself.
-	 */
-	if (IsBootstrapProcessingMode() ||
-		ReindexIsProcessingHeap(RelationRelationId))
-	{
-		/* don't assume syscache will work */
-		TableScanDesc pg_class_scan;
-		ScanKeyData key[1];
-
-		ScanKeyInit(&key[0],
-					Anum_pg_class_oid,
-					BTEqualStrategyNumber, F_OIDEQ,
-					ObjectIdGetDatum(relid));
-
-		pg_class_scan = table_beginscan_catalog(pg_class, 1, key);
-		tuple = heap_getnext(pg_class_scan, ForwardScanDirection);
-		tuple = heap_copytuple(tuple);
-		table_endscan(pg_class_scan);
-	}
-	else
-	{
-		/* normal case, use syscache */
-		tuple = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relid));
-	}
+	ScanKeyInit(&key[0],
+				Anum_pg_class_oid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(relid));
+	systable_inplace_update_begin(pg_class, ClassOidIndexId, true, NULL,
+								  1, key, &tuple, &state);
 
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "could not find tuple for relation %u", relid);
@@ -2908,11 +2889,12 @@ index_update_stats(Relation rel,
 	 */
 	if (dirty)
 	{
-		heap_inplace_update(pg_class, tuple);
+		systable_inplace_update_finish(state, tuple);
 		/* the above sends a cache inval message */
 	}
 	else
 	{
+		systable_inplace_update_cancel(state);
 		/* no need to change tuple, but force relcache inval anyway */
 		CacheInvalidateRelcacheByTuple(tuple);
 	}
diff --git a/src/backend/catalog/toasting.c b/src/backend/catalog/toasting.c
index 738bc46..ad3082c 100644
--- a/src/backend/catalog/toasting.c
+++ b/src/backend/catalog/toasting.c
@@ -29,6 +29,7 @@
 #include "catalog/toasting.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
+#include "utils/fmgroids.h"
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
@@ -333,21 +334,36 @@ create_toast_table(Relation rel, Oid toastOid, Oid toastIndexOid,
 	 */
 	class_rel = table_open(RelationRelationId, RowExclusiveLock);
 
-	reltup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relOid));
-	if (!HeapTupleIsValid(reltup))
-		elog(ERROR, "cache lookup failed for relation %u", relOid);
-
-	((Form_pg_class) GETSTRUCT(reltup))->reltoastrelid = toast_relid;
-
 	if (!IsBootstrapProcessingMode())
 	{
 		/* normal case, use a transactional update */
+		reltup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relOid));
+		if (!HeapTupleIsValid(reltup))
+			elog(ERROR, "cache lookup failed for relation %u", relOid);
+
+		((Form_pg_class) GETSTRUCT(reltup))->reltoastrelid = toast_relid;
+
 		CatalogTupleUpdate(class_rel, &reltup->t_self, reltup);
 	}
 	else
 	{
 		/* While bootstrapping, we cannot UPDATE, so overwrite in-place */
-		heap_inplace_update(class_rel, reltup);
+
+		ScanKeyData key[1];
+		void	   *state;
+
+		ScanKeyInit(&key[0],
+					Anum_pg_class_oid,
+					BTEqualStrategyNumber, F_OIDEQ,
+					ObjectIdGetDatum(relOid));
+		systable_inplace_update_begin(class_rel, ClassOidIndexId, true,
+									  NULL, 1, key, &reltup, &state);
+		if (!HeapTupleIsValid(reltup))
+			elog(ERROR, "cache lookup failed for relation %u", relOid);
+
+		((Form_pg_class) GETSTRUCT(reltup))->reltoastrelid = toast_relid;
+
+		systable_inplace_update_finish(state, reltup);
 	}
 
 	heap_freetuple(reltup);
diff --git a/src/backend/commands/dbcommands.c b/src/backend/commands/dbcommands.c
index d00ae40..86a08d7 100644
--- a/src/backend/commands/dbcommands.c
+++ b/src/backend/commands/dbcommands.c
@@ -1651,7 +1651,7 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	Relation	pgdbrel;
 	HeapTuple	tup;
 	ScanKeyData scankey;
-	SysScanDesc scan;
+	void	   *inplace_state;
 	Form_pg_database datform;
 	int			notherbackends;
 	int			npreparedxacts;
@@ -1790,24 +1790,6 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	pgstat_drop_database(db_id);
 
 	/*
-	 * Get the pg_database tuple to scribble on.  Note that this does not
-	 * directly rely on the syscache to avoid issues with flattened toast
-	 * values for the in-place update.
-	 */
-	ScanKeyInit(&scankey,
-				Anum_pg_database_datname,
-				BTEqualStrategyNumber, F_NAMEEQ,
-				CStringGetDatum(dbname));
-
-	scan = systable_beginscan(pgdbrel, DatabaseNameIndexId, true,
-							  NULL, 1, &scankey);
-
-	tup = systable_getnext(scan);
-	if (!HeapTupleIsValid(tup))
-		elog(ERROR, "cache lookup failed for database %u", db_id);
-	datform = (Form_pg_database) GETSTRUCT(tup);
-
-	/*
 	 * Except for the deletion of the catalog row, subsequent actions are not
 	 * transactional (consider DropDatabaseBuffers() discarding modified
 	 * buffers). But we might crash or get interrupted below. To prevent
@@ -1818,8 +1800,17 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	 * modification is durable before performing irreversible filesystem
 	 * operations.
 	 */
+	ScanKeyInit(&scankey,
+				Anum_pg_database_datname,
+				BTEqualStrategyNumber, F_NAMEEQ,
+				CStringGetDatum(dbname));
+	systable_inplace_update_begin(pgdbrel, DatabaseNameIndexId, true,
+								  NULL, 1, &scankey, &tup, &inplace_state);
+	if (!HeapTupleIsValid(tup))
+		elog(ERROR, "cache lookup failed for database %u", db_id);
+	datform = (Form_pg_database) GETSTRUCT(tup);
 	datform->datconnlimit = DATCONNLIMIT_INVALID_DB;
-	heap_inplace_update(pgdbrel, tup);
+	systable_inplace_update_finish(inplace_state, tup);
 	XLogFlush(XactLastRecEnd);
 
 	/*
@@ -1827,8 +1818,7 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	 * the row will be gone, but if we fail, dropdb() can be invoked again.
 	 */
 	CatalogTupleDelete(pgdbrel, &tup->t_self);
-
-	systable_endscan(scan);
+	heap_freetuple(tup);
 
 	/*
 	 * Drop db-specific replication slots.
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 7a5ed6b..55baf10 100644
--- a/src/backend/commands/event_trigger.c
+++ b/src/backend/commands/event_trigger.c
@@ -946,25 +946,18 @@ EventTriggerOnLogin(void)
 		{
 			Relation	pg_db = table_open(DatabaseRelationId, RowExclusiveLock);
 			HeapTuple	tuple;
+			void	   *state;
 			Form_pg_database db;
 			ScanKeyData key[1];
-			SysScanDesc scan;
 
-			/*
-			 * Get the pg_database tuple to scribble on.  Note that this does
-			 * not directly rely on the syscache to avoid issues with
-			 * flattened toast values for the in-place update.
-			 */
+			/* Fetch a copy of the tuple to scribble on */
 			ScanKeyInit(&key[0],
 						Anum_pg_database_oid,
 						BTEqualStrategyNumber, F_OIDEQ,
 						ObjectIdGetDatum(MyDatabaseId));
 
-			scan = systable_beginscan(pg_db, DatabaseOidIndexId, true,
-									  NULL, 1, key);
-			tuple = systable_getnext(scan);
-			tuple = heap_copytuple(tuple);
-			systable_endscan(scan);
+			systable_inplace_update_begin(pg_db, DatabaseOidIndexId, true,
+										  NULL, 1, key, &tuple, &state);
 
 			if (!HeapTupleIsValid(tuple))
 				elog(ERROR, "could not find tuple for database %u", MyDatabaseId);
@@ -980,13 +973,15 @@ EventTriggerOnLogin(void)
 				 * that avoids possible waiting on the row-level lock. Second,
 				 * that avoids dealing with TOAST.
 				 *
-				 * It's known that changes made by heap_inplace_update() may
-				 * be lost due to concurrent normal updates.  However, we are
-				 * OK with that.  The subsequent connections will still have a
-				 * chance to set "dathasloginevt" to false.
+				 * 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.
 				 */
-				heap_inplace_update(pg_db, tuple);
+				systable_inplace_update_finish(state, tuple);
 			}
+			else
+				systable_inplace_update_cancel(state);
 			table_close(pg_db, RowExclusiveLock);
 			heap_freetuple(tuple);
 		}
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 7d8e9d2..9304b8c 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -1402,7 +1402,9 @@ vac_update_relstats(Relation relation,
 {
 	Oid			relid = RelationGetRelid(relation);
 	Relation	rd;
+	ScanKeyData key[1];
 	HeapTuple	ctup;
+	void	   *inplace_state;
 	Form_pg_class pgcform;
 	bool		dirty,
 				futurexid,
@@ -1413,7 +1415,12 @@ vac_update_relstats(Relation relation,
 	rd = table_open(RelationRelationId, RowExclusiveLock);
 
 	/* Fetch a copy of the tuple to scribble on */
-	ctup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relid));
+	ScanKeyInit(&key[0],
+				Anum_pg_class_oid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(relid));
+	systable_inplace_update_begin(rd, ClassOidIndexId, true,
+								  NULL, 1, key, &ctup, &inplace_state);
 	if (!HeapTupleIsValid(ctup))
 		elog(ERROR, "pg_class entry for relid %u vanished during vacuuming",
 			 relid);
@@ -1521,7 +1528,9 @@ vac_update_relstats(Relation relation,
 
 	/* If anything changed, write out the tuple. */
 	if (dirty)
-		heap_inplace_update(rd, ctup);
+		systable_inplace_update_finish(inplace_state, ctup);
+	else
+		systable_inplace_update_cancel(inplace_state);
 
 	table_close(rd, RowExclusiveLock);
 
@@ -1573,6 +1582,7 @@ vac_update_datfrozenxid(void)
 	bool		bogus = false;
 	bool		dirty = false;
 	ScanKeyData key[1];
+	void	   *inplace_state;
 
 	/*
 	 * Restrict this task to one backend per database.  This avoids race
@@ -1696,20 +1706,18 @@ vac_update_datfrozenxid(void)
 	relation = table_open(DatabaseRelationId, RowExclusiveLock);
 
 	/*
-	 * Get the pg_database tuple to scribble on.  Note that this does not
-	 * directly rely on the syscache to avoid issues with flattened toast
-	 * values for the in-place update.
+	 * Fetch a copy of the tuple to scribble on.  We could check the syscache
+	 * tuple first.  If that concluded !dirty, we'd avoid waiting on
+	 * concurrent heap_update() and would avoid exclusive-locking the buffer.
+	 * For now, don't optimize that.
 	 */
 	ScanKeyInit(&key[0],
 				Anum_pg_database_oid,
 				BTEqualStrategyNumber, F_OIDEQ,
 				ObjectIdGetDatum(MyDatabaseId));
 
-	scan = systable_beginscan(relation, DatabaseOidIndexId, true,
-							  NULL, 1, key);
-	tuple = systable_getnext(scan);
-	tuple = heap_copytuple(tuple);
-	systable_endscan(scan);
+	systable_inplace_update_begin(relation, DatabaseOidIndexId, true,
+								  NULL, 1, key, &tuple, &inplace_state);
 
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "could not find tuple for database %u", MyDatabaseId);
@@ -1743,7 +1751,9 @@ vac_update_datfrozenxid(void)
 		newMinMulti = dbform->datminmxid;
 
 	if (dirty)
-		heap_inplace_update(relation, tuple);
+		systable_inplace_update_finish(inplace_state, tuple);
+	else
+		systable_inplace_update_cancel(inplace_state);
 
 	heap_freetuple(tuple);
 	table_close(relation, RowExclusiveLock);
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index fdcfbe8..c25f5d1 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -233,5 +233,14 @@ extern SysScanDesc systable_beginscan_ordered(Relation heapRelation,
 extern HeapTuple systable_getnext_ordered(SysScanDesc sysscan,
 										  ScanDirection direction);
 extern void systable_endscan_ordered(SysScanDesc sysscan);
+extern void systable_inplace_update_begin(Relation relation,
+										  Oid indexId,
+										  bool indexOK,
+										  Snapshot snapshot,
+										  int nkeys, const ScanKeyData *key,
+										  HeapTuple *oldtupcopy,
+										  void **state);
+extern void systable_inplace_update_finish(void *state, HeapTuple tuple);
+extern void systable_inplace_update_cancel(void *state);
 
 #endif							/* GENAM_H */
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 9e9aec8..85ad32a 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -336,7 +336,13 @@ extern TM_Result heap_lock_tuple(Relation relation, HeapTuple tuple,
 								 bool follow_updates,
 								 Buffer *buffer, struct TM_FailureData *tmfd);
 
-extern void heap_inplace_update(Relation relation, HeapTuple tuple);
+extern bool heap_inplace_lock(Relation relation,
+							  HeapTuple oldtup_ptr, Buffer buffer);
+extern void heap_inplace_update(Relation relation,
+								HeapTuple oldtup, HeapTuple tuple,
+								Buffer buffer);
+extern void heap_inplace_unlock(Relation relation,
+								HeapTuple oldtup, Buffer buffer);
 extern bool heap_prepare_freeze_tuple(HeapTupleHeader tuple,
 									  const struct VacuumCutoffs *cutoffs,
 									  HeapPageFreeze *pagefrz,
diff --git a/src/test/isolation/expected/intra-grant-inplace-db.out b/src/test/isolation/expected/intra-grant-inplace-db.out
index 432ece5..a91402c 100644
--- a/src/test/isolation/expected/intra-grant-inplace-db.out
+++ b/src/test/isolation/expected/intra-grant-inplace-db.out
@@ -9,20 +9,20 @@ step b1: BEGIN;
 step grant1: 
 	GRANT TEMP ON DATABASE isolation_regression TO regress_temp_grantee;
 
-step vac2: VACUUM (FREEZE);
+step vac2: VACUUM (FREEZE); <waiting ...>
 step snap3: 
 	INSERT INTO frozen_witness
 	SELECT datfrozenxid FROM pg_database WHERE datname = current_catalog;
 
 step c1: COMMIT;
+step vac2: <... completed>
 step cmp3: 
 	SELECT 'datfrozenxid retreated'
 	FROM pg_database
 	WHERE datname = current_catalog
 		AND age(datfrozenxid) > (SELECT min(age(x)) FROM frozen_witness);
 
-?column?              
-----------------------
-datfrozenxid retreated
-(1 row)
+?column?
+--------
+(0 rows)
 
diff --git a/src/test/isolation/expected/intra-grant-inplace.out b/src/test/isolation/expected/intra-grant-inplace.out
index cc1e47a..fe26984 100644
--- a/src/test/isolation/expected/intra-grant-inplace.out
+++ b/src/test/isolation/expected/intra-grant-inplace.out
@@ -14,15 +14,16 @@ relhasindex
 f          
 (1 row)
 
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
 step c1: COMMIT;
+step addk2: <... completed>
 step read2: 
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass;
 
 relhasindex
 -----------
-f          
+t          
 (1 row)
 
 
@@ -58,8 +59,9 @@ relhasindex
 f          
 (1 row)
 
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
 step r3: ROLLBACK;
+step addk2: <... completed>
 
 starting permutation: b2 sfnku2 addk2 c2
 step b2: BEGIN;
@@ -98,7 +100,7 @@ f
 step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
 step c2: COMMIT;
 
-starting permutation: b3 sfu3 b1 grant1 read2 addk2 r3 c1 read2
+starting permutation: b3 sfu3 b1 grant1 read2 as3 addk2 r3 c1 read2
 step b3: BEGIN ISOLATION LEVEL READ COMMITTED;
 step sfu3: 
 	SELECT relhasindex FROM pg_class
@@ -122,17 +124,19 @@ relhasindex
 f          
 (1 row)
 
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
+step as3: LOCK TABLE intra_grant_inplace IN ACCESS SHARE MODE;
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
 step r3: ROLLBACK;
 step grant1: <... completed>
 step c1: COMMIT;
+step addk2: <... completed>
 step read2: 
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass;
 
 relhasindex
 -----------
-f          
+t          
 (1 row)
 
 
diff --git a/src/test/isolation/specs/intra-grant-inplace-db.spec b/src/test/isolation/specs/intra-grant-inplace-db.spec
index bbecd5d..9de40ec 100644
--- a/src/test/isolation/specs/intra-grant-inplace-db.spec
+++ b/src/test/isolation/specs/intra-grant-inplace-db.spec
@@ -42,5 +42,4 @@ step cmp3	{
 }
 
 
-# XXX extant bug
 permutation snap3 b1 grant1 vac2(c1) snap3 c1 cmp3
diff --git a/src/test/isolation/specs/intra-grant-inplace.spec b/src/test/isolation/specs/intra-grant-inplace.spec
index 3cd696b..d07ed3b 100644
--- a/src/test/isolation/specs/intra-grant-inplace.spec
+++ b/src/test/isolation/specs/intra-grant-inplace.spec
@@ -48,6 +48,7 @@ step sfu3	{
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass FOR UPDATE;
 }
+step as3	{ LOCK TABLE intra_grant_inplace IN ACCESS SHARE MODE; }
 step r3	{ ROLLBACK; }
 
 # Additional heap_update()
@@ -73,7 +74,7 @@ step keyshr5	{
 teardown	{ ROLLBACK; }
 
 
-# XXX extant bugs: permutation comments refer to planned post-bugfix behavior
+# XXX extant bugs: permutation comments refer to planned future LockTuple()
 
 permutation
 	b1
@@ -117,6 +118,7 @@ permutation
 	b1
 	grant1(r3)	# acquire LockTuple(), await sfu3 xmax
 	read2
+	as3			# XXX temporary until patch adds locking to addk2
 	addk2(c1)	# block in LockTuple() behind grant1
 	r3			# unblock grant1; addk2 now awaits grant1 xmax
 	c1
diff --git a/src/test/modules/injection_points/expected/inplace.out b/src/test/modules/injection_points/expected/inplace.out
index 123f45a..db7dab6 100644
--- a/src/test/modules/injection_points/expected/inplace.out
+++ b/src/test/modules/injection_points/expected/inplace.out
@@ -40,4 +40,301 @@ step read1:
 	SELECT reltuples = -1 AS reltuples_unknown
 	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
 
-ERROR:  could not create unique index "pg_class_oid_index"
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 c2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step c2: COMMIT;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 r2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step r2: ROLLBACK;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 c2 revoke2 grant2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step c2: COMMIT;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: vac1 begin2 grant2 revoke2 mkrels3 c2 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step c2: COMMIT;
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 r2 grant2 revoke2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step r2: ROLLBACK;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
+
+starting permutation: begin2 grant2 vac1 c2 revoke2 vac3 mkrels3 read1
+mkrels
+------
+      
+(1 row)
+
+injection_points_attach
+-----------------------
+                       
+(1 row)
+
+step begin2: BEGIN;
+step grant2: GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC;
+step vac1: VACUUM vactest.orig50;  -- wait during inplace update <waiting ...>
+step c2: COMMIT;
+step revoke2: REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC;
+step vac3: VACUUM pg_class;
+step mkrels3: 
+	SELECT vactest.mkrels('intruder', 1, 100);  -- repopulate LP_UNUSED
+	SELECT injection_points_detach('inplace-before-pin');
+	SELECT injection_points_wakeup('inplace-before-pin');
+
+mkrels
+------
+      
+(1 row)
+
+injection_points_detach
+-----------------------
+                       
+(1 row)
+
+injection_points_wakeup
+-----------------------
+                       
+(1 row)
+
+step vac1: <... completed>
+step read1: 
+	REINDEX TABLE pg_class;  -- look for duplicates
+	SELECT reltuples = -1 AS reltuples_unknown
+	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
+
+reltuples_unknown
+-----------------
+f                
+(1 row)
+
diff --git a/src/test/modules/injection_points/specs/inplace.spec b/src/test/modules/injection_points/specs/inplace.spec
index e957713..86539a5 100644
--- a/src/test/modules/injection_points/specs/inplace.spec
+++ b/src/test/modules/injection_points/specs/inplace.spec
@@ -32,12 +32,9 @@ setup
 	CREATE TABLE vactest.orig50 ();
 	SELECT vactest.mkrels('orig', 51, 100);
 }
-
-# XXX DROP causes an assertion failure; adopt DROP once fixed
 teardown
 {
-	--DROP SCHEMA vactest CASCADE;
-	DO $$BEGIN EXECUTE 'ALTER SCHEMA vactest RENAME TO schema' || oid FROM pg_namespace where nspname = 'vactest'; END$$;
+	DROP SCHEMA vactest CASCADE;
 	DROP EXTENSION injection_points;
 }
 
@@ -56,11 +53,13 @@ step read1	{
 	FROM pg_class WHERE oid = 'vactest.orig50'::regclass;
 }
 
-
 # Transactional updates of the tuple vac1 is waiting to inplace-update.
 session s2
 step grant2		{ GRANT SELECT ON TABLE vactest.orig50 TO PUBLIC; }
-
+step revoke2	{ REVOKE SELECT ON TABLE vactest.orig50 FROM PUBLIC; }
+step begin2		{ BEGIN; }
+step c2			{ COMMIT; }
+step r2			{ ROLLBACK; }
 
 # Non-blocking actions.
 session s3
@@ -74,10 +73,69 @@ step mkrels3	{
 }
 
 
-# XXX extant bug
+# target gains a successor at the last moment
 permutation
 	vac1(mkrels3)	# reads pg_class tuple T0 for vactest.orig50, xmax invalid
 	grant2			# T0 becomes eligible for pruning, T1 is successor
 	vac3			# T0 becomes LP_UNUSED
-	mkrels3			# T0 reused; vac1 wakes and overwrites the reused T0
+	mkrels3			# vac1 wakes, scans to T1
 	read1
+
+# target already has a successor, which commits
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1
+	vac1(mkrels3)	# reads T0 for vactest.orig50
+	c2				# T0 becomes eligible for pruning
+	vac3			# T0 becomes LP_UNUSED
+	mkrels3			# vac1 wakes, scans to T1
+	read1
+
+# target already has a successor, which becomes LP_UNUSED at the last moment
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1
+	vac1(mkrels3)	# reads T0 for vactest.orig50
+	r2				# T1 becomes eligible for pruning
+	vac3			# T1 becomes LP_UNUSED
+	mkrels3			# reuse T1; vac1 scans to T0
+	read1
+
+# target already has a successor, which becomes LP_REDIRECT at the last moment
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1, non-HOT due to filled page
+	vac1(mkrels3)	# reads T0
+	c2
+	revoke2			# HOT update to T2
+	grant2			# HOT update to T3
+	vac3			# T1 becomes LP_REDIRECT
+	mkrels3			# reuse T2; vac1 scans to T3
+	read1
+
+# waiting for updater to end
+permutation
+	vac1(c2)		# reads pg_class tuple T0 for vactest.orig50, xmax invalid
+	begin2
+	grant2			# T0.t_ctid = T1, non-HOT due to filled page
+	revoke2			# HOT update to T2
+	mkrels3			# vac1 awakes briefly, then waits for s2
+	c2
+	read1
+
+# Another LP_UNUSED.  This time, do change the live tuple.  Final live tuple
+# body is identical to original, at a different TID.
+permutation
+	begin2
+	grant2			# T0.t_ctid = T1, non-HOT due to filled page
+	vac1(mkrels3)	# reads T0
+	r2				# T1 becomes eligible for pruning
+	grant2			# T0.t_ctid = T2; T0 becomes eligible for pruning
+	revoke2			# T2.t_ctid = T3; T2 becomes eligible for pruning
+	vac3			# T0, T1 & T2 become LP_UNUSED
+	mkrels3			# reuse T0, T1 & T2; vac1 scans to T3
+	read1
+
+# Another LP_REDIRECT.  Compared to the earlier test, omit the last grant2.
+# Hence, final live tuple body is identical to original, at a different TID.
+permutation begin2 grant2 vac1(mkrels3) c2 revoke2 vac3 mkrels3 read1
inplace120-locktag-v9.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Make heap_update() callers wait for inplace update.
    
    The previous commit fixed some ways of losing an inplace update.  It
    remained possible to lose one when a backend working toward a
    heap_update() copied a tuple into memory just before inplace update of
    that tuple.  In catalogs eligible for inplace update, use LOCKTAG_TUPLE
    to govern admission to the steps of copying an old tuple, modifying it,
    and issuing heap_update().  This includes UPDATE and MERGE commands.  To
    avoid changing most of the pg_class DDL, don't require LOCKTAG_TUPLE
    when holding a relation lock sufficient to exclude inplace updaters.
    Back-patch to v12 (all supported versions).
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20231027214946.79.nmisch@google.com

diff --git a/src/backend/access/heap/README.tuplock b/src/backend/access/heap/README.tuplock
index ddb2def..95828ce 100644
--- a/src/backend/access/heap/README.tuplock
+++ b/src/backend/access/heap/README.tuplock
@@ -154,6 +154,48 @@ The following infomask bits are applicable:
 We currently never set the HEAP_XMAX_COMMITTED when the HEAP_XMAX_IS_MULTI bit
 is set.
 
+Locking to write inplace-updated tables
+---------------------------------------
+
+If IsInplaceUpdateRelation() returns true for a table, the table is a system
+catalog that receives heap_inplace_update_scan() calls.  Preparing a
+heap_update() of these tables follows additional locking rules, to ensure we
+don't lose the effects of an inplace update.  In particular, consider a moment
+when a backend has fetched the old tuple to modify, not yet having called
+heap_update().  Another backend's inplace update starting then can't conclude
+until the heap_update() places its new tuple in a buffer.  We enforce that
+using locktags as follows.  While DDL code is the main audience, the executor
+follows these rules to make e.g. "MERGE INTO pg_class" safer.  Locking rules
+are per-catalog:
+
+  pg_class heap_inplace_update_scan() callers: before the call, acquire a lock
+  on the relation in mode ShareUpdateExclusiveLock or stricter.  If the update
+  targets a row of RELKIND_INDEX (but not RELKIND_PARTITIONED_INDEX), that
+  lock must be on the table.  Locking the index rel is not necessary.  (This
+  allows VACUUM to overwrite per-index pg_class while holding a lock on the
+  table alone.) heap_inplace_update_scan() acquires and releases LOCKTAG_TUPLE
+  in InplaceUpdateTupleLock, an alias for ExclusiveLock, on each tuple it
+  overwrites.
+
+  pg_class heap_update() callers: before copying the tuple to modify, take a
+  lock on the tuple, a ShareUpdateExclusiveLock on the relation, or a
+  ShareRowExclusiveLock or stricter on the relation.
+
+  SearchSysCacheLocked1() is one convenient way to acquire the tuple lock.
+  Most heap_update() callers already hold a suitable lock on the relation for
+  other reasons and can skip the tuple lock.  If you do acquire the tuple
+  lock, release it immediately after the update.
+
+
+  pg_database: before copying the tuple to modify, all updaters of pg_database
+  rows acquire LOCKTAG_TUPLE.  (Few updaters acquire LOCKTAG_OBJECT on the
+  database OID, so it wasn't worth extending that as a second option.)
+
+Ideally, DDL might want to perform permissions checks before LockTuple(), as
+we do with RangeVarGetRelidExtended() callbacks.  We typically don't bother.
+LOCKTAG_TUPLE acquirers release it after each row, so the potential
+inconvenience is lower.
+
 Reading inplace-updated columns
 -------------------------------
 
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 24f7e62..7de60c1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -51,6 +51,8 @@
 #include "access/xloginsert.h"
 #include "access/xlogutils.h"
 #include "catalog/catalog.h"
+#include "catalog/pg_database.h"
+#include "catalog/pg_database_d.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
@@ -75,6 +77,12 @@ static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
 								  Buffer newbuf, HeapTuple oldtup,
 								  HeapTuple newtup, HeapTuple old_key_tuple,
 								  bool all_visible_cleared, bool new_all_visible_cleared);
+#ifdef USE_ASSERT_CHECKING
+static void check_lock_if_inplace_updateable_rel(Relation relation,
+												 ItemPointer otid,
+												 HeapTuple newtup);
+static void check_inplace_rel_lock(HeapTuple oldtup);
+#endif
 static Bitmapset *HeapDetermineColumnsInfo(Relation relation,
 										   Bitmapset *interesting_cols,
 										   Bitmapset *external_cols,
@@ -121,6 +129,8 @@ static HeapTuple ExtractReplicaIdentity(Relation relation, HeapTuple tp, bool ke
  * heavyweight lock mode and MultiXactStatus values to use for any particular
  * tuple lock strength.
  *
+ * These interact with InplaceUpdateTupleLock, an alias for ExclusiveLock.
+ *
  * Don't look at lockstatus/updstatus directly!  Use get_mxact_status_for_lock
  * instead.
  */
@@ -3207,6 +3217,10 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
 				(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
 				 errmsg("cannot update tuples during a parallel operation")));
 
+#ifdef USE_ASSERT_CHECKING
+	check_lock_if_inplace_updateable_rel(relation, otid, newtup);
+#endif
+
 	/*
 	 * Fetch the list of attributes to be checked for various operations.
 	 *
@@ -4071,6 +4085,128 @@ l2:
 	return TM_Ok;
 }
 
+#ifdef USE_ASSERT_CHECKING
+/*
+ * Confirm adequate lock held during heap_update(), per rules from
+ * README.tuplock section "Locking to write inplace-updated tables".
+ */
+static void
+check_lock_if_inplace_updateable_rel(Relation relation,
+									 ItemPointer otid,
+									 HeapTuple newtup)
+{
+	/* LOCKTAG_TUPLE acceptable for any catalog */
+	switch (RelationGetRelid(relation))
+	{
+		case RelationRelationId:
+		case DatabaseRelationId:
+			{
+				LOCKTAG		tuptag;
+
+				SET_LOCKTAG_TUPLE(tuptag,
+								  relation->rd_lockInfo.lockRelId.dbId,
+								  relation->rd_lockInfo.lockRelId.relId,
+								  ItemPointerGetBlockNumber(otid),
+								  ItemPointerGetOffsetNumber(otid));
+				if (LockHeldByMe(&tuptag, InplaceUpdateTupleLock, false))
+					return;
+			}
+			break;
+		default:
+			Assert(!IsInplaceUpdateRelation(relation));
+			return;
+	}
+
+	switch (RelationGetRelid(relation))
+	{
+		case RelationRelationId:
+			{
+				/* LOCKTAG_TUPLE or LOCKTAG_RELATION ok */
+				Form_pg_class classForm = (Form_pg_class) GETSTRUCT(newtup);
+				Oid			relid = classForm->oid;
+				Oid			dbid;
+				LOCKTAG		tag;
+
+				if (IsSharedRelation(relid))
+					dbid = InvalidOid;
+				else
+					dbid = MyDatabaseId;
+
+				if (classForm->relkind == RELKIND_INDEX)
+				{
+					Relation	irel = index_open(relid, AccessShareLock);
+
+					SET_LOCKTAG_RELATION(tag, dbid, irel->rd_index->indrelid);
+					index_close(irel, AccessShareLock);
+				}
+				else
+					SET_LOCKTAG_RELATION(tag, dbid, relid);
+
+				if (!LockHeldByMe(&tag, ShareUpdateExclusiveLock, false) &&
+					!LockHeldByMe(&tag, ShareRowExclusiveLock, true))
+					elog(WARNING,
+						 "missing lock for relation \"%s\" (OID %u, relkind %c) @ TID (%u,%u)",
+						 NameStr(classForm->relname),
+						 relid,
+						 classForm->relkind,
+						 ItemPointerGetBlockNumber(otid),
+						 ItemPointerGetOffsetNumber(otid));
+			}
+			break;
+		case DatabaseRelationId:
+			{
+				/* LOCKTAG_TUPLE required */
+				Form_pg_database dbForm = (Form_pg_database) GETSTRUCT(newtup);
+
+				elog(WARNING,
+					 "missing lock on database \"%s\" (OID %u) @ TID (%u,%u)",
+					 NameStr(dbForm->datname),
+					 dbForm->oid,
+					 ItemPointerGetBlockNumber(otid),
+					 ItemPointerGetOffsetNumber(otid));
+			}
+			break;
+	}
+}
+
+/*
+ * Confirm adequate relation lock held, per rules from README.tuplock section
+ * "Locking to write inplace-updated tables".
+ */
+static void
+check_inplace_rel_lock(HeapTuple oldtup)
+{
+	Form_pg_class classForm = (Form_pg_class) GETSTRUCT(oldtup);
+	Oid			relid = classForm->oid;
+	Oid			dbid;
+	LOCKTAG		tag;
+
+	if (IsSharedRelation(relid))
+		dbid = InvalidOid;
+	else
+		dbid = MyDatabaseId;
+
+	if (classForm->relkind == RELKIND_INDEX)
+	{
+		Relation	irel = index_open(relid, AccessShareLock);
+
+		SET_LOCKTAG_RELATION(tag, dbid, irel->rd_index->indrelid);
+		index_close(irel, AccessShareLock);
+	}
+	else
+		SET_LOCKTAG_RELATION(tag, dbid, relid);
+
+	if (!LockHeldByMe(&tag, ShareUpdateExclusiveLock, true))
+		elog(WARNING,
+			 "missing lock for relation \"%s\" (OID %u, relkind %c) @ TID (%u,%u)",
+			 NameStr(classForm->relname),
+			 relid,
+			 classForm->relkind,
+			 ItemPointerGetBlockNumber(&oldtup->t_self),
+			 ItemPointerGetOffsetNumber(&oldtup->t_self));
+}
+#endif
+
 /*
  * Check if the specified attribute's values are the same.  Subroutine for
  * HeapDetermineColumnsInfo.
@@ -6087,15 +6223,21 @@ heap_inplace_lock(Relation relation,
 	TM_Result	result;
 	bool		ret;
 
+#ifdef USE_ASSERT_CHECKING
+	if (RelationGetRelid(relation) == RelationRelationId)
+		check_inplace_rel_lock(oldtup_ptr);
+#endif
+
 	Assert(BufferIsValid(buffer));
 
+	LockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
 	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
 
 	/*----------
 	 * Interpret HeapTupleSatisfiesUpdate() like heap_update() does, except:
 	 *
 	 * - wait unconditionally
-	 * - no tuple locks
+	 * - already locked tuple above, since inplace needs that unconditionally
 	 * - don't recheck header after wait: simpler to defer to next iteration
 	 * - don't try to continue even if the updater aborts: likewise
 	 * - no crosscheck
@@ -6179,7 +6321,10 @@ heap_inplace_lock(Relation relation,
 	 * don't bother optimizing that.
 	 */
 	if (!ret)
+	{
+		UnlockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
 		InvalidateCatalogSnapshot();
+	}
 	return ret;
 }
 
@@ -6188,6 +6333,8 @@ heap_inplace_lock(Relation relation,
  *
  * The tuple cannot change size, and therefore its header fields and null
  * bitmap (if any) don't change either.
+ *
+ * Since we hold LOCKTAG_TUPLE, no updater has a local copy of this tuple.
  */
 void
 heap_inplace_update(Relation relation,
@@ -6271,6 +6418,7 @@ heap_inplace_unlock(Relation relation,
 					HeapTuple oldtup, Buffer buffer)
 {
 	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+	UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
 }
 
 #define		FRM_NOOP				0x0001
diff --git a/src/backend/access/index/genam.c b/src/backend/access/index/genam.c
index 5f55e8c..a3f2dc2 100644
--- a/src/backend/access/index/genam.c
+++ b/src/backend/access/index/genam.c
@@ -755,7 +755,9 @@ systable_endscan_ordered(SysScanDesc sysscan)
  *
  * Overwriting violates both MVCC and transactional safety, so the uses of
  * this function in Postgres are extremely limited.  Nonetheless we find some
- * places to use it.  Standard flow:
+ * places to use it.  See README.tuplock section "Locking to write
+ * inplace-updated tables" and later sections for expectations of readers and
+ * writers of a table that gets inplace updates.  Standard flow:
  *
  * ... [any slow preparation not requiring oldtup] ...
  * systable_inplace_update_begin([...], &tup, &inplace_state);
diff --git a/src/backend/catalog/aclchk.c b/src/backend/catalog/aclchk.c
index a44ccee..bc0e259 100644
--- a/src/backend/catalog/aclchk.c
+++ b/src/backend/catalog/aclchk.c
@@ -75,6 +75,7 @@
 #include "nodes/makefuncs.h"
 #include "parser/parse_func.h"
 #include "parser/parse_type.h"
+#include "storage/lmgr.h"
 #include "utils/acl.h"
 #include "utils/aclchk_internal.h"
 #include "utils/builtins.h"
@@ -1848,7 +1849,7 @@ ExecGrant_Relation(InternalGrant *istmt)
 		HeapTuple	tuple;
 		ListCell   *cell_colprivs;
 
-		tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(relOid));
+		tuple = SearchSysCacheLocked1(RELOID, ObjectIdGetDatum(relOid));
 		if (!HeapTupleIsValid(tuple))
 			elog(ERROR, "cache lookup failed for relation %u", relOid);
 		pg_class_tuple = (Form_pg_class) GETSTRUCT(tuple);
@@ -2060,6 +2061,7 @@ ExecGrant_Relation(InternalGrant *istmt)
 										 values, nulls, replaces);
 
 			CatalogTupleUpdate(relation, &newtuple->t_self, newtuple);
+			UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
 
 			/* Update initial privileges for extensions */
 			recordExtensionInitPriv(relOid, RelationRelationId, 0, new_acl);
@@ -2072,6 +2074,8 @@ ExecGrant_Relation(InternalGrant *istmt)
 
 			pfree(new_acl);
 		}
+		else
+			UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
 
 		/*
 		 * Handle column-level privileges, if any were specified or implied.
@@ -2185,7 +2189,7 @@ ExecGrant_common(InternalGrant *istmt, Oid classid, AclMode default_privs,
 		Oid		   *oldmembers;
 		Oid		   *newmembers;
 
-		tuple = SearchSysCache1(cacheid, ObjectIdGetDatum(objectid));
+		tuple = SearchSysCacheLocked1(cacheid, ObjectIdGetDatum(objectid));
 		if (!HeapTupleIsValid(tuple))
 			elog(ERROR, "cache lookup failed for %s %u", get_object_class_descr(classid), objectid);
 
@@ -2261,6 +2265,7 @@ ExecGrant_common(InternalGrant *istmt, Oid classid, AclMode default_privs,
 									 nulls, replaces);
 
 		CatalogTupleUpdate(relation, &newtuple->t_self, newtuple);
+		UnlockTuple(relation, &tuple->t_self, InplaceUpdateTupleLock);
 
 		/* Update initial privileges for extensions */
 		recordExtensionInitPriv(objectid, classid, 0, new_acl);
diff --git a/src/backend/catalog/catalog.c b/src/backend/catalog/catalog.c
index 6c39434..8aefbcd 100644
--- a/src/backend/catalog/catalog.c
+++ b/src/backend/catalog/catalog.c
@@ -138,6 +138,15 @@ IsCatalogRelationOid(Oid relid)
 /*
  * IsInplaceUpdateRelation
  *		True iff core code performs inplace updates on the relation.
+ *
+ *		This is used for assertions and for making the executor follow the
+ *		locking protocol described at README.tuplock section "Locking to write
+ *		inplace-updated tables".  Extensions may inplace-update other heap
+ *		tables, but concurrent SQL UPDATE on the same table may overwrite
+ *		those modifications.
+ *
+ *		The executor can assume these are not partitions or partitioned and
+ *		have no triggers.
  */
 bool
 IsInplaceUpdateRelation(Relation relation)
diff --git a/src/backend/commands/dbcommands.c b/src/backend/commands/dbcommands.c
index 86a08d7..2987ce9 100644
--- a/src/backend/commands/dbcommands.c
+++ b/src/backend/commands/dbcommands.c
@@ -1877,6 +1877,7 @@ RenameDatabase(const char *oldname, const char *newname)
 {
 	Oid			db_id;
 	HeapTuple	newtup;
+	ItemPointerData otid;
 	Relation	rel;
 	int			notherbackends;
 	int			npreparedxacts;
@@ -1948,11 +1949,13 @@ RenameDatabase(const char *oldname, const char *newname)
 				 errdetail_busy_db(notherbackends, npreparedxacts)));
 
 	/* rename */
-	newtup = SearchSysCacheCopy1(DATABASEOID, ObjectIdGetDatum(db_id));
+	newtup = SearchSysCacheLockedCopy1(DATABASEOID, ObjectIdGetDatum(db_id));
 	if (!HeapTupleIsValid(newtup))
 		elog(ERROR, "cache lookup failed for database %u", db_id);
+	otid = newtup->t_self;
 	namestrcpy(&(((Form_pg_database) GETSTRUCT(newtup))->datname), newname);
-	CatalogTupleUpdate(rel, &newtup->t_self, newtup);
+	CatalogTupleUpdate(rel, &otid, newtup);
+	UnlockTuple(rel, &otid, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(DatabaseRelationId, db_id, 0);
 
@@ -2201,6 +2204,7 @@ movedb(const char *dbname, const char *tblspcname)
 			ereport(ERROR,
 					(errcode(ERRCODE_UNDEFINED_DATABASE),
 					 errmsg("database \"%s\" does not exist", dbname)));
+		LockTuple(pgdbrel, &oldtuple->t_self, InplaceUpdateTupleLock);
 
 		new_record[Anum_pg_database_dattablespace - 1] = ObjectIdGetDatum(dst_tblspcoid);
 		new_record_repl[Anum_pg_database_dattablespace - 1] = true;
@@ -2209,6 +2213,7 @@ movedb(const char *dbname, const char *tblspcname)
 									 new_record,
 									 new_record_nulls, new_record_repl);
 		CatalogTupleUpdate(pgdbrel, &oldtuple->t_self, newtuple);
+		UnlockTuple(pgdbrel, &oldtuple->t_self, InplaceUpdateTupleLock);
 
 		InvokeObjectPostAlterHook(DatabaseRelationId, db_id, 0);
 
@@ -2439,6 +2444,7 @@ AlterDatabase(ParseState *pstate, AlterDatabaseStmt *stmt, bool isTopLevel)
 		ereport(ERROR,
 				(errcode(ERRCODE_UNDEFINED_DATABASE),
 				 errmsg("database \"%s\" does not exist", stmt->dbname)));
+	LockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	datform = (Form_pg_database) GETSTRUCT(tuple);
 	dboid = datform->oid;
@@ -2488,6 +2494,7 @@ AlterDatabase(ParseState *pstate, AlterDatabaseStmt *stmt, bool isTopLevel)
 	newtuple = heap_modify_tuple(tuple, RelationGetDescr(rel), new_record,
 								 new_record_nulls, new_record_repl);
 	CatalogTupleUpdate(rel, &tuple->t_self, newtuple);
+	UnlockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(DatabaseRelationId, dboid, 0);
 
@@ -2537,6 +2544,7 @@ AlterDatabaseRefreshColl(AlterDatabaseRefreshCollStmt *stmt)
 	if (!object_ownercheck(DatabaseRelationId, db_id, GetUserId()))
 		aclcheck_error(ACLCHECK_NOT_OWNER, OBJECT_DATABASE,
 					   stmt->dbname);
+	LockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	datum = heap_getattr(tuple, Anum_pg_database_datcollversion, RelationGetDescr(rel), &isnull);
 	oldversion = isnull ? NULL : TextDatumGetCString(datum);
@@ -2565,6 +2573,7 @@ AlterDatabaseRefreshColl(AlterDatabaseRefreshCollStmt *stmt)
 		bool		nulls[Natts_pg_database] = {0};
 		bool		replaces[Natts_pg_database] = {0};
 		Datum		values[Natts_pg_database] = {0};
+		HeapTuple	newtuple;
 
 		ereport(NOTICE,
 				(errmsg("changing version from %s to %s",
@@ -2573,14 +2582,15 @@ AlterDatabaseRefreshColl(AlterDatabaseRefreshCollStmt *stmt)
 		values[Anum_pg_database_datcollversion - 1] = CStringGetTextDatum(newversion);
 		replaces[Anum_pg_database_datcollversion - 1] = true;
 
-		tuple = heap_modify_tuple(tuple, RelationGetDescr(rel),
-								  values, nulls, replaces);
-		CatalogTupleUpdate(rel, &tuple->t_self, tuple);
-		heap_freetuple(tuple);
+		newtuple = heap_modify_tuple(tuple, RelationGetDescr(rel),
+									 values, nulls, replaces);
+		CatalogTupleUpdate(rel, &tuple->t_self, newtuple);
+		heap_freetuple(newtuple);
 	}
 	else
 		ereport(NOTICE,
 				(errmsg("version has not changed")));
+	UnlockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(DatabaseRelationId, db_id, 0);
 
@@ -2692,6 +2702,8 @@ AlterDatabaseOwner(const char *dbname, Oid newOwnerId)
 					(errcode(ERRCODE_INSUFFICIENT_PRIVILEGE),
 					 errmsg("permission denied to change owner of database")));
 
+		LockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
+
 		repl_repl[Anum_pg_database_datdba - 1] = true;
 		repl_val[Anum_pg_database_datdba - 1] = ObjectIdGetDatum(newOwnerId);
 
@@ -2713,6 +2725,7 @@ AlterDatabaseOwner(const char *dbname, Oid newOwnerId)
 
 		newtuple = heap_modify_tuple(tuple, RelationGetDescr(rel), repl_val, repl_null, repl_repl);
 		CatalogTupleUpdate(rel, &newtuple->t_self, newtuple);
+		UnlockTuple(rel, &tuple->t_self, InplaceUpdateTupleLock);
 
 		heap_freetuple(newtuple);
 
diff --git a/src/backend/commands/event_trigger.c b/src/backend/commands/event_trigger.c
index 55baf10..05a6de6 100644
--- a/src/backend/commands/event_trigger.c
+++ b/src/backend/commands/event_trigger.c
@@ -388,6 +388,7 @@ SetDatabaseHasLoginEventTriggers(void)
 	/* Set dathasloginevt flag in pg_database */
 	Form_pg_database db;
 	Relation	pg_db = table_open(DatabaseRelationId, RowExclusiveLock);
+	ItemPointerData otid;
 	HeapTuple	tuple;
 
 	/*
@@ -399,16 +400,18 @@ SetDatabaseHasLoginEventTriggers(void)
 	 */
 	LockSharedObject(DatabaseRelationId, MyDatabaseId, 0, AccessExclusiveLock);
 
-	tuple = SearchSysCacheCopy1(DATABASEOID, ObjectIdGetDatum(MyDatabaseId));
+	tuple = SearchSysCacheLockedCopy1(DATABASEOID, ObjectIdGetDatum(MyDatabaseId));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "cache lookup failed for database %u", MyDatabaseId);
+	otid = tuple->t_self;
 	db = (Form_pg_database) GETSTRUCT(tuple);
 	if (!db->dathasloginevt)
 	{
 		db->dathasloginevt = true;
-		CatalogTupleUpdate(pg_db, &tuple->t_self, tuple);
+		CatalogTupleUpdate(pg_db, &otid, tuple);
 		CommandCounterIncrement();
 	}
+	UnlockTuple(pg_db, &otid, InplaceUpdateTupleLock);
 	table_close(pg_db, RowExclusiveLock);
 	heap_freetuple(tuple);
 }
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index c5a56c7..6b22a88 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -4413,14 +4413,17 @@ update_relispartition(Oid relationId, bool newval)
 {
 	HeapTuple	tup;
 	Relation	classRel;
+	ItemPointerData otid;
 
 	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relationId));
+	tup = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(relationId));
 	if (!HeapTupleIsValid(tup))
 		elog(ERROR, "cache lookup failed for relation %u", relationId);
+	otid = tup->t_self;
 	Assert(((Form_pg_class) GETSTRUCT(tup))->relispartition != newval);
 	((Form_pg_class) GETSTRUCT(tup))->relispartition = newval;
-	CatalogTupleUpdate(classRel, &tup->t_self, tup);
+	CatalogTupleUpdate(classRel, &otid, tup);
+	UnlockTuple(classRel, &otid, InplaceUpdateTupleLock);
 	heap_freetuple(tup);
 	table_close(classRel, RowExclusiveLock);
 }
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 52ce6b0..03278f6 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3590,6 +3590,7 @@ SetRelationTableSpace(Relation rel,
 {
 	Relation	pg_class;
 	HeapTuple	tuple;
+	ItemPointerData otid;
 	Form_pg_class rd_rel;
 	Oid			reloid = RelationGetRelid(rel);
 
@@ -3598,9 +3599,10 @@ SetRelationTableSpace(Relation rel,
 	/* Get a modifiable copy of the relation's pg_class row. */
 	pg_class = table_open(RelationRelationId, RowExclusiveLock);
 
-	tuple = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(reloid));
+	tuple = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(reloid));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "cache lookup failed for relation %u", reloid);
+	otid = tuple->t_self;
 	rd_rel = (Form_pg_class) GETSTRUCT(tuple);
 
 	/* Update the pg_class row. */
@@ -3608,7 +3610,8 @@ SetRelationTableSpace(Relation rel,
 		InvalidOid : newTableSpaceId;
 	if (RelFileNumberIsValid(newRelFilenumber))
 		rd_rel->relfilenode = newRelFilenumber;
-	CatalogTupleUpdate(pg_class, &tuple->t_self, tuple);
+	CatalogTupleUpdate(pg_class, &otid, tuple);
+	UnlockTuple(pg_class, &otid, InplaceUpdateTupleLock);
 
 	/*
 	 * Record dependency on tablespace.  This is only required for relations
@@ -4102,6 +4105,7 @@ RenameRelationInternal(Oid myrelid, const char *newrelname, bool is_internal, bo
 {
 	Relation	targetrelation;
 	Relation	relrelation;	/* for RELATION relation */
+	ItemPointerData otid;
 	HeapTuple	reltup;
 	Form_pg_class relform;
 	Oid			namespaceId;
@@ -4124,7 +4128,8 @@ RenameRelationInternal(Oid myrelid, const char *newrelname, bool is_internal, bo
 	 */
 	relrelation = table_open(RelationRelationId, RowExclusiveLock);
 
-	reltup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(myrelid));
+	reltup = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(myrelid));
+	otid = reltup->t_self;
 	if (!HeapTupleIsValid(reltup))	/* shouldn't happen */
 		elog(ERROR, "cache lookup failed for relation %u", myrelid);
 	relform = (Form_pg_class) GETSTRUCT(reltup);
@@ -4151,7 +4156,8 @@ RenameRelationInternal(Oid myrelid, const char *newrelname, bool is_internal, bo
 	 */
 	namestrcpy(&(relform->relname), newrelname);
 
-	CatalogTupleUpdate(relrelation, &reltup->t_self, reltup);
+	CatalogTupleUpdate(relrelation, &otid, reltup);
+	UnlockTuple(relrelation, &otid, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHookArg(RelationRelationId, myrelid, 0,
 								 InvalidOid, is_internal);
@@ -14926,7 +14932,7 @@ ATExecSetRelOptions(Relation rel, List *defList, AlterTableType operation,
 
 	/* Fetch heap tuple */
 	relid = RelationGetRelid(rel);
-	tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
+	tuple = SearchSysCacheLocked1(RELOID, ObjectIdGetDatum(relid));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "cache lookup failed for relation %u", relid);
 
@@ -15030,6 +15036,7 @@ ATExecSetRelOptions(Relation rel, List *defList, AlterTableType operation,
 								 repl_val, repl_null, repl_repl);
 
 	CatalogTupleUpdate(pgclass, &newtuple->t_self, newtuple);
+	UnlockTuple(pgclass, &tuple->t_self, InplaceUpdateTupleLock);
 
 	InvokeObjectPostAlterHook(RelationRelationId, RelationGetRelid(rel), 0);
 
@@ -17179,7 +17186,8 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 	ObjectAddress thisobj;
 	bool		already_done = false;
 
-	classTup = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relOid));
+	/* no rel lock for relkind=c so use LOCKTAG_TUPLE */
+	classTup = SearchSysCacheLockedCopy1(RELOID, ObjectIdGetDatum(relOid));
 	if (!HeapTupleIsValid(classTup))
 		elog(ERROR, "cache lookup failed for relation %u", relOid);
 	classForm = (Form_pg_class) GETSTRUCT(classTup);
@@ -17198,6 +17206,8 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 	already_done = object_address_present(&thisobj, objsMoved);
 	if (!already_done && oldNspOid != newNspOid)
 	{
+		ItemPointerData otid = classTup->t_self;
+
 		/* check for duplicate name (more friendly than unique-index failure) */
 		if (get_relname_relid(NameStr(classForm->relname),
 							  newNspOid) != InvalidOid)
@@ -17210,7 +17220,9 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 		/* classTup is a copy, so OK to scribble on */
 		classForm->relnamespace = newNspOid;
 
-		CatalogTupleUpdate(classRel, &classTup->t_self, classTup);
+		CatalogTupleUpdate(classRel, &otid, classTup);
+		UnlockTuple(classRel, &otid, InplaceUpdateTupleLock);
+
 
 		/* Update dependency on schema if caller said so */
 		if (hasDependEntry &&
@@ -17222,6 +17234,8 @@ AlterRelationNamespaceInternal(Relation classRel, Oid relOid,
 			elog(ERROR, "could not change schema dependency for relation \"%s\"",
 				 NameStr(classForm->relname));
 	}
+	else
+		UnlockTuple(classRel, &classTup->t_self, InplaceUpdateTupleLock);
 	if (!already_done)
 	{
 		add_exact_object_address(&thisobj, objsMoved);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 29e186f..f880f90 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1204,6 +1204,8 @@ InitResultRelInfo(ResultRelInfo *resultRelInfo,
 	resultRelInfo->ri_NumIndices = 0;
 	resultRelInfo->ri_IndexRelationDescs = NULL;
 	resultRelInfo->ri_IndexRelationInfo = NULL;
+	resultRelInfo->ri_needLockTagTuple =
+		IsInplaceUpdateRelation(resultRelationDesc);
 	/* make a copy so as not to depend on relcache info not changing... */
 	resultRelInfo->ri_TrigDesc = CopyTriggerDesc(resultRelationDesc->trigdesc);
 	if (resultRelInfo->ri_TrigDesc)
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 1086cbc..54025c9 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -661,8 +661,12 @@ ExecSimpleRelationUpdate(ResultRelInfo *resultRelInfo,
 	Relation	rel = resultRelInfo->ri_RelationDesc;
 	ItemPointer tid = &(searchslot->tts_tid);
 
-	/* For now we support only tables. */
+	/*
+	 * We support only non-system tables, with
+	 * check_publication_add_relation() accountable.
+	 */
 	Assert(rel->rd_rel->relkind == RELKIND_RELATION);
+	Assert(!IsCatalogRelation(rel));
 
 	CheckCmdReplicaIdentity(rel, CMD_UPDATE);
 
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 8bf4c80..1161520 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -2324,6 +2324,8 @@ ExecUpdate(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 	}
 	else
 	{
+		ItemPointerData lockedtid;
+
 		/*
 		 * If we generate a new candidate tuple after EvalPlanQual testing, we
 		 * must loop back here to try again.  (We don't need to redo triggers,
@@ -2332,6 +2334,7 @@ ExecUpdate(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 		 * to do them again.)
 		 */
 redo_act:
+		lockedtid = *tupleid;
 		result = ExecUpdateAct(context, resultRelInfo, tupleid, oldtuple, slot,
 							   canSetTag, &updateCxt);
 
@@ -2425,6 +2428,14 @@ redo_act:
 								ExecInitUpdateProjection(context->mtstate,
 														 resultRelInfo);
 
+							if (resultRelInfo->ri_needLockTagTuple)
+							{
+								UnlockTuple(resultRelationDesc,
+											&lockedtid, InplaceUpdateTupleLock);
+								LockTuple(resultRelationDesc,
+										  tupleid, InplaceUpdateTupleLock);
+							}
+
 							/* Fetch the most recent version of old tuple. */
 							oldSlot = resultRelInfo->ri_oldTupleSlot;
 							if (!table_tuple_fetch_row_version(resultRelationDesc,
@@ -2529,6 +2540,14 @@ ExecOnConflictUpdate(ModifyTableContext *context,
 	TransactionId xmin;
 	bool		isnull;
 
+	/*
+	 * Parse analysis should have blocked ON CONFLICT for all system
+	 * relations, which includes these.  There's no fundamental obstacle to
+	 * supporting this; we'd just need to handle LOCKTAG_TUPLE like the other
+	 * ExecUpdate() caller.
+	 */
+	Assert(!resultRelInfo->ri_needLockTagTuple);
+
 	/* Determine lock mode to use */
 	lockmode = ExecUpdateLockMode(context->estate, resultRelInfo);
 
@@ -2854,6 +2873,7 @@ ExecMergeMatched(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 {
 	ModifyTableState *mtstate = context->mtstate;
 	List	  **mergeActions = resultRelInfo->ri_MergeActions;
+	ItemPointerData lockedtid;
 	List	   *actionStates;
 	TupleTableSlot *newslot = NULL;
 	TupleTableSlot *rslot = NULL;
@@ -2890,14 +2910,32 @@ ExecMergeMatched(ModifyTableContext *context, ResultRelInfo *resultRelInfo,
 	 * target wholerow junk attr.
 	 */
 	Assert(tupleid != NULL || oldtuple != NULL);
+	ItemPointerSetInvalid(&lockedtid);
 	if (oldtuple != NULL)
+	{
+		Assert(!resultRelInfo->ri_needLockTagTuple);
 		ExecForceStoreHeapTuple(oldtuple, resultRelInfo->ri_oldTupleSlot,
 								false);
-	else if (!table_tuple_fetch_row_version(resultRelInfo->ri_RelationDesc,
-											tupleid,
-											SnapshotAny,
-											resultRelInfo->ri_oldTupleSlot))
-		elog(ERROR, "failed to fetch the target tuple");
+	}
+	else
+	{
+		if (resultRelInfo->ri_needLockTagTuple)
+		{
+			/*
+			 * This locks even for CMD_DELETE, for CMD_NOTHING, and for tuples
+			 * that don't match mas_whenqual.  MERGE on system catalogs is a
+			 * minor use case, so don't bother optimizing those.
+			 */
+			LockTuple(resultRelInfo->ri_RelationDesc, tupleid,
+					  InplaceUpdateTupleLock);
+			lockedtid = *tupleid;
+		}
+		if (!table_tuple_fetch_row_version(resultRelInfo->ri_RelationDesc,
+										   tupleid,
+										   SnapshotAny,
+										   resultRelInfo->ri_oldTupleSlot))
+			elog(ERROR, "failed to fetch the target tuple");
+	}
 
 	/*
 	 * Test the join condition.  If it's satisfied, perform a MATCHED action.
@@ -2969,7 +3007,7 @@ lmerge_matched:
 										tupleid, NULL, newslot, &result))
 				{
 					if (result == TM_Ok)
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 
 					break;		/* concurrent update/delete */
 				}
@@ -2980,11 +3018,11 @@ lmerge_matched:
 				{
 					if (!ExecIRUpdateTriggers(estate, resultRelInfo,
 											  oldtuple, newslot))
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 				}
 				else
 				{
-					/* called table_tuple_fetch_row_version() above */
+					/* checked ri_needLockTagTuple above */
 					Assert(oldtuple == NULL);
 
 					result = ExecUpdateAct(context, resultRelInfo, tupleid,
@@ -3003,7 +3041,8 @@ lmerge_matched:
 					if (updateCxt.crossPartUpdate)
 					{
 						mtstate->mt_merge_updated += 1;
-						return context->cpUpdateReturningSlot;
+						rslot = context->cpUpdateReturningSlot;
+						goto out;
 					}
 				}
 
@@ -3021,7 +3060,7 @@ lmerge_matched:
 										NULL, NULL, &result))
 				{
 					if (result == TM_Ok)
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 
 					break;		/* concurrent update/delete */
 				}
@@ -3032,11 +3071,11 @@ lmerge_matched:
 				{
 					if (!ExecIRDeleteTriggers(estate, resultRelInfo,
 											  oldtuple))
-						return NULL;	/* "do nothing" */
+						goto out;	/* "do nothing" */
 				}
 				else
 				{
-					/* called table_tuple_fetch_row_version() above */
+					/* checked ri_needLockTagTuple above */
 					Assert(oldtuple == NULL);
 
 					result = ExecDeleteAct(context, resultRelInfo, tupleid,
@@ -3118,7 +3157,7 @@ lmerge_matched:
 				 * let caller handle it under NOT MATCHED [BY TARGET] clauses.
 				 */
 				*matched = false;
-				return NULL;
+				goto out;
 
 			case TM_Updated:
 				{
@@ -3192,7 +3231,7 @@ lmerge_matched:
 								 * more to do.
 								 */
 								if (TupIsNull(epqslot))
-									return NULL;
+									goto out;
 
 								/*
 								 * If we got a NULL ctid from the subplan, the
@@ -3210,6 +3249,15 @@ lmerge_matched:
 								 * we need to switch to the NOT MATCHED BY
 								 * SOURCE case.
 								 */
+								if (resultRelInfo->ri_needLockTagTuple)
+								{
+									if (ItemPointerIsValid(&lockedtid))
+										UnlockTuple(resultRelInfo->ri_RelationDesc, &lockedtid,
+													InplaceUpdateTupleLock);
+									LockTuple(resultRelInfo->ri_RelationDesc, &context->tmfd.ctid,
+											  InplaceUpdateTupleLock);
+									lockedtid = context->tmfd.ctid;
+								}
 								if (!table_tuple_fetch_row_version(resultRelationDesc,
 																   &context->tmfd.ctid,
 																   SnapshotAny,
@@ -3238,7 +3286,7 @@ lmerge_matched:
 							 * MATCHED [BY TARGET] actions
 							 */
 							*matched = false;
-							return NULL;
+							goto out;
 
 						case TM_SelfModified:
 
@@ -3266,13 +3314,13 @@ lmerge_matched:
 
 							/* This shouldn't happen */
 							elog(ERROR, "attempted to update or delete invisible tuple");
-							return NULL;
+							goto out;
 
 						default:
 							/* see table_tuple_lock call in ExecDelete() */
 							elog(ERROR, "unexpected table_tuple_lock status: %u",
 								 result);
-							return NULL;
+							goto out;
 					}
 				}
 
@@ -3319,6 +3367,10 @@ lmerge_matched:
 	/*
 	 * Successfully executed an action or no qualifying action was found.
 	 */
+out:
+	if (ItemPointerIsValid(&lockedtid))
+		UnlockTuple(resultRelInfo->ri_RelationDesc, &lockedtid,
+					InplaceUpdateTupleLock);
 	return rslot;
 }
 
@@ -3770,6 +3822,7 @@ ExecModifyTable(PlanState *pstate)
 	HeapTupleData oldtupdata;
 	HeapTuple	oldtuple;
 	ItemPointer tupleid;
+	bool		tuplock;
 
 	CHECK_FOR_INTERRUPTS();
 
@@ -4082,6 +4135,8 @@ ExecModifyTable(PlanState *pstate)
 				break;
 
 			case CMD_UPDATE:
+				tuplock = false;
+
 				/* Initialize projection info if first time for this table */
 				if (unlikely(!resultRelInfo->ri_projectNewInfoValid))
 					ExecInitUpdateProjection(node, resultRelInfo);
@@ -4093,6 +4148,7 @@ ExecModifyTable(PlanState *pstate)
 				oldSlot = resultRelInfo->ri_oldTupleSlot;
 				if (oldtuple != NULL)
 				{
+					Assert(!resultRelInfo->ri_needLockTagTuple);
 					/* Use the wholerow junk attr as the old tuple. */
 					ExecForceStoreHeapTuple(oldtuple, oldSlot, false);
 				}
@@ -4101,6 +4157,11 @@ ExecModifyTable(PlanState *pstate)
 					/* Fetch the most recent version of old tuple. */
 					Relation	relation = resultRelInfo->ri_RelationDesc;
 
+					if (resultRelInfo->ri_needLockTagTuple)
+					{
+						LockTuple(relation, tupleid, InplaceUpdateTupleLock);
+						tuplock = true;
+					}
 					if (!table_tuple_fetch_row_version(relation, tupleid,
 													   SnapshotAny,
 													   oldSlot))
@@ -4112,6 +4173,9 @@ ExecModifyTable(PlanState *pstate)
 				/* Now apply the update. */
 				slot = ExecUpdate(&context, resultRelInfo, tupleid, oldtuple,
 								  slot, node->canSetTag);
+				if (tuplock)
+					UnlockTuple(resultRelInfo->ri_RelationDesc, tupleid,
+								InplaceUpdateTupleLock);
 				break;
 
 			case CMD_DELETE:
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 66ed24e..5abb97c 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -3768,6 +3768,7 @@ RelationSetNewRelfilenumber(Relation relation, char persistence)
 {
 	RelFileNumber newrelfilenumber;
 	Relation	pg_class;
+	ItemPointerData otid;
 	HeapTuple	tuple;
 	Form_pg_class classform;
 	MultiXactId minmulti = InvalidMultiXactId;
@@ -3810,11 +3811,12 @@ RelationSetNewRelfilenumber(Relation relation, char persistence)
 	 */
 	pg_class = table_open(RelationRelationId, RowExclusiveLock);
 
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(relation)));
+	tuple = SearchSysCacheLockedCopy1(RELOID,
+									  ObjectIdGetDatum(RelationGetRelid(relation)));
 	if (!HeapTupleIsValid(tuple))
 		elog(ERROR, "could not find tuple for relation %u",
 			 RelationGetRelid(relation));
+	otid = tuple->t_self;
 	classform = (Form_pg_class) GETSTRUCT(tuple);
 
 	/*
@@ -3934,9 +3936,10 @@ RelationSetNewRelfilenumber(Relation relation, char persistence)
 		classform->relminmxid = minmulti;
 		classform->relpersistence = persistence;
 
-		CatalogTupleUpdate(pg_class, &tuple->t_self, tuple);
+		CatalogTupleUpdate(pg_class, &otid, tuple);
 	}
 
+	UnlockTuple(pg_class, &otid, InplaceUpdateTupleLock);
 	heap_freetuple(tuple);
 
 	table_close(pg_class, RowExclusiveLock);
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index 3e03dfc..50c9440 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -30,7 +30,10 @@
 #include "catalog/pg_shseclabel_d.h"
 #include "common/int.h"
 #include "lib/qunique.h"
+#include "miscadmin.h"
+#include "storage/lmgr.h"
 #include "utils/catcache.h"
+#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/rel.h"
 #include "utils/syscache.h"
@@ -269,6 +272,98 @@ ReleaseSysCache(HeapTuple tuple)
 }
 
 /*
+ * SearchSysCacheLocked1
+ *
+ * Combine SearchSysCache1() with acquiring a LOCKTAG_TUPLE at mode
+ * InplaceUpdateTupleLock.  This is a tool for complying with the
+ * README.tuplock section "Locking to write inplace-updated tables".  After
+ * the caller's heap_update(), it should UnlockTuple(InplaceUpdateTupleLock)
+ * and ReleaseSysCache().
+ *
+ * The returned tuple may be the subject of an uncommitted update, so this
+ * doesn't prevent the "tuple concurrently updated" error.
+ */
+HeapTuple
+SearchSysCacheLocked1(int cacheId,
+					  Datum key1)
+{
+	ItemPointerData tid;
+	LOCKTAG		tag;
+	Oid			dboid =
+		SysCache[cacheId]->cc_relisshared ? InvalidOid : MyDatabaseId;
+	Oid			reloid = cacheinfo[cacheId].reloid;
+
+	/*----------
+	 * Since inplace updates may happen just before our LockTuple(), we must
+	 * return content acquired after LockTuple() of the TID we return.  If we
+	 * just fetched twice instead of looping, the following sequence would
+	 * defeat our locking:
+	 *
+	 * GRANT:   SearchSysCache1() = TID (1,5)
+	 * GRANT:   LockTuple(pg_class, (1,5))
+	 * [no more inplace update of (1,5) until we release the lock]
+	 * CLUSTER: SearchSysCache1() = TID (1,5)
+	 * CLUSTER: heap_update() = TID (1,8)
+	 * CLUSTER: COMMIT
+	 * GRANT:   SearchSysCache1() = TID (1,8)
+	 * GRANT:   return (1,8) from SearchSysCacheLocked1()
+	 * VACUUM:  SearchSysCache1() = TID (1,8)
+	 * VACUUM:  LockTuple(pg_class, (1,8))  # two TIDs now locked for one rel
+	 * VACUUM:  inplace update
+	 * GRANT:   heap_update() = (1,9)  # lose inplace update
+	 *
+	 * In the happy case, this takes two fetches, one to determine the TID to
+	 * lock and another to get the content and confirm the TID didn't change.
+	 *
+	 * This is valid even if the row gets updated to a new TID, the old TID
+	 * becomes LP_UNUSED, and the row gets updated back to its old TID.  We'd
+	 * still hold the right LOCKTAG_TUPLE and a copy of the row captured after
+	 * the LOCKTAG_TUPLE.
+	 */
+	ItemPointerSetInvalid(&tid);
+	for (;;)
+	{
+		HeapTuple	tuple;
+		LOCKMODE	lockmode = InplaceUpdateTupleLock;
+
+		tuple = SearchSysCache1(cacheId, key1);
+		if (ItemPointerIsValid(&tid))
+		{
+			if (!HeapTupleIsValid(tuple))
+			{
+				LockRelease(&tag, lockmode, false);
+				return tuple;
+			}
+			if (ItemPointerEquals(&tid, &tuple->t_self))
+				return tuple;
+			LockRelease(&tag, lockmode, false);
+		}
+		else if (!HeapTupleIsValid(tuple))
+			return tuple;
+
+		tid = tuple->t_self;
+		ReleaseSysCache(tuple);
+		/* like: LockTuple(rel, &tid, lockmode) */
+		SET_LOCKTAG_TUPLE(tag, dboid, reloid,
+						  ItemPointerGetBlockNumber(&tid),
+						  ItemPointerGetOffsetNumber(&tid));
+		(void) LockAcquire(&tag, lockmode, false, false);
+
+		/*
+		 * 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.
+		 *
+		 * If a heap_update() call just released its LOCKTAG_TUPLE, we'll
+		 * probably find the old tuple and reach "tuple concurrently updated".
+		 * If that heap_update() aborts, our LOCKTAG_TUPLE blocks inplace
+		 * updates while our caller works.
+		 */
+		AcceptInvalidationMessages();
+	}
+}
+
+/*
  * SearchSysCacheCopy
  *
  * A convenience routine that does SearchSysCache and (if successful)
@@ -295,6 +390,28 @@ SearchSysCacheCopy(int cacheId,
 }
 
 /*
+ * SearchSysCacheLockedCopy1
+ *
+ * Meld SearchSysCacheLockedCopy1 with SearchSysCacheCopy().  After the
+ * caller's heap_update(), it should UnlockTuple(InplaceUpdateTupleLock) and
+ * heap_freetuple().
+ */
+HeapTuple
+SearchSysCacheLockedCopy1(int cacheId,
+						  Datum key1)
+{
+	HeapTuple	tuple,
+				newtuple;
+
+	tuple = SearchSysCacheLocked1(cacheId, key1);
+	if (!HeapTupleIsValid(tuple))
+		return tuple;
+	newtuple = heap_copytuple(tuple);
+	ReleaseSysCache(tuple);
+	return newtuple;
+}
+
+/*
  * SearchSysCacheExists
  *
  * A convenience routine that just probes to see if a tuple can be found.
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index af7d8fd..b078a6e 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -482,6 +482,9 @@ typedef struct ResultRelInfo
 	/* Have the projection and the slots above been initialized? */
 	bool		ri_projectNewInfoValid;
 
+	/* updates do LockTuple() before oldtup read; see README.tuplock */
+	bool		ri_needLockTagTuple;
+
 	/* triggers to be fired, if any */
 	TriggerDesc *ri_TrigDesc;
 
diff --git a/src/include/storage/lockdefs.h b/src/include/storage/lockdefs.h
index 934ba84..810b297 100644
--- a/src/include/storage/lockdefs.h
+++ b/src/include/storage/lockdefs.h
@@ -47,6 +47,8 @@ typedef int LOCKMODE;
 
 #define MaxLockMode				8	/* highest standard lock mode */
 
+/* See README.tuplock section "Locking to write inplace-updated tables" */
+#define InplaceUpdateTupleLock ExclusiveLock
 
 /* WAL representation of an AccessExclusiveLock on a table */
 typedef struct xl_standby_lock
diff --git a/src/include/utils/syscache.h b/src/include/utils/syscache.h
index 03a27dd..b541911 100644
--- a/src/include/utils/syscache.h
+++ b/src/include/utils/syscache.h
@@ -43,9 +43,14 @@ extern HeapTuple SearchSysCache4(int cacheId,
 
 extern void ReleaseSysCache(HeapTuple tuple);
 
+extern HeapTuple SearchSysCacheLocked1(int cacheId,
+									   Datum key1);
+
 /* convenience routines */
 extern HeapTuple SearchSysCacheCopy(int cacheId,
 									Datum key1, Datum key2, Datum key3, Datum key4);
+extern HeapTuple SearchSysCacheLockedCopy1(int cacheId,
+										   Datum key1);
 extern bool SearchSysCacheExists(int cacheId,
 								 Datum key1, Datum key2, Datum key3, Datum key4);
 extern Oid	GetSysCacheOid(int cacheId, AttrNumber oidcol,
diff --git a/src/test/isolation/expected/intra-grant-inplace.out b/src/test/isolation/expected/intra-grant-inplace.out
index fe26984..b5fe8b0 100644
--- a/src/test/isolation/expected/intra-grant-inplace.out
+++ b/src/test/isolation/expected/intra-grant-inplace.out
@@ -100,7 +100,7 @@ f
 step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
 step c2: COMMIT;
 
-starting permutation: b3 sfu3 b1 grant1 read2 as3 addk2 r3 c1 read2
+starting permutation: b3 sfu3 b1 grant1 read2 addk2 r3 c1 read2
 step b3: BEGIN ISOLATION LEVEL READ COMMITTED;
 step sfu3: 
 	SELECT relhasindex FROM pg_class
@@ -124,7 +124,6 @@ relhasindex
 f          
 (1 row)
 
-step as3: LOCK TABLE intra_grant_inplace IN ACCESS SHARE MODE;
 step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
 step r3: ROLLBACK;
 step grant1: <... completed>
@@ -155,9 +154,11 @@ step b1: BEGIN;
 step grant1: 
 	GRANT SELECT ON intra_grant_inplace TO PUBLIC;
  <waiting ...>
-step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c);
-step c2: COMMIT;
+step addk2: ALTER TABLE intra_grant_inplace ADD PRIMARY KEY (c); <waiting ...>
+step addk2: <... completed>
+ERROR:  deadlock detected
 step grant1: <... completed>
+step c2: COMMIT;
 step c1: COMMIT;
 step read2: 
 	SELECT relhasindex FROM pg_class
@@ -195,9 +196,8 @@ relhasindex
 f          
 (1 row)
 
-s4: WARNING:  got: tuple concurrently updated
-step revoke4: <... completed>
 step r3: ROLLBACK;
+step revoke4: <... completed>
 
 starting permutation: b1 drop1 b3 sfu3 revoke4 c1 r3
 step b1: BEGIN;
@@ -224,6 +224,6 @@ relhasindex
 -----------
 (0 rows)
 
-s4: WARNING:  got: tuple concurrently deleted
+s4: WARNING:  got: cache lookup failed for relation REDACTED
 step revoke4: <... completed>
 step r3: ROLLBACK;
diff --git a/src/test/isolation/specs/eval-plan-qual.spec b/src/test/isolation/specs/eval-plan-qual.spec
index 3a74406..07307e6 100644
--- a/src/test/isolation/specs/eval-plan-qual.spec
+++ b/src/test/isolation/specs/eval-plan-qual.spec
@@ -194,7 +194,7 @@ step simplepartupdate_noroute {
 	update parttbl set b = 2 where c = 1 returning *;
 }
 
-# test system class updates
+# test system class LockTuple()
 
 step sys1	{
 	UPDATE pg_class SET reltuples = 123 WHERE oid = 'accounts'::regclass;
diff --git a/src/test/isolation/specs/intra-grant-inplace.spec b/src/test/isolation/specs/intra-grant-inplace.spec
index d07ed3b..2992c85 100644
--- a/src/test/isolation/specs/intra-grant-inplace.spec
+++ b/src/test/isolation/specs/intra-grant-inplace.spec
@@ -14,6 +14,7 @@ teardown
 
 # heap_update()
 session s1
+setup	{ SET deadlock_timeout = '100s'; }
 step b1	{ BEGIN; }
 step grant1	{
 	GRANT SELECT ON intra_grant_inplace TO PUBLIC;
@@ -25,6 +26,7 @@ step c1	{ COMMIT; }
 
 # inplace update
 session s2
+setup	{ SET deadlock_timeout = '10ms'; }
 step read2	{
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass;
@@ -48,7 +50,6 @@ step sfu3	{
 	SELECT relhasindex FROM pg_class
 	WHERE oid = 'intra_grant_inplace'::regclass FOR UPDATE;
 }
-step as3	{ LOCK TABLE intra_grant_inplace IN ACCESS SHARE MODE; }
 step r3	{ ROLLBACK; }
 
 # Additional heap_update()
@@ -74,8 +75,6 @@ step keyshr5	{
 teardown	{ ROLLBACK; }
 
 
-# XXX extant bugs: permutation comments refer to planned future LockTuple()
-
 permutation
 	b1
 	grant1
@@ -118,7 +117,6 @@ permutation
 	b1
 	grant1(r3)	# acquire LockTuple(), await sfu3 xmax
 	read2
-	as3			# XXX temporary until patch adds locking to addk2
 	addk2(c1)	# block in LockTuple() behind grant1
 	r3			# unblock grant1; addk2 now awaits grant1 xmax
 	c1
@@ -128,8 +126,8 @@ permutation
 	b2
 	sfnku2
 	b1
-	grant1(c2)		# acquire LockTuple(), await sfnku2 xmax
-	addk2			# block in LockTuple() behind grant1 = deadlock
+	grant1(addk2)	# acquire LockTuple(), await sfnku2 xmax
+	addk2(*)		# block in LockTuple() behind grant1 = deadlock
 	c2
 	c1
 	read2
@@ -140,7 +138,7 @@ permutation
 	grant1
 	b3
 	sfu3(c1)	# acquire LockTuple(), await grant1 xmax
-	revoke4(sfu3)	# block in LockTuple() behind sfu3
+	revoke4(r3)	# block in LockTuple() behind sfu3
 	c1
 	r3			# revoke4 unlocks old tuple and finds new
 
inplace160-inval-durability-inplace-v4.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Core code no longer needs XLOG_INVALIDATIONS, but this leaves removing
    it for future work.  Extensions could be relying on that mechanism, so
    that removal would not be back-patch material.  Back-patch to v12 (all
    supported versions).
    
    Reviewed by FIXME and (in earlier versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 7de60c1..da6cff4 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6344,6 +6344,9 @@ heap_inplace_update(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;
@@ -6351,6 +6354,29 @@ heap_inplace_update(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();
 
@@ -6380,9 +6406,16 @@ heap_inplace_update(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);
@@ -6394,17 +6427,28 @@ heap_inplace_update(Relation relation,
 		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
-	END_CRIT_SECTION();
-
-	heap_inplace_unlock(relation, oldtup, buffer);
+	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 
 	/*
-	 * 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.
+	 * Send invalidations to shared queue.  SearchSysCacheLocked1() assumes we
+	 * do this before UnlockTuple().
 	 *
-	 * XXX ROLLBACK discards the invalidation.  See test inplace-inval.spec.
+	 * 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);
+
+	AcceptInvalidationMessages();	/* local processing of just-sent inval */
+
+	/*
+	 * 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);
@@ -10151,6 +10195,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 5f5673e..f31cc3a 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 25f870b..32e509a 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 dfc8cf2..7b6e0aa 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1368,14 +1368,23 @@ 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 formerly did so, they now send
+		 * immediate invalidations.  Extensions might still do so, 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 e4608b9..c3ac8f3 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2890,12 +2890,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 05a6de6..a586d24 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 d687cee..e73576a 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 492a033..f1e970b 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2288,7 +2288,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;
@@ -2329,7 +2330,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)
 		{
@@ -2338,7 +2339,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 603aa41..b3d3adb 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;
 
-	/* Events emitted by previous commands of this (sub)transaction */
-	InvalidationMsgsGroup PriorCmdInvalidMsgs;
-
 	/* 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;
+
+	/* 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,
@@ -1038,16 +1139,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
@@ -1058,6 +1159,45 @@ AtEOXact_Inval(bool isCommit)
 
 	/* Need not free anything explicitly */
 	transInvalInfo = NULL;
+	inplaceInvalInfo = NULL;
+}
+
+/*
+ * 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;
 }
 
 /*
@@ -1125,18 +1265,21 @@ 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;
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Successful inplace update must clear this. */
+		Assert(inplaceInvalInfo == NULL);
 	}
 	else
 	{
@@ -1148,6 +1291,9 @@ AtEOSubXact_Inval(bool isCommit)
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Reset from aborted inplace update. */
+		inplaceInvalInfo = NULL;
 	}
 }
 
@@ -1177,7 +1323,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 +1331,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 +1369,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 +1379,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 +1457,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 +1513,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1362,15 +1537,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 +1557,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1399,14 +1572,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 +1592,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 +1781,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 50c9440..f41b1c2 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".
diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 42736f3..4591e9a 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -20,6 +20,7 @@
 #include "storage/buf.h"
 #include "storage/bufpage.h"
 #include "storage/relfilelocator.h"
+#include "storage/sinval.h"
 #include "utils/relcache.h"
 
 
@@ -425,9 +426,14 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
+	Oid			dbId;			/* MyDatabaseId */
+	Oid			tsId;			/* MyDatabaseTableSpace */
+	bool		relcacheInitFileInval;	/* invalidate relcache init files */
+	int			nmsgs;			/* number of shared inval msgs */
+	SharedInvalidationMessage msgs[FLEXIBLE_ARRAY_MEMBER];
 } xl_heap_inplace;
 
-#define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
+#define MinSizeOfHeapInplace	(offsetof(xl_heap_inplace, nmsgs) + sizeof(int))
 
 /*
  * This is what we need to know about setting a visibility map bit
diff --git a/src/include/storage/sinval.h b/src/include/storage/sinval.h
index 8f5744b..c812237 100644
--- a/src/include/storage/sinval.h
+++ b/src/include/storage/sinval.h
@@ -144,6 +144,8 @@ extern void ProcessCatchupInterrupt(void);
 
 extern int	xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 												 bool *RelcacheInitFileInval);
+extern int	inplaceGetInvalidationMessages(SharedInvalidationMessage **msgs,
+										   bool *RelcacheInitFileInval);
 extern void ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 												 int nmsgs, bool RelcacheInitFileInval,
 												 Oid dbid, Oid tsid);
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..3390e7a 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 6d424c8..13dd553 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1256,6 +1256,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
#11Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#10)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

Rebased.

Attachments:

inplace160-inval-durability-inplace-v5.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Core code no longer needs XLOG_INVALIDATIONS, but this leaves removing
    it for future work.  Extensions could be relying on that mechanism, so
    that removal would not be back-patch material.  Back-patch to v12 (all
    supported versions).
    
    Reviewed by FIXME and (in earlier versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index da5e656..7c82a95 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6327,6 +6327,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;
@@ -6334,6 +6337,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();
 
@@ -6363,9 +6389,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);
@@ -6377,17 +6410,28 @@ heap_inplace_update_and_unlock(Relation relation,
 		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
-	END_CRIT_SECTION();
-
-	heap_inplace_unlock(relation, oldtup, buffer);
+	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 
 	/*
-	 * 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.
+	 * Send invalidations to shared queue.  SearchSysCacheLocked1() assumes we
+	 * do this before UnlockTuple().
 	 *
-	 * XXX ROLLBACK discards the invalidation.  See test inplace-inval.spec.
+	 * 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);
+
+	AcceptInvalidationMessages();	/* local processing of just-sent inval */
+
+	/*
+	 * 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 6dae723..c5208f3 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 5f5673e..f31cc3a 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 25f870b..32e509a 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 87700c7..9eb7db4 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1368,14 +1368,23 @@ 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 formerly did so, they now send
+		 * immediate invalidations.  Extensions might still do so, 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 6084dfa..36f5bba 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 05a6de6..a586d24 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 d687cee..e73576a 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 10276aa..ee303dc 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 603aa41..b3d3adb 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;
 
-	/* Events emitted by previous commands of this (sub)transaction */
-	InvalidationMsgsGroup PriorCmdInvalidMsgs;
-
 	/* 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;
+
+	/* 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,
@@ -1038,16 +1139,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
@@ -1058,6 +1159,45 @@ AtEOXact_Inval(bool isCommit)
 
 	/* Need not free anything explicitly */
 	transInvalInfo = NULL;
+	inplaceInvalInfo = NULL;
+}
+
+/*
+ * 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;
 }
 
 /*
@@ -1125,18 +1265,21 @@ 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;
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Successful inplace update must clear this. */
+		Assert(inplaceInvalInfo == NULL);
 	}
 	else
 	{
@@ -1148,6 +1291,9 @@ AtEOSubXact_Inval(bool isCommit)
 
 		/* Need not free anything else explicitly */
 		pfree(myInfo);
+
+		/* Reset from aborted inplace update. */
+		inplaceInvalInfo = NULL;
 	}
 }
 
@@ -1177,7 +1323,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 +1331,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 +1369,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 +1379,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 +1457,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 +1513,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1362,15 +1537,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 +1557,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1399,14 +1572,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 +1592,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 +1781,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 50c9440..f41b1c2 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".
diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 42736f3..4591e9a 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -20,6 +20,7 @@
 #include "storage/buf.h"
 #include "storage/bufpage.h"
 #include "storage/relfilelocator.h"
+#include "storage/sinval.h"
 #include "utils/relcache.h"
 
 
@@ -425,9 +426,14 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
+	Oid			dbId;			/* MyDatabaseId */
+	Oid			tsId;			/* MyDatabaseTableSpace */
+	bool		relcacheInitFileInval;	/* invalidate relcache init files */
+	int			nmsgs;			/* number of shared inval msgs */
+	SharedInvalidationMessage msgs[FLEXIBLE_ARRAY_MEMBER];
 } xl_heap_inplace;
 
-#define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
+#define MinSizeOfHeapInplace	(offsetof(xl_heap_inplace, nmsgs) + sizeof(int))
 
 /*
  * This is what we need to know about setting a visibility map bit
diff --git a/src/include/storage/sinval.h b/src/include/storage/sinval.h
index 8f5744b..c812237 100644
--- a/src/include/storage/sinval.h
+++ b/src/include/storage/sinval.h
@@ -144,6 +144,8 @@ extern void ProcessCatchupInterrupt(void);
 
 extern int	xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 												 bool *RelcacheInitFileInval);
+extern int	inplaceGetInvalidationMessages(SharedInvalidationMessage **msgs,
+										   bool *RelcacheInitFileInval);
 extern void ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 												 int nmsgs, bool RelcacheInitFileInval,
 												 Oid dbid, Oid tsid);
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..3390e7a 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 5fabb12..85c10da 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1257,6 +1257,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
#12Nitin Motiani
nitinmotiani@google.com
In reply to: Noah Misch (#11)
Re: Inval reliability, especially for inplace updates

On Sat, Oct 12, 2024 at 5:47 PM Noah Misch <noah@leadboat.com> wrote:

Rebased.

Hi,

I have a couple of questions :

1. In heap_inplace_update_and_unlock, currently both buffer and tuple
are unlocked outside the critical section. Why do we have to move the
buffer unlock within the critical section here? My guess is that it
needs to be unlocked for the inplace invals to be processed. But what
is the reasoning behind that?

2. Is there any benefit in CacheInvalidateHeapTupleCommon taking the
preapre_callback argument? Wouldn't it be simpler to just pass an
InvalidationInfo* to the function?

Also is inval-requires-xid-v0.patch planned to be fixed up to inplace160?

Thanks

#13Noah Misch
noah@leadboat.com
In reply to: Nitin Motiani (#12)
Re: Inval reliability, especially for inplace updates

On Sat, Oct 12, 2024 at 06:05:06PM +0530, Nitin Motiani wrote:

1. In heap_inplace_update_and_unlock, currently both buffer and tuple
are unlocked outside the critical section. Why do we have to move the
buffer unlock within the critical section here? My guess is that it
needs to be unlocked for the inplace invals to be processed. But what
is the reasoning behind that?

AtInplace_Inval() acquires SInvalWriteLock. There are two reasons to want to
release the buffer lock before acquiring SInvalWriteLock:

1. Otherwise, we'd need to maintain the invariant that no other part of the
system tries to lock the buffer while holding SInvalWriteLock. (That would
cause an undetected deadlock.)

2. Concurrency is better if we release a no-longer-needed LWLock before doing
something time-consuming, like acquiring another LWLock potentially is.

Inplace invals do need to happen in the critical section, because we've
already written the change to shared buffers, making it the new authoritative
value. If we fail to invalidate, other backends may continue operating with
stale caches.

2. Is there any benefit in CacheInvalidateHeapTupleCommon taking the
preapre_callback argument? Wouldn't it be simpler to just pass an
InvalidationInfo* to the function?

CacheInvalidateHeapTupleCommon() has three conditions that cause it to return
without invoking the callback. Every heap_update() calls
CacheInvalidateHeapTuple(). In typical performance-critical systems, non-DDL
changes dwarf DDL. Hence, the overwhelming majority of heap_update() calls
involve !IsCatalogRelation(). I wouldn't want to allocate InvalidationInfo in
DDL-free transactions. To pass in InvalidationInfo*, I suppose I'd move those
three conditions to a function and make the callers look like:

CacheInvalidateHeapTuple(Relation relation,
HeapTuple tuple,
HeapTuple newtuple)
{
if (NeedsInvalidateHeapTuple(relation))
CacheInvalidateHeapTupleCommon(relation, tuple, newtuple,
PrepareInvalidationState());
}

I don't have a strong preference between that and the callback way.

Also is inval-requires-xid-v0.patch planned to be fixed up to inplace160?

I figure I'll pursue that on a different thread, after inplace160 and
inplace180. If there's cause to pursue it earlier, let me know.

Thanks,
nm

#14Nitin Motiani
nitinmotiani@google.com
In reply to: Noah Misch (#13)
Re: Inval reliability, especially for inplace updates

On Sun, Oct 13, 2024 at 6:15 AM Noah Misch <noah@leadboat.com> wrote:

On Sat, Oct 12, 2024 at 06:05:06PM +0530, Nitin Motiani wrote:

1. In heap_inplace_update_and_unlock, currently both buffer and tuple
are unlocked outside the critical section. Why do we have to move the
buffer unlock within the critical section here? My guess is that it
needs to be unlocked for the inplace invals to be processed. But what
is the reasoning behind that?

AtInplace_Inval() acquires SInvalWriteLock. There are two reasons to want to
release the buffer lock before acquiring SInvalWriteLock:

1. Otherwise, we'd need to maintain the invariant that no other part of the
system tries to lock the buffer while holding SInvalWriteLock. (That would
cause an undetected deadlock.)

2. Concurrency is better if we release a no-longer-needed LWLock before doing
something time-consuming, like acquiring another LWLock potentially is.

Inplace invals do need to happen in the critical section, because we've
already written the change to shared buffers, making it the new authoritative
value. If we fail to invalidate, other backends may continue operating with
stale caches.

Thanks for the clarification.

2. Is there any benefit in CacheInvalidateHeapTupleCommon taking the
preapre_callback argument? Wouldn't it be simpler to just pass an
InvalidationInfo* to the function?

CacheInvalidateHeapTupleCommon() has three conditions that cause it to return
without invoking the callback. Every heap_update() calls
CacheInvalidateHeapTuple(). In typical performance-critical systems, non-DDL
changes dwarf DDL. Hence, the overwhelming majority of heap_update() calls
involve !IsCatalogRelation(). I wouldn't want to allocate InvalidationInfo in
DDL-free transactions. To pass in InvalidationInfo*, I suppose I'd move those
three conditions to a function and make the callers look like:

CacheInvalidateHeapTuple(Relation relation,
HeapTuple tuple,
HeapTuple newtuple)
{
if (NeedsInvalidateHeapTuple(relation))
CacheInvalidateHeapTupleCommon(relation, tuple, newtuple,
PrepareInvalidationState());
}

I don't have a strong preference between that and the callback way.

Thanks. I would have probably done it using the
NeedsInvalidateHeapTuple. But I don't have a strong enough preference
to change it from the callback way. So the current approach seems
good.

Also is inval-requires-xid-v0.patch planned to be fixed up to inplace160?

I figure I'll pursue that on a different thread, after inplace160 and
inplace180. If there's cause to pursue it earlier, let me know.

Sure. Can be done in a different thread.

Thanks,
Nitin Motiani
Google

#15Nitin Motiani
nitinmotiani@google.com
In reply to: Nitin Motiani (#14)
Re: Inval reliability, especially for inplace updates

On Mon, Oct 14, 2024 at 3:15 PM Nitin Motiani <nitinmotiani@google.com> wrote:

On Sun, Oct 13, 2024 at 6:15 AM Noah Misch <noah@leadboat.com> wrote:

On Sat, Oct 12, 2024 at 06:05:06PM +0530, Nitin Motiani wrote:

1. In heap_inplace_update_and_unlock, currently both buffer and tuple
are unlocked outside the critical section. Why do we have to move the
buffer unlock within the critical section here? My guess is that it
needs to be unlocked for the inplace invals to be processed. But what
is the reasoning behind that?

AtInplace_Inval() acquires SInvalWriteLock. There are two reasons to want to
release the buffer lock before acquiring SInvalWriteLock:

1. Otherwise, we'd need to maintain the invariant that no other part of the
system tries to lock the buffer while holding SInvalWriteLock. (That would
cause an undetected deadlock.)

2. Concurrency is better if we release a no-longer-needed LWLock before doing
something time-consuming, like acquiring another LWLock potentially is.

Inplace invals do need to happen in the critical section, because we've
already written the change to shared buffers, making it the new authoritative
value. If we fail to invalidate, other backends may continue operating with
stale caches.

Thanks for the clarification.

2. Is there any benefit in CacheInvalidateHeapTupleCommon taking the
preapre_callback argument? Wouldn't it be simpler to just pass an
InvalidationInfo* to the function?

CacheInvalidateHeapTupleCommon() has three conditions that cause it to return
without invoking the callback. Every heap_update() calls
CacheInvalidateHeapTuple(). In typical performance-critical systems, non-DDL
changes dwarf DDL. Hence, the overwhelming majority of heap_update() calls
involve !IsCatalogRelation(). I wouldn't want to allocate InvalidationInfo in
DDL-free transactions. To pass in InvalidationInfo*, I suppose I'd move those
three conditions to a function and make the callers look like:

CacheInvalidateHeapTuple(Relation relation,
HeapTuple tuple,
HeapTuple newtuple)
{
if (NeedsInvalidateHeapTuple(relation))
CacheInvalidateHeapTupleCommon(relation, tuple, newtuple,
PrepareInvalidationState());
}

I don't have a strong preference between that and the callback way.

Thanks. I would have probably done it using the
NeedsInvalidateHeapTuple. But I don't have a strong enough preference
to change it from the callback way. So the current approach seems
good.

Also is inval-requires-xid-v0.patch planned to be fixed up to inplace160?

I figure I'll pursue that on a different thread, after inplace160 and
inplace180. If there's cause to pursue it earlier, let me know.

Sure. Can be done in a different thread.

I tested the patch locally and it works. And I have no other question
regarding the structure. So this patch looks good to me to commit.

Thanks,
Nitin Motiani
Google

#16Noah Misch
noah@leadboat.com
In reply to: Nitin Motiani (#15)
8 attachment(s)
Re: Inval reliability, especially for inplace updates

With the releases wrapping in 2.5 weeks, I'm ambivalent about pushing this
before the release or after. Pushing before means fewer occurrences of
corruption, but pushing after gives more bake time to discover these changes
were defective. It's hard to predict which helps users more, on a
risk-adjusted basis. I'm leaning toward pushing this week. Opinions?

On Sun, Oct 20, 2024 at 06:41:37PM +0530, Nitin Motiani wrote:

I tested the patch locally and it works. And I have no other question
regarding the structure. So this patch looks good to me to commit.

Thanks. While resolving a back-branch merge conflict, I found
AtEOXact_Inval() and AtEOSubXact_Inval() were skipping inplaceInvalInfo tasks
if transInvalInfo==NULL. If one PreInplace_Inval() failed, the session's next
inplace update would get an assertion failure. Non-assert builds left
inplaceInvalInfo pointing to freed memory, but I didn't find a reachable
malfunction. Separately, the xact.c comment edit wasn't reflecting that v4
brought back the transactional inval. v6 fixes those. Regarding the
back-branch alternatives to the WAL format change:

On Tue, Jun 18, 2024 at 08:23:49AM -0700, Noah Misch wrote:

On Mon, Jun 17, 2024 at 06:57:30PM -0700, Andres Freund wrote:

On 2024-06-17 16:58:54 -0700, Noah Misch wrote:

On Sat, Jun 15, 2024 at 03:37:18PM -0700, Noah Misch wrote:

- heap_xlog_inplace() could set the shared-inval-queue overflow signal on
every backend. This is more wasteful, but inplace updates might be rare
enough (~once per VACUUM) to make it tolerable.

We already set that surprisingly frequently, as
a) The size of the sinval queue is small
b) If a backend is busy, it does not process catchup interrupts
(i.e. executing queries, waiting for a lock prevents processing)
c) There's no deduplication of invals, we often end up sending the same inval
over and over.

So I suspect this might not be too bad, compared to the current badness.

That is good.

I benchmarked that by hacking 027_stream_regress.pl to run "pgbench
--no-vacuum --client=4 -T 30 -b select-only" on the standby, concurrent with
the primary running the regression tests. Standby clients acted on sinval
resetState ~16k times, and pgbench tps decreased 4.5%. That doesn't
necessarily mean the real-life cost would be unacceptable, but it was enough
of a decrease that I switched to the next choice:

We might be able to do the overflow signal once at end of
recovery, like RelationCacheInitFileRemove() does for the init file. That's
mildly harder to reason about, but it would be cheaper. Hmmm.

I'm attaching the branch-specific patches for that and for the main fix.
Other notes from back-patching:

- All branches change the ABI of PrepareToInvalidateCacheTuple(), a function
catcache.c exports for the benefit of inval.c. No PGXN extension calls
that, and I can't think of a use case in extensions.

- Due to v15 commit 3aafc03, the patch for v14 differs to an unusual degree
from the patch for v15+v16. The difference is mostly mechanical, though.

Thanks,
nm

Attachments:

inplace160-inval-durability-inplace-v6_17.patchtext/plain; charset=us-asciiDownload
commit f52d713 (HEAD, zzy_test-commit-REL_17_STABLE)
Author:     Noah Misch <noah@leadboat.com>
AuthorDate: Tue Oct 22 12:59:28 2024 -0700
Commit:     Noah Misch <noah@leadboat.com>
CommitDate: Wed Oct 23 17:46:13 2024 -0700

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Core code no longer needs XLOG_INVALIDATIONS, but this leaves removing
    it for future work.  Extensions could be relying on that mechanism, so
    that removal would not be back-patch material.  Back-patch to v12 (all
    supported versions).  The back branch versions don't change WAL, because
    those branches just added end-of-recovery SIResetAll().
    
    Reviewed by Nitin Motiani and (in earlier versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com
    
    Conflicts:
    	src/backend/access/heap/heapam_xlog.c
    	src/include/access/xlog_internal.h
---
 src/backend/access/heap/heapam.c              |  43 +++-
 src/backend/access/transam/xact.c             |  26 ++-
 src/backend/catalog/index.c                   |  11 +-
 src/backend/commands/event_trigger.c          |   5 -
 src/backend/replication/logical/decode.c      |  26 +--
 src/backend/utils/cache/catcache.c            |   7 +-
 src/backend/utils/cache/inval.c               | 304 ++++++++++++++++++--------
 src/backend/utils/cache/syscache.c            |   3 +-
 src/include/utils/catcache.h                  |   3 +-
 src/include/utils/inval.h                     |   6 +
 src/test/isolation/expected/inplace-inval.out |  10 +-
 src/test/isolation/specs/inplace-inval.spec   |  12 +-
 src/tools/pgindent/typedefs.list              |   1 +
 13 files changed, 320 insertions(+), 137 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cb65faa..517b374 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6352,6 +6352,24 @@ 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);
+
+	/*
+	 * 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();
 
@@ -6395,17 +6413,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/transam/xact.c b/src/backend/access/transam/xact.c
index 4cecf63..053a200 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1358,14 +1358,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 3eaf1f2..2156e21 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2889,12 +2889,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 05a6de6..a586d24 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 8ec5adf..03f1617 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -508,23 +508,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 111d8a2..ea8ca0e 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2288,7 +2288,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;
@@ -2329,7 +2330,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)
 		{
@@ -2338,7 +2339,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 603aa41..93bfa1e 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,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 										msgs,
 										n * sizeof(SharedInvalidationMessage)),
 								 nmsgs += n));
-	ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 								RelCacheMsgs,
 								(memcpy(msgarray + nmsgs,
 										msgs,
@@ -1024,7 +1081,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 +1097,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 +1120,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 +1179,20 @@ void
 AtEOSubXact_Inval(bool isCommit)
 {
 	int			my_level;
-	TransInvalidationInfo *myInfo = transInvalInfo;
+	TransInvalidationInfo *myInfo;
 
-	/* Quick exit if no messages. */
+	/*
+	 * 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 transactional messages. */
+	myInfo = transInvalInfo;
 	if (myInfo == NULL)
 		return;
 
@@ -1125,12 +1233,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 +1285,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 +1293,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 +1331,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 +1341,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 +1419,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 +1475,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1362,15 +1499,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 +1519,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1399,14 +1534,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 +1554,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 +1743,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 50c9440..f41b1c2 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".
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..3390e7a 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 5628b3f..3b57e78 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1253,6 +1253,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
inplace160-inval-durability-inplace-v6.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.  Back-patch to v12 (all supported versions).
    The back branch versions don't change WAL, because those branches just
    added end-of-recovery SIResetAll().  All branches change the ABI of
    extern function PrepareToInvalidateCacheTuple().  No PGXN extension
    calls that, and there's no apparent use case in extensions.
    
    Reviewed by Nitin Motiani and (in earlier versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index da5e656..7c82a95 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6327,6 +6327,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;
@@ -6334,6 +6337,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();
 
@@ -6363,9 +6389,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);
@@ -6377,17 +6410,28 @@ heap_inplace_update_and_unlock(Relation relation,
 		PageSetLSN(BufferGetPage(buffer), recptr);
 	}
 
-	END_CRIT_SECTION();
-
-	heap_inplace_unlock(relation, oldtup, buffer);
+	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 
 	/*
-	 * 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.
+	 * Send invalidations to shared queue.  SearchSysCacheLocked1() assumes we
+	 * do this before UnlockTuple().
 	 *
-	 * XXX ROLLBACK discards the invalidation.  See test inplace-inval.spec.
+	 * 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);
+
+	AcceptInvalidationMessages();	/* local processing of just-sent inval */
+
+	/*
+	 * 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 6dae723..c5208f3 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 5f5673e..f31cc3a 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 25f870b..32e509a 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 87700c7..20b7307 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1368,14 +1368,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 12822d0..9162b9f 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 05a6de6..a586d24 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 d687cee..e73576a 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 10276aa..ee303dc 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 603aa41..986850c 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;
 
-	/* Events emitted by previous commands of this (sub)transaction */
-	InvalidationMsgsGroup PriorCmdInvalidMsgs;
-
 	/* 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;
+
+	/* 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;
 
-	/* Quick exit if no messages. */
+	/*
+	 * 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 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 50c9440..f41b1c2 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".
diff --git a/src/include/access/heapam_xlog.h b/src/include/access/heapam_xlog.h
index 42736f3..4591e9a 100644
--- a/src/include/access/heapam_xlog.h
+++ b/src/include/access/heapam_xlog.h
@@ -20,6 +20,7 @@
 #include "storage/buf.h"
 #include "storage/bufpage.h"
 #include "storage/relfilelocator.h"
+#include "storage/sinval.h"
 #include "utils/relcache.h"
 
 
@@ -425,9 +426,14 @@ typedef struct xl_heap_confirm
 typedef struct xl_heap_inplace
 {
 	OffsetNumber offnum;		/* updated tuple's offset on page */
+	Oid			dbId;			/* MyDatabaseId */
+	Oid			tsId;			/* MyDatabaseTableSpace */
+	bool		relcacheInitFileInval;	/* invalidate relcache init files */
+	int			nmsgs;			/* number of shared inval msgs */
+	SharedInvalidationMessage msgs[FLEXIBLE_ARRAY_MEMBER];
 } xl_heap_inplace;
 
-#define SizeOfHeapInplace	(offsetof(xl_heap_inplace, offnum) + sizeof(OffsetNumber))
+#define MinSizeOfHeapInplace	(offsetof(xl_heap_inplace, nmsgs) + sizeof(int))
 
 /*
  * This is what we need to know about setting a visibility map bit
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 5ef244b..d9cf51a 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -31,7 +31,7 @@
 /*
  * Each page of XLOG file has a header like this:
  */
-#define XLOG_PAGE_MAGIC 0xD117	/* can be used as WAL version indicator */
+#define XLOG_PAGE_MAGIC 0xD118	/* can be used as WAL version indicator */
 
 typedef struct XLogPageHeaderData
 {
diff --git a/src/include/storage/sinval.h b/src/include/storage/sinval.h
index 8f5744b..c812237 100644
--- a/src/include/storage/sinval.h
+++ b/src/include/storage/sinval.h
@@ -144,6 +144,8 @@ extern void ProcessCatchupInterrupt(void);
 
 extern int	xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 												 bool *RelcacheInitFileInval);
+extern int	inplaceGetInvalidationMessages(SharedInvalidationMessage **msgs,
+										   bool *RelcacheInitFileInval);
 extern void ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
 												 int nmsgs, bool RelcacheInitFileInval,
 												 Oid dbid, Oid tsid);
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..3390e7a 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 54bf29b..1fb4801 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1258,6 +1258,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
inplace160-inval-durability-inplace-v6_16.patchtext/plain; charset=us-asciiDownload
commit 89c3673 (HEAD, zzy_test-commit-REL_16_STABLE)
Author:     Noah Misch <noah@leadboat.com>
AuthorDate: Tue Oct 22 14:42:05 2024 -0700
Commit:     Noah Misch <noah@leadboat.com>
CommitDate: Wed Oct 23 17:46:53 2024 -0700

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.  Back-patch to v12 (all supported versions).
    The back branch versions don't change WAL, because those branches just
    added end-of-recovery SIResetAll().  All branches change the ABI of
    extern function PrepareToInvalidateCacheTuple().  No PGXN extension
    calls that, and there's no apparent use case in extensions.
    
    Reviewed by Nitin Motiani and (in earlier versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com
    
    Conflicts:
    	src/backend/commands/event_trigger.c
    	src/backend/utils/cache/inval.c
---
 src/backend/access/heap/heapam.c              |  43 +++-
 src/backend/access/transam/xact.c             |  26 ++-
 src/backend/catalog/index.c                   |  11 +-
 src/backend/replication/logical/decode.c      |  26 +--
 src/backend/utils/cache/catcache.c            |   7 +-
 src/backend/utils/cache/inval.c               | 304 ++++++++++++++++++--------
 src/backend/utils/cache/syscache.c            |   3 +-
 src/include/utils/catcache.h                  |   3 +-
 src/include/utils/inval.h                     |   6 +
 src/test/isolation/expected/inplace-inval.out |  10 +-
 src/test/isolation/specs/inplace-inval.spec   |  12 +-
 src/tools/pgindent/typedefs.list              |   1 +
 12 files changed, 320 insertions(+), 132 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 75e8216..fd17d1c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6192,6 +6192,24 @@ 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);
+
+	/*
+	 * 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();
 
@@ -6235,17 +6253,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/transam/xact.c b/src/backend/access/transam/xact.c
index 4a2ea4a..91dbfcc 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1337,14 +1337,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 ec56bb6..b0ffe94 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2915,12 +2915,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/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index d91055a..70af02c 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -511,23 +511,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 000e81a..9f249fd 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2219,7 +2219,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;
@@ -2260,7 +2261,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)
 		{
@@ -2269,7 +2270,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 0008826..7ad69ee7 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.
@@ -131,13 +135,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
@@ -148,7 +154,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
@@ -201,7 +209,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
@@ -217,26 +225,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;
 
@@ -544,9 +562,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);
 }
 
@@ -556,10 +577,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);
 }
 
 /*
@@ -568,10 +588,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
@@ -588,7 +607,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;
 }
 
 /*
@@ -598,10 +617,9 @@ 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);
 }
 
 /*
@@ -791,14 +809,18 @@ AcceptInvalidationMessages(void)
  * 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,
@@ -821,7 +843,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");
 
 		/*
@@ -830,8 +852,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
@@ -847,6 +869,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;
 }
 
 /*
@@ -904,7 +961,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
@@ -915,7 +972,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 	 * not new ones.
 	 */
 	nummsgs = NumMessagesInGroup(&transInvalInfo->PriorCmdInvalidMsgs) +
-		NumMessagesInGroup(&transInvalInfo->CurrentCmdInvalidMsgs);
+		NumMessagesInGroup(&transInvalInfo->ii.CurrentCmdInvalidMsgs);
 
 	*msgs = msgarray = (SharedInvalidationMessage *)
 		MemoryContextAlloc(CurTransactionContext,
@@ -928,7 +985,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 										msgs,
 										n * sizeof(SharedInvalidationMessage)),
 								 nmsgs += n));
-	ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 								CatCacheMsgs,
 								(memcpy(msgarray + nmsgs,
 										msgs,
@@ -940,7 +997,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 										msgs,
 										n * sizeof(SharedInvalidationMessage)),
 								 nmsgs += n));
-	ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 								RelCacheMsgs,
 								(memcpy(msgarray + nmsgs,
 										msgs,
@@ -1027,7 +1084,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;
 
@@ -1041,16 +1100,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
@@ -1064,6 +1123,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.
  *
@@ -1085,9 +1182,20 @@ void
 AtEOSubXact_Inval(bool isCommit)
 {
 	int			my_level;
-	TransInvalidationInfo *myInfo = transInvalInfo;
+	TransInvalidationInfo *myInfo;
 
-	/* Quick exit if no messages. */
+	/*
+	 * 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 transactional messages. */
+	myInfo = transInvalInfo;
 	if (myInfo == NULL)
 		return;
 
@@ -1128,12 +1236,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;
@@ -1180,7 +1288,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 */
@@ -1188,26 +1296,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;
@@ -1231,11 +1334,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
@@ -1244,11 +1344,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
@@ -1321,7 +1422,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);
 }
 
 /*
@@ -1340,14 +1478,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1365,15 +1502,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);
 }
 
 /*
@@ -1386,9 +1522,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1402,14 +1537,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);
 }
 
 /*
@@ -1423,8 +1557,6 @@ CacheInvalidateRelcacheByRelid(Oid relid)
 {
 	HeapTuple	tup;
 
-	PrepareInvalidationState();
-
 	tup = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
 	if (!HeapTupleIsValid(tup))
 		elog(ERROR, "cache lookup failed for relation %u", relid);
@@ -1614,7 +1746,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 697b17b..b0cc6a0 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -951,8 +951,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".
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index a32d722..9204663 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -228,7 +228,8 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 extern void PrintCatCacheLeakWarning(HeapTuple tuple);
 extern void PrintCatCacheListLeakWarning(CatCList *list);
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 14b4eac..e807779 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 4357043..fcaa2e6 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1205,6 +1205,7 @@ InternalGrant
 Interval
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
inplace155-backbranch-inval-v1_14.patchtext/plain; charset=us-asciiDownload
commit 5704b32 (HEAD, zzy_test-commit-REL_14_STABLE)
Author:     Noah Misch <noah@leadboat.com>
AuthorDate: Wed Oct 23 15:27:13 2024 -0700
Commit:     Noah Misch <noah@leadboat.com>
CommitDate: Wed Oct 23 15:33:18 2024 -0700

    At end of recovery, reset all sinval-managed caches.
    
    An inplace update's invalidation messages are part of its transaction's
    commit record.  However, the update survives even if its transaction
    aborts or we stop recovery before replaying its transaction commit.
    After recovery, a backend that started in recovery could update the row
    without incorporating the inplace update.  That could result in a table
    with an index, yet relhasindex=f.  That is a source of index corruption.
    
    This bulk invalidation avoids the functional consequences.  A future
    change can fix the !RecoveryInProgress() scenario without changing the
    WAL format.  Back-patch to v17 - v12 (all supported versions).  v18 will
    instead add invalidations to WAL.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240618152349.7f.nmisch@google.com
    
    Conflicts:
    	src/backend/access/transam/xlog.c
    
    Conflicts:
    	src/backend/access/transam/xlog.c
---
 src/backend/access/transam/xlog.c   | 25 ++++++++++++++++++++++
 src/backend/storage/ipc/sinvaladt.c | 41 +++++++++++++++++++++++++++++++++++++
 src/include/storage/sinvaladt.h     |  1 +
 3 files changed, 67 insertions(+)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 7654ad7..f1a795b 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -70,6 +70,7 @@
 #include "storage/proc.h"
 #include "storage/procarray.h"
 #include "storage/reinit.h"
+#include "storage/sinvaladt.h"
 #include "storage/smgr.h"
 #include "storage/spin.h"
 #include "storage/sync.h"
@@ -8020,6 +8021,30 @@ StartupXLOG(void)
 	}
 
 	/*
+	 * Invalidate all sinval-managed caches before READ WRITE transactions
+	 * begin.  The xl_heap_inplace WAL record doesn't store sufficient data
+	 * for invalidations.  The commit record, if any, has the invalidations.
+	 * However, the inplace update is permanent, whether or not we reach a
+	 * commit record.  Fortunately, read-only transactions tolerate caches not
+	 * reflecting the latest inplace updates.  Read-only transactions
+	 * experience the notable inplace updates as follows:
+	 *
+	 * - relhasindex=true affects readers only after the CREATE INDEX
+	 * transaction commit makes an index fully available to them.
+	 *
+	 * - datconnlimit=DATCONNLIMIT_INVALID_DB affects readers only at
+	 * InitPostgres() time, and that read does not use a cache.
+	 *
+	 * - relfrozenxid, datfrozenxid, relminmxid, and datminmxid have no effect
+	 * on readers.
+	 *
+	 * Hence, hot standby queries (all READ ONLY) function correctly without
+	 * the missing invalidations.  This avoided changing the WAL format in
+	 * back branches.
+	 */
+	SIResetAll();
+
+	/*
 	 * Preallocate additional log files, if wanted.
 	 */
 	PreallocXlogFiles(EndOfLog);
diff --git a/src/backend/storage/ipc/sinvaladt.c b/src/backend/storage/ipc/sinvaladt.c
index 946bd8e..f0e2fb9 100644
--- a/src/backend/storage/ipc/sinvaladt.c
+++ b/src/backend/storage/ipc/sinvaladt.c
@@ -748,6 +748,47 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
 	}
 }
 
+/*
+ * SIResetAll
+ *		Mark all active backends as "reset"
+ *
+ * Use this when we don't know what needs to be invalidated.  It's a
+ * cluster-wide InvalidateSystemCaches().  This was a back-branch-only remedy
+ * to avoid a WAL format change.
+ *
+ * The implementation is like SICleanupQueue(false, MAXNUMMESSAGES + 1), with
+ * one addition.  SICleanupQueue() assumes minFree << MAXNUMMESSAGES, so it
+ * assumes hasMessages==true for any backend it resets.  We're resetting even
+ * fully-caught-up backends, so we set hasMessages.
+ */
+void
+SIResetAll(void)
+{
+	SISeg	   *segP = shmInvalBuffer;
+	int			i;
+
+	LWLockAcquire(SInvalWriteLock, LW_EXCLUSIVE);
+	LWLockAcquire(SInvalReadLock, LW_EXCLUSIVE);
+
+	for (i = 0; i < segP->lastBackend; i++)
+	{
+		ProcState  *stateP = &segP->procState[i];
+
+		if (stateP->procPid == 0 || stateP->sendOnly)
+			continue;
+
+		/* Consuming the reset will update "nextMsgNum" and "signaled". */
+		stateP->resetState = true;
+		stateP->hasMessages = true;
+	}
+
+	segP->minMsgNum = segP->maxMsgNum;
+	segP->nextThreshold = CLEANUP_MIN;
+
+	LWLockRelease(SInvalReadLock);
+	LWLockRelease(SInvalWriteLock);
+}
+
 
 /*
  * GetNextLocalTransactionId --- allocate a new LocalTransactionId
diff --git a/src/include/storage/sinvaladt.h b/src/include/storage/sinvaladt.h
index 14148bf..07d229a 100644
--- a/src/include/storage/sinvaladt.h
+++ b/src/include/storage/sinvaladt.h
@@ -37,6 +37,7 @@ extern void BackendIdGetTransactionIds(int backendID, TransactionId *xid, Transa
 extern void SIInsertDataEntries(const SharedInvalidationMessage *data, int n);
 extern int	SIGetDataEntries(SharedInvalidationMessage *data, int datasize);
 extern void SICleanupQueue(bool callerHasWriteLock, int minFree);
+extern void SIResetAll(void);
 
 extern LocalTransactionId GetNextLocalTransactionId(void);
 
inplace155-backbranch-inval-v1_16.patchtext/plain; charset=us-asciiDownload
commit 1bfa06e (HEAD, zzy_test-commit-REL_16_STABLE)
Author:     Noah Misch <noah@leadboat.com>
AuthorDate: Wed Oct 23 15:27:13 2024 -0700
Commit:     Noah Misch <noah@leadboat.com>
CommitDate: Wed Oct 23 15:31:54 2024 -0700

    At end of recovery, reset all sinval-managed caches.
    
    An inplace update's invalidation messages are part of its transaction's
    commit record.  However, the update survives even if its transaction
    aborts or we stop recovery before replaying its transaction commit.
    After recovery, a backend that started in recovery could update the row
    without incorporating the inplace update.  That could result in a table
    with an index, yet relhasindex=f.  That is a source of index corruption.
    
    This bulk invalidation avoids the functional consequences.  A future
    change can fix the !RecoveryInProgress() scenario without changing the
    WAL format.  Back-patch to v17 - v12 (all supported versions).  v18 will
    instead add invalidations to WAL.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240618152349.7f.nmisch@google.com
    
    Conflicts:
    	src/backend/access/transam/xlog.c
---
 src/backend/access/transam/xlog.c   | 25 ++++++++++++++++++++++
 src/backend/storage/ipc/sinvaladt.c | 41 +++++++++++++++++++++++++++++++++++++
 src/include/storage/sinvaladt.h     |  1 +
 3 files changed, 67 insertions(+)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index a19ba71..46faf40 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -94,6 +94,7 @@
 #include "storage/proc.h"
 #include "storage/procarray.h"
 #include "storage/reinit.h"
+#include "storage/sinvaladt.h"
 #include "storage/smgr.h"
 #include "storage/spin.h"
 #include "storage/sync.h"
@@ -5660,6 +5661,30 @@ StartupXLOG(void)
 	XLogCtl->LogwrtRqst.Flush = EndOfLog;
 
 	/*
+	 * Invalidate all sinval-managed caches before READ WRITE transactions
+	 * begin.  The xl_heap_inplace WAL record doesn't store sufficient data
+	 * for invalidations.  The commit record, if any, has the invalidations.
+	 * However, the inplace update is permanent, whether or not we reach a
+	 * commit record.  Fortunately, read-only transactions tolerate caches not
+	 * reflecting the latest inplace updates.  Read-only transactions
+	 * experience the notable inplace updates as follows:
+	 *
+	 * - relhasindex=true affects readers only after the CREATE INDEX
+	 * transaction commit makes an index fully available to them.
+	 *
+	 * - datconnlimit=DATCONNLIMIT_INVALID_DB affects readers only at
+	 * InitPostgres() time, and that read does not use a cache.
+	 *
+	 * - relfrozenxid, datfrozenxid, relminmxid, and datminmxid have no effect
+	 * on readers.
+	 *
+	 * Hence, hot standby queries (all READ ONLY) function correctly without
+	 * the missing invalidations.  This avoided changing the WAL format in
+	 * back branches.
+	 */
+	SIResetAll();
+
+	/*
 	 * Preallocate additional log files, if wanted.
 	 */
 	PreallocXlogFiles(EndOfLog, newTLI);
diff --git a/src/backend/storage/ipc/sinvaladt.c b/src/backend/storage/ipc/sinvaladt.c
index 3d97c75..099c292 100644
--- a/src/backend/storage/ipc/sinvaladt.c
+++ b/src/backend/storage/ipc/sinvaladt.c
@@ -762,6 +762,47 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
 	}
 }
 
+/*
+ * SIResetAll
+ *		Mark all active backends as "reset"
+ *
+ * Use this when we don't know what needs to be invalidated.  It's a
+ * cluster-wide InvalidateSystemCaches().  This was a back-branch-only remedy
+ * to avoid a WAL format change.
+ *
+ * The implementation is like SICleanupQueue(false, MAXNUMMESSAGES + 1), with
+ * one addition.  SICleanupQueue() assumes minFree << MAXNUMMESSAGES, so it
+ * assumes hasMessages==true for any backend it resets.  We're resetting even
+ * fully-caught-up backends, so we set hasMessages.
+ */
+void
+SIResetAll(void)
+{
+	SISeg	   *segP = shmInvalBuffer;
+	int			i;
+
+	LWLockAcquire(SInvalWriteLock, LW_EXCLUSIVE);
+	LWLockAcquire(SInvalReadLock, LW_EXCLUSIVE);
+
+	for (i = 0; i < segP->lastBackend; i++)
+	{
+		ProcState  *stateP = &segP->procState[i];
+
+		if (stateP->procPid == 0 || stateP->sendOnly)
+			continue;
+
+		/* Consuming the reset will update "nextMsgNum" and "signaled". */
+		stateP->resetState = true;
+		stateP->hasMessages = true;
+	}
+
+	segP->minMsgNum = segP->maxMsgNum;
+	segP->nextThreshold = CLEANUP_MIN;
+
+	LWLockRelease(SInvalReadLock);
+	LWLockRelease(SInvalWriteLock);
+}
+
 
 /*
  * GetNextLocalTransactionId --- allocate a new LocalTransactionId
diff --git a/src/include/storage/sinvaladt.h b/src/include/storage/sinvaladt.h
index db38819..1f9eaf2 100644
--- a/src/include/storage/sinvaladt.h
+++ b/src/include/storage/sinvaladt.h
@@ -39,6 +39,7 @@ extern void BackendIdGetTransactionIds(int backendID, TransactionId *xid,
 extern void SIInsertDataEntries(const SharedInvalidationMessage *data, int n);
 extern int	SIGetDataEntries(SharedInvalidationMessage *data, int datasize);
 extern void SICleanupQueue(bool callerHasWriteLock, int minFree);
+extern void SIResetAll(void);
 
 extern LocalTransactionId GetNextLocalTransactionId(void);
 
inplace155-backbranch-inval-v1_17.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    At end of recovery, reset all sinval-managed caches.
    
    An inplace update's invalidation messages are part of its transaction's
    commit record.  However, the update survives even if its transaction
    aborts or we stop recovery before replaying its transaction commit.
    After recovery, a backend that started in recovery could update the row
    without incorporating the inplace update.  That could result in a table
    with an index, yet relhasindex=f.  That is a source of index corruption.
    
    This bulk invalidation avoids the functional consequences.  A future
    change can fix the !RecoveryInProgress() scenario without changing the
    WAL format.  Back-patch to v17 - v12 (all supported versions).  v18 will
    instead add invalidations to WAL.
    
    Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/20240618152349.7f.nmisch@google.com

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 7f13602..c1dd8b5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -92,6 +92,7 @@
 #include "storage/proc.h"
 #include "storage/procarray.h"
 #include "storage/reinit.h"
+#include "storage/sinvaladt.h"
 #include "storage/spin.h"
 #include "storage/sync.h"
 #include "utils/guc_hooks.h"
@@ -6038,6 +6039,30 @@ StartupXLOG(void)
 	XLogCtl->LogwrtRqst.Flush = EndOfLog;
 
 	/*
+	 * Invalidate all sinval-managed caches before READ WRITE transactions
+	 * begin.  The xl_heap_inplace WAL record doesn't store sufficient data
+	 * for invalidations.  The commit record, if any, has the invalidations.
+	 * However, the inplace update is permanent, whether or not we reach a
+	 * commit record.  Fortunately, read-only transactions tolerate caches not
+	 * reflecting the latest inplace updates.  Read-only transactions
+	 * experience the notable inplace updates as follows:
+	 *
+	 * - relhasindex=true affects readers only after the CREATE INDEX
+	 * transaction commit makes an index fully available to them.
+	 *
+	 * - datconnlimit=DATCONNLIMIT_INVALID_DB affects readers only at
+	 * InitPostgres() time, and that read does not use a cache.
+	 *
+	 * - relfrozenxid, datfrozenxid, relminmxid, and datminmxid have no effect
+	 * on readers.
+	 *
+	 * Hence, hot standby queries (all READ ONLY) function correctly without
+	 * the missing invalidations.  This avoided changing the WAL format in
+	 * back branches.
+	 */
+	SIResetAll();
+
+	/*
 	 * Preallocate additional log files, if wanted.
 	 */
 	PreallocXlogFiles(EndOfLog, newTLI);
diff --git a/src/backend/storage/ipc/sinvaladt.c b/src/backend/storage/ipc/sinvaladt.c
index b486d8d..2712129 100644
--- a/src/backend/storage/ipc/sinvaladt.c
+++ b/src/backend/storage/ipc/sinvaladt.c
@@ -683,6 +683,48 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
 	}
 }
 
+/*
+ * SIResetAll
+ *		Mark all active backends as "reset"
+ *
+ * Use this when we don't know what needs to be invalidated.  It's a
+ * cluster-wide InvalidateSystemCaches().  This was a back-branch-only remedy
+ * to avoid a WAL format change.
+ *
+ * The implementation is like SICleanupQueue(false, MAXNUMMESSAGES + 1), with
+ * one addition.  SICleanupQueue() assumes minFree << MAXNUMMESSAGES, so it
+ * assumes hasMessages==true for any backend it resets.  We're resetting even
+ * fully-caught-up backends, so we set hasMessages.
+ */
+void
+SIResetAll(void)
+{
+	SISeg	   *segP = shmInvalBuffer;
+	int			i;
+
+	LWLockAcquire(SInvalWriteLock, LW_EXCLUSIVE);
+	LWLockAcquire(SInvalReadLock, LW_EXCLUSIVE);
+
+	for (i = 0; i < segP->numProcs; i++)
+	{
+		ProcState  *stateP = &segP->procState[segP->pgprocnos[i]];
+
+		Assert(stateP->procPid != 0);
+		if (stateP->sendOnly)
+			continue;
+
+		/* Consuming the reset will update "nextMsgNum" and "signaled". */
+		stateP->resetState = true;
+		stateP->hasMessages = true;
+	}
+
+	segP->minMsgNum = segP->maxMsgNum;
+	segP->nextThreshold = CLEANUP_MIN;
+
+	LWLockRelease(SInvalReadLock);
+	LWLockRelease(SInvalWriteLock);
+}
+
 
 /*
  * GetNextLocalTransactionId --- allocate a new LocalTransactionId
diff --git a/src/include/storage/sinvaladt.h b/src/include/storage/sinvaladt.h
index c3c97b3..7c8f450 100644
--- a/src/include/storage/sinvaladt.h
+++ b/src/include/storage/sinvaladt.h
@@ -35,6 +35,7 @@ extern void SharedInvalBackendInit(bool sendOnly);
 extern void SIInsertDataEntries(const SharedInvalidationMessage *data, int n);
 extern int	SIGetDataEntries(SharedInvalidationMessage *data, int datasize);
 extern void SICleanupQueue(bool callerHasWriteLock, int minFree);
+extern void SIResetAll(void);
 
 extern LocalTransactionId GetNextLocalTransactionId(void);
 
inplace160-inval-durability-inplace-v6_13.patchtext/plain; charset=us-asciiDownload
commit efc6d94 (HEAD, zzy_test-commit-REL_13_STABLE)
Author:     Noah Misch <noah@leadboat.com>
AuthorDate: Wed Oct 23 15:35:02 2024 -0700
Commit:     Noah Misch <noah@leadboat.com>
CommitDate: Wed Oct 23 17:54:08 2024 -0700

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.  Back-patch to v12 (all supported versions).
    The back branch versions don't change WAL, because those branches just
    added end-of-recovery SIResetAll().  All branches change the ABI of
    extern function PrepareToInvalidateCacheTuple().  No PGXN extension
    calls that, and there's no apparent use case in extensions.
    
    Reviewed by Nitin Motiani and (in earlier versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com
    
    Conflicts:
    	src/backend/replication/logical/decode.c
    	src/backend/utils/cache/inval.c
    	src/tools/pgindent/typedefs.list
    
    Conflicts:
    	src/backend/utils/cache/inval.c
---
 src/backend/access/heap/heapam.c              |  43 ++++-
 src/backend/access/transam/xact.c             |  26 ++-
 src/backend/catalog/index.c                   |  11 +-
 src/backend/replication/logical/decode.c      |  26 ++-
 src/backend/utils/cache/catcache.c            |   7 +-
 src/backend/utils/cache/inval.c               | 259 +++++++++++++++++++-------
 src/backend/utils/cache/syscache.c            |   3 +-
 src/include/utils/catcache.h                  |   3 +-
 src/include/utils/inval.h                     |   6 +
 src/test/isolation/expected/inplace-inval.out |  10 +-
 src/test/isolation/specs/inplace-inval.spec   |  12 +-
 src/tools/pgindent/typedefs.list              |   4 +-
 12 files changed, 291 insertions(+), 119 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 846650c..ce10537 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6112,6 +6112,24 @@ 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);
+
+	/*
+	 * 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();
 
@@ -6155,17 +6173,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/transam/xact.c b/src/backend/access/transam/xact.c
index ffe26e2..cda6e33 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1249,14 +1249,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 4ca86d9..9ab38b9 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2882,12 +2882,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/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 297eb11..b3b51a9 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -460,23 +460,19 @@ DecodeHeapOp(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;
-
-			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 b53232f..9a61890 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2129,7 +2129,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;
@@ -2170,7 +2171,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)
 		{
@@ -2179,7 +2180,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 8fe1f0d..a0dc745 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -99,6 +99,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.
+ *
  *
  * Portions Copyright (c) 1996-2020, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
@@ -150,7 +154,7 @@ typedef struct InvalidationListHeader
 } InvalidationListHeader;
 
 /*----------------
- * Invalidation info is divided into two lists:
+ * Transactional invalidation info is divided into two lists:
  *	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
@@ -166,26 +170,36 @@ typedef struct InvalidationListHeader
  *----------------
  */
 
-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;
-
 	/* head of current-command event list */
 	InvalidationListHeader CurrentCmdInvalidMsgs;
 
+	/* init file must be invalidated? */
+	bool		RelcacheInitFileInval;
+} InvalidationInfo;
+
+/* subclass adding fields specific to transactional invalidation */
+typedef struct TransInvalidationInfo
+{
+	/* Base class */
+	struct InvalidationInfo ii;
+
 	/* head of previous-commands event list */
 	InvalidationListHeader 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;
+
 static SharedInvalidationMessage *SharedInvalidMessagesArray;
 static int	numSharedInvalidMessagesArray;
 static int	maxSharedInvalidMessagesArray;
@@ -499,9 +513,12 @@ ProcessInvalidationMessagesMulti(InvalidationListHeader *hdr,
 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);
 }
 
@@ -511,10 +528,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);
 }
 
 /*
@@ -523,10 +539,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
@@ -543,7 +558,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;
 }
 
 /*
@@ -553,10 +568,9 @@ 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);
 }
 
 /*
@@ -748,14 +762,18 @@ AcceptInvalidationMessages(void)
  * PrepareInvalidationState
  *		Initialize inval lists 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,
@@ -771,6 +789,29 @@ PrepareInvalidationState(void)
 		   myInfo->my_level > transInvalInfo->my_level);
 
 	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));
+
+	inplaceInvalInfo = myInfo;
+	return myInfo;
 }
 
 /*
@@ -866,7 +907,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;
 
 	/*
 	 * Walk through TransInvalidationInfo to collect all the messages into a
@@ -878,7 +919,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 	 */
 	oldcontext = MemoryContextSwitchTo(CurTransactionContext);
 
-	ProcessInvalidationMessagesMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessInvalidationMessagesMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 									 MakeSharedInvalidMessagesArray);
 	ProcessInvalidationMessagesMulti(&transInvalInfo->PriorCmdInvalidMsgs,
 									 MakeSharedInvalidMessagesArray);
@@ -968,7 +1009,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;
 
@@ -982,16 +1025,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
@@ -1007,6 +1050,45 @@ 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;
+	/* inplace doesn't use SharedInvalidMessagesArray */
+}
+
+/*
  * AtEOSubXact_Inval
  *		Process queued-up invalidation messages at end of subtransaction.
  *
@@ -1028,9 +1110,20 @@ void
 AtEOSubXact_Inval(bool isCommit)
 {
 	int			my_level;
-	TransInvalidationInfo *myInfo = transInvalInfo;
+	TransInvalidationInfo *myInfo;
 
-	/* Quick exit if no messages. */
+	/*
+	 * 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 transactional messages. */
+	myInfo = transInvalInfo;
 	if (myInfo == NULL)
 		return;
 
@@ -1064,8 +1157,8 @@ AtEOSubXact_Inval(bool isCommit)
 								   &myInfo->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;
@@ -1112,29 +1205,24 @@ CommandEndInvalidationMessages(void)
 	if (transInvalInfo == NULL)
 		return;
 
-	ProcessInvalidationMessages(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessInvalidationMessages(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 								LocalExecuteInvalidationMessage);
 	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;
@@ -1158,11 +1246,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
@@ -1171,11 +1256,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
@@ -1248,7 +1334,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);
 }
 
 /*
@@ -1267,14 +1390,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1292,15 +1414,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);
 }
 
 /*
@@ -1313,9 +1434,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1329,14 +1449,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);
 }
 
 /*
@@ -1350,8 +1469,6 @@ CacheInvalidateRelcacheByRelid(Oid relid)
 {
 	HeapTuple	tup;
 
-	PrepareInvalidationState();
-
 	tup = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
 	if (!HeapTupleIsValid(tup))
 		elog(ERROR, "cache lookup failed for relation %u", relid);
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index 8ad8791..e4a8fb2 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -1249,8 +1249,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".
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index f4aa316..2722173 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -223,7 +223,8 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 extern void PrintCatCacheLeakWarning(HeapTuple tuple);
 extern void PrintCatCacheListLeakWarning(CatCList *list);
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 4c6b86c..3ff52e8 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -27,6 +27,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -36,6 +39,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
inplace160-inval-durability-inplace-v6_14.patchtext/plain; charset=us-asciiDownload
commit 8133c66 (HEAD, zzy_test-commit-REL_14_STABLE)
Author:     Noah Misch <noah@leadboat.com>
AuthorDate: Wed Oct 23 15:35:02 2024 -0700
Commit:     Noah Misch <noah@leadboat.com>
CommitDate: Wed Oct 23 17:50:57 2024 -0700

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.  Back-patch to v12 (all supported versions).
    The back branch versions don't change WAL, because those branches just
    added end-of-recovery SIResetAll().  All branches change the ABI of
    extern function PrepareToInvalidateCacheTuple().  No PGXN extension
    calls that, and there's no apparent use case in extensions.
    
    Reviewed by Nitin Motiani and (in earlier versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com
    
    Conflicts:
    	src/backend/replication/logical/decode.c
    	src/backend/utils/cache/inval.c
    	src/tools/pgindent/typedefs.list
---
 src/backend/access/heap/heapam.c              |  43 ++++-
 src/backend/access/transam/xact.c             |  26 ++-
 src/backend/catalog/index.c                   |  11 +-
 src/backend/replication/logical/decode.c      |  26 ++-
 src/backend/utils/cache/catcache.c            |   7 +-
 src/backend/utils/cache/inval.c               | 261 +++++++++++++++++++-------
 src/backend/utils/cache/syscache.c            |   3 +-
 src/include/utils/catcache.h                  |   3 +-
 src/include/utils/inval.h                     |   6 +
 src/test/isolation/expected/inplace-inval.out |  10 +-
 src/test/isolation/specs/inplace-inval.spec   |  12 +-
 src/tools/pgindent/typedefs.list              |   4 +-
 12 files changed, 292 insertions(+), 120 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index a7267fb..6970bdc 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6384,6 +6384,24 @@ 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);
+
+	/*
+	 * 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();
 
@@ -6427,17 +6445,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/transam/xact.c b/src/backend/access/transam/xact.c
index 3f0f711..395650d 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1276,14 +1276,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 e4fd1fd..f0dd962 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2895,12 +2895,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/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 1c0058f..e1753dd 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -547,23 +547,19 @@ DecodeHeapOp(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;
-
-			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 5979030..091b061 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2131,7 +2131,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;
@@ -2172,7 +2173,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)
 		{
@@ -2181,7 +2182,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 e281a45..c793be5 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -99,6 +99,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.
@@ -154,7 +158,7 @@ typedef struct InvalidationListHeader
 } InvalidationListHeader;
 
 /*----------------
- * Invalidation info is divided into two lists:
+ * Transactional invalidation info is divided into two lists:
  *	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
@@ -170,26 +174,36 @@ typedef struct InvalidationListHeader
  *----------------
  */
 
-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;
-
 	/* head of current-command event list */
 	InvalidationListHeader CurrentCmdInvalidMsgs;
 
+	/* init file must be invalidated? */
+	bool		RelcacheInitFileInval;
+} InvalidationInfo;
+
+/* subclass adding fields specific to transactional invalidation */
+typedef struct TransInvalidationInfo
+{
+	/* Base class */
+	struct InvalidationInfo ii;
+
 	/* head of previous-commands event list */
 	InvalidationListHeader 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;
+
 static SharedInvalidationMessage *SharedInvalidMessagesArray;
 static int	numSharedInvalidMessagesArray;
 static int	maxSharedInvalidMessagesArray;
@@ -505,9 +519,12 @@ ProcessInvalidationMessagesMulti(InvalidationListHeader *hdr,
 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);
 }
 
@@ -517,10 +534,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);
 }
 
 /*
@@ -529,10 +545,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
@@ -549,7 +564,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;
 }
 
 /*
@@ -559,10 +574,9 @@ 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);
 }
 
 /*
@@ -752,14 +766,18 @@ AcceptInvalidationMessages(void)
  * PrepareInvalidationState
  *		Initialize inval lists 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,
@@ -775,6 +793,29 @@ PrepareInvalidationState(void)
 		   myInfo->my_level > transInvalInfo->my_level);
 
 	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));
+
+	inplaceInvalInfo = myInfo;
+	return myInfo;
 }
 
 /*
@@ -870,7 +911,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;
 
 	/*
 	 * Walk through TransInvalidationInfo to collect all the messages into a
@@ -882,7 +923,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 	 */
 	oldcontext = MemoryContextSwitchTo(CurTransactionContext);
 
-	ProcessInvalidationMessagesMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessInvalidationMessagesMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 									 MakeSharedInvalidMessagesArray);
 	ProcessInvalidationMessagesMulti(&transInvalInfo->PriorCmdInvalidMsgs,
 									 MakeSharedInvalidMessagesArray);
@@ -972,7 +1013,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;
 
@@ -986,16 +1029,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
@@ -1011,6 +1054,45 @@ 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;
+	/* inplace doesn't use SharedInvalidMessagesArray */
+}
+
+/*
  * AtEOSubXact_Inval
  *		Process queued-up invalidation messages at end of subtransaction.
  *
@@ -1032,9 +1114,20 @@ void
 AtEOSubXact_Inval(bool isCommit)
 {
 	int			my_level;
-	TransInvalidationInfo *myInfo = transInvalInfo;
+	TransInvalidationInfo *myInfo;
 
-	/* Quick exit if no messages. */
+	/*
+	 * 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 transactional messages. */
+	myInfo = transInvalInfo;
 	if (myInfo == NULL)
 		return;
 
@@ -1068,8 +1161,8 @@ AtEOSubXact_Inval(bool isCommit)
 								   &myInfo->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;
@@ -1116,7 +1209,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 */
@@ -1124,26 +1217,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;
@@ -1167,11 +1255,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
@@ -1180,11 +1265,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
@@ -1257,7 +1343,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);
 }
 
 /*
@@ -1276,14 +1399,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1301,15 +1423,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);
 }
 
 /*
@@ -1322,9 +1443,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1338,14 +1458,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);
 }
 
 /*
@@ -1359,8 +1478,6 @@ CacheInvalidateRelcacheByRelid(Oid relid)
 {
 	HeapTuple	tup;
 
-	PrepareInvalidationState();
-
 	tup = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
 	if (!HeapTupleIsValid(tup))
 		elog(ERROR, "cache lookup failed for relation %u", relid);
@@ -1549,7 +1666,7 @@ LogLogicalInvalidations()
 	if (transInvalInfo == NULL)
 		return;
 
-	ProcessInvalidationMessagesMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessInvalidationMessagesMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 									 MakeSharedInvalidMessagesArray);
 
 	Assert(!(numSharedInvalidMessagesArray > 0 &&
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index 7d573b6..ce5d2e8 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -1260,8 +1260,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".
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index ddc2762..c4fd4d1 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -223,7 +223,8 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 extern void PrintCatCacheLeakWarning(HeapTuple tuple);
 extern void PrintCatCacheListLeakWarning(CatCList *list);
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 877e66c..c89ba72 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,9 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +40,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
#17Nitin Motiani
nitinmotiani@google.com
In reply to: Noah Misch (#16)
Re: Inval reliability, especially for inplace updates

On Thu, Oct 24, 2024 at 8:24 AM Noah Misch <noah@leadboat.com> wrote:

With the releases wrapping in 2.5 weeks, I'm ambivalent about pushing this
before the release or after. Pushing before means fewer occurrences of
corruption, but pushing after gives more bake time to discover these changes
were defective. It's hard to predict which helps users more, on a
risk-adjusted basis. I'm leaning toward pushing this week. Opinions?

I lean towards pushing after the release. This is based on my
assumption that since this bug has been around for a while, it is
(probably) not hit often. And a few weeks delay is better than
introducing a new defect.

Thanks

#18Noah Misch
noah@leadboat.com
In reply to: Nitin Motiani (#17)
Re: Inval reliability, especially for inplace updates

On Mon, Oct 28, 2024 at 02:27:03PM +0530, Nitin Motiani wrote:

On Thu, Oct 24, 2024 at 8:24 AM Noah Misch <noah@leadboat.com> wrote:

With the releases wrapping in 2.5 weeks, I'm ambivalent about pushing this
before the release or after. Pushing before means fewer occurrences of
corruption, but pushing after gives more bake time to discover these changes
were defective. It's hard to predict which helps users more, on a
risk-adjusted basis. I'm leaning toward pushing this week. Opinions?

I lean towards pushing after the release. This is based on my
assumption that since this bug has been around for a while, it is
(probably) not hit often. And a few weeks delay is better than
introducing a new defect.

I had pushed this during the indicated week, before your mail. Reverting it
is an option. Let's see if more opinions arrive.

#19Alexander Lakhin
exclusion@gmail.com
In reply to: Noah Misch (#18)
3 attachment(s)
Re: Inval reliability, especially for inplace updates

Hello Noah,

31.10.2024 04:39, Noah Misch wrote:

I had pushed this during the indicated week, before your mail. Reverting it
is an option. Let's see if more opinions arrive.

I've accidentally discovered an incorrect behaviour caused by commit
4eac5a1fa. Running this script:
for ((j=1; j<=100; j++)); do
echo "iteration $j"

cat << 'EOF' | timeout 60 psql >>psql-$SID.log || { res=1; echo "hanged on iteration $j"; break; }
SELECT format('CREATE TABLE t%s (a int, b text);', g) FROM generate_series(1, 50) g
\gexec

SELECT format('DROP TABLE t%s;', g) FROM generate_series(1, 50) g
\gexec
EOF
done

with
autovacuum = on
autovacuum_naptime = 1s
autovacuum_vacuum_threshold = 1
autovacuum_analyze_threshold = 1

in parallel using separate servers (the full script is attached), like:
parallel -j40 --linebuffer --tag .../reproi.sh ::: `seq 40`

I can catch the following:
...
3       hanged on iteration 51
...
19      hanged on iteration 64
...
39      hanged on iteration 99

And after the script run, I see the server processes hanging:
law      1081433       1  0 16:22 ?        00:00:00 .../usr/local/pgsql/bin/postgres
law      1081452 1081433  0 16:22 ?        00:00:00 postgres: checkpointer
law      1081453 1081433  0 16:22 ?        00:00:00 postgres: background writer
law      1081460 1081433  0 16:22 ?        00:00:00 postgres: walwriter
law      1081462 1081433  0 16:22 ?        00:00:00 postgres: autovacuum launcher
law      1081464 1081433  0 16:22 ?        00:00:00 postgres: logical replication launcher
law      1143065 1081433  0 16:32 ?        00:00:00 postgres: postgres postgres [local] CREATE TABLE
law      1143263 1081433  0 16:32 ?        00:00:00 postgres: autovacuum worker postgres
law      1143320 1081433  0 16:32 ?        00:00:00 postgres: autovacuum worker postgres
law      1143403 1081433  0 16:32 ?        00:00:00 postgres: autovacuum worker

Attaching to process 1143065
...
(gdb) bt
#0  __futex_abstimed_wait_common64 (private=<optimized out>, cancel=true, abstime=0x0, op=265, expected=0,
futex_word=0x7fed9a8171b8) at ./nptl/futex-internal.c:57
#1  __futex_abstimed_wait_common (cancel=true, private=<optimized out>, abstime=0x0, clockid=0, expected=0,
futex_word=0x7fed9a8171b8) at ./nptl/futex-internal.c:87
#2  __GI___futex_abstimed_wait_cancelable64 (futex_word=futex_word@entry=0x7fed9a8171b8, expected=expected@entry=0,
clockid=clockid@entry=0, abstime=abstime@entry=0x0, private=<optimized out>) at ./nptl/futex-internal.c:139
#3  0x00007feda4674c5f in do_futex_wait (sem=sem@entry=0x7fed9a8171b8, abstime=0x0, clockid=0) at
./nptl/sem_waitcommon.c:111
#4  0x00007feda4674cf8 in __new_sem_wait_slow64 (sem=0x7fed9a8171b8, abstime=0x0, clockid=0) at ./nptl/sem_waitcommon.c:183
#5  0x0000561dd715f1fe in PGSemaphoreLock (sema=0x7fed9a8171b8) at pg_sema.c:327
#6  0x0000561dd722fe02 in LWLockAcquire (lock=0x7fed9ad9b4e4, mode=LW_SHARED) at lwlock.c:1318
#7  0x0000561dd71f8423 in LockBuffer (buffer=36, mode=1) at bufmgr.c:4182
#8  0x0000561dd6d4f506 in heapam_index_fetch_tuple (scan=0x561dd8cb6588, tid=0x561dd8cb64d0, snapshot=0x561dd8bfee28,
slot=0x561dd8cb75a0, call_again=0x561dd8cb64d6, all_dead=0x7ffdd63842c6) at heapam_handler.c:146
...
(the full backtrace is attached)

All three autovacuum workers (1143263, 1143320, 1143403) are also waiting
for the same buffer lock:
#5  0x0000561dd715f1fe in PGSemaphoreLock (sema=0x7fed9a817338) at pg_sema.c:327
#6  0x0000561dd722fe02 in LWLockAcquire (lock=0x7fed9ad9b4e4, mode=LW_SHARED) at lwlock.c:1318
#7  0x0000561dd71f8423 in LockBuffer (buffer=36, mode=1) at bufmgr.c:4182

Probably, this can be reproduced with VACUUM pg_class/pg_type/..., but I
haven't found out the exact combination needed yet.

Also as a side note, these processes can't be terminated with SIGTERM, I
have to kill them.

Initially I saw this on a slowed down VM, but with the attached patch
applied I could reproduce it on my workstation too.

Best regards,
Alexander

Attachments:

reproi.shapplication/x-shellscript; name=reproi.shDownload
delay-before-lockbuffer.patchtext/x-patch; charset=UTF-8; name=delay-before-lockbuffer.patchDownload
diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c
index 2168259247..730ef9b5a2 100644
--- a/src/backend/access/heap/heapam_handler.c
+++ b/src/backend/access/heap/heapam_handler.c
@@ -138,6 +138,10 @@ heapam_index_fetch_tuple(struct IndexFetchTableData *scan,
 			heap_page_prune_opt(hscan->xs_base.rel, hscan->xs_cbuf);
 	}
 
+uint64 r = 0;
+for (uint64 i = 0; i < 100000; i++) r += (r ^ i);
+Assert(r != 0);
+
 	/* Obtain share-lock on the buffer so we can examine visibility */
 	LockBuffer(hscan->xs_cbuf, BUFFER_LOCK_SHARE);
 	got_heap_tuple = heap_hot_search_buffer(tid,
1143065-bt.txttext/plain; charset=UTF-8; name=1143065-bt.txtDownload
#20Noah Misch
noah@leadboat.com
In reply to: Alexander Lakhin (#19)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

On Thu, Oct 31, 2024 at 05:00:02PM +0300, Alexander Lakhin wrote:

I've accidentally discovered an incorrect behaviour caused by commit
4eac5a1fa. Running this script:

Thanks. This looks important.

parallel -j40 --linebuffer --tag .../reproi.sh ::: `seq 40`

This didn't reproduce it for me, at -j20, -j40, or -j80. I tested at commit
fb7e27a. At what commit(s) does it reproduce for you? At what commits, if
any, did your test not reproduce this?

All three autovacuum workers (1143263, 1143320, 1143403) are also waiting
for the same buffer lock:
#5� 0x0000561dd715f1fe in PGSemaphoreLock (sema=0x7fed9a817338) at pg_sema.c:327
#6� 0x0000561dd722fe02 in LWLockAcquire (lock=0x7fed9ad9b4e4, mode=LW_SHARED) at lwlock.c:1318
#7� 0x0000561dd71f8423 in LockBuffer (buffer=36, mode=1) at bufmgr.c:4182

Can you share the full backtrace for the autovacuum workers?

This looks like four backends all waiting for BUFFER_LOCK_SHARE on the same
pg_class page. One backend is in CREATE TABLE, and three are in autovacuum.
There are no other apparent processes that would hold the
BUFFER_LOCK_EXCLUSIVE blocking these four processes.

Also as a side note, these processes can't be terminated with SIGTERM, I
have to kill them.

That suggests they're trying to acquire one LWLock while holding another.
I'll recreate your CREATE TABLE stack trace and study its conditions. It's
not readily clear to me how that would end up holding relevant lwlocks.

Guessing how this happened did lead me to a bad decision in commit a07e03f,
but I expect fixing that bad decision won't fix the hang you captured. That
commit made index_update_stats() needlessly call RelationGetNumberOfBlocks()
and visibilitymap_count() with a pg_class heap buffer lock held. Both do I/O,
and the latter can exclusive-lock a visibility map buffer. The attached patch
corrects that. Since the hang you captured involved a pg_class heap buffer
lock, I don't think this patch will fix that hang. The other inplace updaters
are free from similar badness.

Attachments:

inplace230-index_update_stats-io-before-buflock-v1.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    

diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 74d0f30..bef1af6 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2806,6 +2806,9 @@ index_update_stats(Relation rel,
 				   bool hasindex,
 				   double reltuples)
 {
+	bool		update_stats;
+	BlockNumber relpages;
+	BlockNumber relallvisible;
 	Oid			relid = RelationGetRelid(rel);
 	Relation	pg_class;
 	ScanKeyData key[1];
@@ -2815,6 +2818,42 @@ index_update_stats(Relation rel,
 	bool		dirty;
 
 	/*
+	 * As a special hack, if we are dealing with an empty table and the
+	 * existing reltuples is -1, we leave that alone.  This ensures that
+	 * creating an index as part of CREATE TABLE doesn't cause the table to
+	 * prematurely look like it's been vacuumed.  The final rd_rel may be
+	 * different from rel->rd_rel due to e.g. commit of concurrent GRANT, but
+	 * the commands that change reltuples take locks conflicting with ours.
+	 * (Even if a command changed reltuples under a weaker lock, this affects
+	 * only statistics for an empty table.)
+	 */
+	if (reltuples == 0 && rel->rd_rel->reltuples < 0)
+		reltuples = -1;
+
+	/*
+	 * Don't update statistics during binary upgrade, because the indexes are
+	 * created before the data is moved into place.
+	 */
+	update_stats = reltuples >= 0 && !IsBinaryUpgrade;
+
+	/*
+	 * Finish I/O and visibility map buffer locks before
+	 * systable_inplace_update_begin() locks the pg_class buffer.  The final
+	 * rd_rel may be different from rel->rd_rel due to e.g. commit of
+	 * concurrent GRANT, but no command changes a relkind from non-index to
+	 * index.  (Even if one did, relallvisible doesn't break functionality.)
+	 */
+	if (update_stats)
+	{
+		relpages = RelationGetNumberOfBlocks(rel);
+
+		if (rel->rd_rel->relkind != RELKIND_INDEX)
+			visibilitymap_count(rel, &relallvisible, NULL);
+		else					/* don't bother for indexes */
+			relallvisible = 0;
+	}
+
+	/*
 	 * We always update the pg_class row using a non-transactional,
 	 * overwrite-in-place update.  There are several reasons for this:
 	 *
@@ -2858,15 +2897,6 @@ index_update_stats(Relation rel,
 	/* Should this be a more comprehensive test? */
 	Assert(rd_rel->relkind != RELKIND_PARTITIONED_INDEX);
 
-	/*
-	 * As a special hack, if we are dealing with an empty table and the
-	 * existing reltuples is -1, we leave that alone.  This ensures that
-	 * creating an index as part of CREATE TABLE doesn't cause the table to
-	 * prematurely look like it's been vacuumed.
-	 */
-	if (reltuples == 0 && rd_rel->reltuples < 0)
-		reltuples = -1;
-
 	/* Apply required updates, if any, to copied tuple */
 
 	dirty = false;
@@ -2876,20 +2906,8 @@ index_update_stats(Relation rel,
 		dirty = true;
 	}
 
-	/*
-	 * Avoid updating statistics during binary upgrade, because the indexes
-	 * are created before the data is moved into place.
-	 */
-	if (reltuples >= 0 && !IsBinaryUpgrade)
+	if (update_stats)
 	{
-		BlockNumber relpages = RelationGetNumberOfBlocks(rel);
-		BlockNumber relallvisible;
-
-		if (rd_rel->relkind != RELKIND_INDEX)
-			visibilitymap_count(rel, &relallvisible, NULL);
-		else					/* don't bother for indexes */
-			relallvisible = 0;
-
 		if (rd_rel->relpages != (int32) relpages)
 		{
 			rd_rel->relpages = (int32) relpages;
#21Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#20)
Re: Inval reliability, especially for inplace updates

On Thu, Oct 31, 2024 at 01:01:39PM -0700, Noah Misch wrote:

On Thu, Oct 31, 2024 at 05:00:02PM +0300, Alexander Lakhin wrote:

I've accidentally discovered an incorrect behaviour caused by commit
4eac5a1fa. Running this script:

Thanks. This looks important.

parallel -j40 --linebuffer --tag .../reproi.sh ::: `seq 40`

This didn't reproduce it for me, at -j20, -j40, or -j80. I tested at commit
fb7e27a. At what commit(s) does it reproduce for you? At what commits, if
any, did your test not reproduce this?

I reproduced this using a tmpfs current working directory.

All three autovacuum workers (1143263, 1143320, 1143403) are also waiting
for the same buffer lock:
#5� 0x0000561dd715f1fe in PGSemaphoreLock (sema=0x7fed9a817338) at pg_sema.c:327
#6� 0x0000561dd722fe02 in LWLockAcquire (lock=0x7fed9ad9b4e4, mode=LW_SHARED) at lwlock.c:1318
#7� 0x0000561dd71f8423 in LockBuffer (buffer=36, mode=1) at bufmgr.c:4182

Can you share the full backtrace for the autovacuum workers?

Here, one of the autovacuum workers had the guilty stack trace, appearing at
the end of this message. heap_inplace_update_and_unlock() calls
CacheInvalidateHeapTupleInplace() while holding BUFFER_LOCK_EXCLUSIVE on a
buffer of pg_class. CacheInvalidateHeapTupleInplace() may call
CatalogCacheInitializeCache(), which opens the cache's rel. If there's not a
valid relcache entry for the catcache's rel, we scan pg_class to make a valid
relcache entry. The ensuing hang makes sense.

Tomorrow, I'll think more about fixes. Two that might work:

1. Call CacheInvalidateHeapTupleInplace() before locking the buffer. Each
time we need to re-find the tuple, discard the previous try's inplace
invals and redo CacheInvalidateHeapTupleInplace(). That's because
concurrent activity may have changed cache key fields like relname.

2. Add some function that we call before locking the buffer. Its charter is
to ensure PrepareToInvalidateCacheTuple() won't have to call
CatalogCacheInitializeCache(). I think nothing resets catcache to the
extent that CatalogCacheInitializeCache() must happen again, so this should
suffice regardless of concurrent sinval traffic, debug_discard_caches, etc.

What else is worth considering? Any preferences among those?

#6 0x0000555e8a3c3afc in LWLockAcquire (lock=0x7f1683a65424, mode=LW_SHARED) at lwlock.c:1287
#7 0x0000555e8a08cec8 in heap_prepare_pagescan (sscan=sscan@entry=0x7f168e99ba48) at heapam.c:512
#8 0x0000555e8a08d6f1 in heapgettup_pagemode (scan=scan@entry=0x7f168e99ba48, dir=<optimized out>, nkeys=<optimized out>, key=<optimized out>) at heapam.c:979
#9 0x0000555e8a08dc9e in heap_getnextslot (sscan=0x7f168e99ba48, direction=<optimized out>, slot=0x7f168e9a58c0) at heapam.c:1299
#10 0x0000555e8a0aad3d in table_scan_getnextslot (direction=ForwardScanDirection, slot=<optimized out>, sscan=<optimized out>)
at ../../../../src/include/access/tableam.h:1080
#11 systable_getnext (sysscan=sysscan@entry=0x7f168e9a79c8) at genam.c:538
#12 0x0000555e8a501391 in ScanPgRelation (targetRelId=<optimized out>, indexOK=false, force_non_historic=force_non_historic@entry=false) at relcache.c:388
#13 0x0000555e8a507ea4 in RelationReloadIndexInfo (relation=0x7f168f1b8808) at relcache.c:2272
#14 RelationRebuildRelation (relation=0x7f168f1b8808) at relcache.c:2573
#15 0x0000555e8a5083e5 in RelationFlushRelation (relation=0x7f168f1b8808) at relcache.c:2848
#16 RelationCacheInvalidateEntry (relationId=<optimized out>) at relcache.c:2910
#17 0x0000555e8a4fafaf in LocalExecuteInvalidationMessage (msg=0x7ffec99ff8a0) at inval.c:795
#18 0x0000555e8a3b711a in ReceiveSharedInvalidMessages (invalFunction=invalFunction@entry=0x555e8a4faf10 <LocalExecuteInvalidationMessage>,
resetFunction=resetFunction@entry=0x555e8a4fa650 <InvalidateSystemCaches>) at sinval.c:88
#19 0x0000555e8a4fa677 in AcceptInvalidationMessages () at inval.c:865
#20 0x0000555e8a3bc809 in LockRelationOid (relid=1259, lockmode=1) at lmgr.c:135
#21 0x0000555e8a05155d in relation_open (relationId=relationId@entry=1259, lockmode=lockmode@entry=1) at relation.c:55
#22 0x0000555e8a0dd809 in table_open (relationId=relationId@entry=1259, lockmode=lockmode@entry=1) at table.c:44
#23 0x0000555e8a501359 in ScanPgRelation (targetRelId=<optimized out>, indexOK=indexOK@entry=true, force_non_historic=force_non_historic@entry=false)
at relcache.c:371
#24 0x0000555e8a507f9a in RelationReloadNailed (relation=0x7f168f1738c8) at relcache.c:2380
#25 RelationRebuildRelation (relation=0x7f168f1738c8) at relcache.c:2579
#26 0x0000555e8a5083e5 in RelationFlushRelation (relation=0x7f168f1738c8) at relcache.c:2848
#27 RelationCacheInvalidateEntry (relationId=<optimized out>) at relcache.c:2910
#28 0x0000555e8a4fafaf in LocalExecuteInvalidationMessage (msg=0x7ffec99ffc80) at inval.c:795
#29 0x0000555e8a3b71aa in ReceiveSharedInvalidMessages (invalFunction=invalFunction@entry=0x555e8a4faf10 <LocalExecuteInvalidationMessage>,
resetFunction=resetFunction@entry=0x555e8a4fa650 <InvalidateSystemCaches>) at sinval.c:118
#30 0x0000555e8a4fa677 in AcceptInvalidationMessages () at inval.c:865
#31 0x0000555e8a3bc809 in LockRelationOid (relid=1259, lockmode=1) at lmgr.c:135
#32 0x0000555e8a05155d in relation_open (relationId=1259, lockmode=lockmode@entry=1) at relation.c:55
#33 0x0000555e8a0dd809 in table_open (relationId=<optimized out>, lockmode=lockmode@entry=1) at table.c:44
#34 0x0000555e8a4f7211 in CatalogCacheInitializeCache (cache=cache@entry=0x555eb26d2180) at catcache.c:1045
#35 0x0000555e8a4f9a68 in PrepareToInvalidateCacheTuple (relation=relation@entry=0x7f168f1738c8, tuple=tuple@entry=0x7f168e9a74e8, newtuple=newtuple@entry=0x0,
function=function@entry=0x555e8a4fa220 <RegisterCatcacheInvalidation>, context=context@entry=0x7f168e9a7428) at catcache.c:2326
#36 0x0000555e8a4fa500 in CacheInvalidateHeapTupleCommon (relation=relation@entry=0x7f168f1738c8, tuple=tuple@entry=0x7f168e9a74e8, newtuple=newtuple@entry=0x0,
prepare_callback=prepare_callback@entry=0x555e8a4fa320 <PrepareInplaceInvalidationState>) at inval.c:1391
#37 0x0000555e8a4fabcb in CacheInvalidateHeapTupleCommon (prepare_callback=0x555e8a4fa320 <PrepareInplaceInvalidationState>, newtuple=newtuple@entry=0x0,
tuple=tuple@entry=0x7f168e9a74e8, relation=relation@entry=0x7f168f1738c8) at inval.c:1504
#38 0x0000555e8a094f7c in heap_inplace_update_and_unlock (relation=0x7f168f1738c8, oldtup=0x555eb277ea58, tuple=0x7f168e9a74e8, buffer=3) at heapam.c:6354
#39 0x0000555e8a0ab35a in systable_inplace_update_finish (state=0x7f168e9a72c8, tuple=<optimized out>) at genam.c:891
#40 0x0000555e8a1fbdc4 in vac_update_relstats (relation=relation@entry=0x7f168f1beae8, num_pages=num_pages@entry=19, num_tuples=<optimized out>,
num_all_visible_pages=<optimized out>, hasindex=hasindex@entry=true, frozenxid=frozenxid@entry=0, minmulti=minmulti@entry=0, frozenxid_updated=0x0,
minmulti_updated=0x0, in_outer_xact=false) at vacuum.c:1545
#41 0x0000555e8a18c97c in do_analyze_rel (onerel=onerel@entry=0x7f168f1beae8, params=params@entry=0x555eb272e2e4, va_cols=va_cols@entry=0x0,
acquirefunc=<optimized out>, relpages=19, inh=inh@entry=false, in_outer_xact=false, elevel=13) at analyze.c:643
#42 0x0000555e8a18e122 in analyze_rel (relid=<optimized out>, relation=<optimized out>, params=params@entry=0x555eb272e2e4, va_cols=0x0,
in_outer_xact=<optimized out>, bstrategy=bstrategy@entry=0x555eb2741238) at analyze.c:249
#43 0x0000555e8a1fc883 in vacuum (relations=0x555eb274b370, relations@entry=0x555eb2749390, params=params@entry=0x555eb272e2e4,
bstrategy=bstrategy@entry=0x555eb2741238, vac_context=vac_context@entry=0x555eb274b220, isTopLevel=isTopLevel@entry=true) at vacuum.c:635
#44 0x0000555e8a330e69 in autovacuum_do_vac_analyze (bstrategy=<optimized out>, tab=<optimized out>) at autovacuum.c:3108
#45 do_autovacuum () at autovacuum.c:2425
#46 0x0000555e8a33132f in AutoVacWorkerMain (startup_data=<optimized out>, startup_data_len=<optimized out>) at autovacuum.c:1571
...
(gdb) p num_held_lwlocks
$2 = 1

#22Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#21)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

On Thu, Oct 31, 2024 at 09:20:52PM -0700, Noah Misch wrote:

Here, one of the autovacuum workers had the guilty stack trace, appearing at
the end of this message. heap_inplace_update_and_unlock() calls
CacheInvalidateHeapTupleInplace() while holding BUFFER_LOCK_EXCLUSIVE on a
buffer of pg_class. CacheInvalidateHeapTupleInplace() may call
CatalogCacheInitializeCache(), which opens the cache's rel. If there's not a
valid relcache entry for the catcache's rel, we scan pg_class to make a valid
relcache entry. The ensuing hang makes sense.

Tomorrow, I'll think more about fixes. Two that might work:

1. Call CacheInvalidateHeapTupleInplace() before locking the buffer. Each
time we need to re-find the tuple, discard the previous try's inplace
invals and redo CacheInvalidateHeapTupleInplace(). That's because
concurrent activity may have changed cache key fields like relname.

Attached. With this, I got no hangs in 1.9h of your test procedure. Without
the patch, I got fourteen hangs in the same amount of time.

2. Add some function that we call before locking the buffer. Its charter is
to ensure PrepareToInvalidateCacheTuple() won't have to call
CatalogCacheInitializeCache().

The existing InitCatalogCachePhase2() could satisfy that charter. I liked
this a little less than (1), for two reasons. First, confirming that it
avoids deadlocks requires thinking through more of the catcache and relcache
procedures. Second, InitCatalogCachePhase2() would init unnecessary caches.

I think nothing resets catcache to the
extent that CatalogCacheInitializeCache() must happen again, so this should
suffice regardless of concurrent sinval traffic, debug_discard_caches, etc.

The comment at CatalogCacheFlushCatalog() confirms that.

What else is worth considering? Any preferences among those?

3. Let a process take LW_SHARED if it already holds LW_EXCLUSIVE.

That would change outcomes well beyond inplace update, and it feels like a
bandage for doing things in a suboptimal order. (1) or (2) would be an
improvement even if we did (3), since they would reduce I/O done while holding
BUFFER_LOCK_EXCLUSIVE.

This was a near miss to having a worst-in-years regression in a minor release,
so I'm proposing this sequence:

- Revert from non-master branches commits 8e7e672 (inplace180, "WAL-log
inplace update before revealing it to other sessions.") and 243e9b4
(inplace160, "For inplace update, send nontransactional invalidations.").

- Back-patch inplace230-index_update_stats-io-before-buflock to harden commit
a07e03f (inplace110, "Fix data loss at inplace update after heap_update()").

- Push attached inplace240 to master.

- Make the commitfest entry a request for review of v17 inplace160+inplace240.
After some amount of additional review and master bake time, the reverted
patches would return to non-master branches.

If someone agrees or if nobody objects by 2024-11-02T15:00+0000, I'll make it
so. That's not much time, but I want to minimize buildfarm members hanging
and maximize inplace230 bake time before the release wrap.

Less urgently, we likely should add defense in depth against adding
rarely-reached LWLock deadlocks. Perhaps one or both of:

- Assert-fail if entering a conditional CatalogCacheInitializeCache() caller
(e.g. SearchCatCacheInternal()) while holding a catalog buffer lock.

- elog(ERROR) instead of sleeping in LWLockAcquire() if the current process is
the current lock holder.

Thanks,
nm

Attachments:

inplace240-buffer-lock-vs-CatalogCacheInitializeCache-v1.patchtext/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    Fix inplace update buffer self-deadlock.
    
    A CacheInvalidateHeapTuple* callee might call
    CatalogCacheInitializeCache(), which needs a relcache entry.  Acquiring
    a valid relcache entry might scan pg_class.  Hence, to prevent
    undetected LWLock self-deadlock, CacheInvalidateHeapTuple* callers must
    not hold BUFFER_LOCK_EXCLUSIVE on buffers of pg_class.  Move the
    CacheInvalidateHeapTupleInplace() before the BUFFER_LOCK_EXCLUSIVE.
    Commit 243e9b40f1b2dd09d6e5bf91ebf6e822a2cd3704 introduced this
    regression.  No back-patch, since I've reverted the culprit from
    non-master branches.
    
    Reported by Alexander Lakhin.  Reviewed by FIXME.
    
    Discussion: https://postgr.es/m/10ec0bc3-5933-1189-6bb8-5dec4114558e@gmail.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1748eaf..9a31cdc 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6214,6 +6214,17 @@ heap_inplace_lock(Relation relation,
 
 	Assert(BufferIsValid(buffer));
 
+	/*
+	 * Construct shared cache inval if necessary.  Because we pass a tuple
+	 * version without our own inplace changes or inplace changes other
+	 * sessions complete while we wait for locks, inplace update mustn't
+	 * change catcache lookup keys.  But we aren't bothering with index
+	 * updates either, so that's true a fortiori.  After LockBuffer(), it
+	 * would be too late, because this might reach a
+	 * CatalogCacheInitializeCache() that locks "buffer".
+	 */
+	CacheInvalidateHeapTupleInplace(relation, oldtup_ptr, NULL);
+
 	LockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
 	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
 
@@ -6309,6 +6320,7 @@ heap_inplace_lock(Relation relation,
 	if (!ret)
 	{
 		UnlockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
+		ForgetInplace_Inval();
 		InvalidateCatalogSnapshot();
 	}
 	return ret;
@@ -6345,14 +6357,6 @@ heap_inplace_update_and_unlock(Relation relation,
 	dst = (char *) htup + htup->t_hoff;
 	src = (char *) tuple->t_data + tuple->t_data->t_hoff;
 
-	/*
-	 * 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,
@@ -6481,6 +6485,7 @@ heap_inplace_unlock(Relation relation,
 {
 	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 	UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
+	ForgetInplace_Inval();
 }
 
 #define		FRM_NOOP				0x0001
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index 986850c..fc972ed 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -1202,6 +1202,18 @@ AtInplace_Inval(void)
 }
 
 /*
+ * ForgetInplace_Inval
+ *		Alternative to PreInplace_Inval()+AtInplace_Inval(): discard queued-up
+ *		invalidations.  This lets inplace update enumerate invalidations
+ *		optimistically, before locking the buffer.
+ */
+void
+ForgetInplace_Inval(void)
+{
+	inplaceInvalInfo = NULL;
+}
+
+/*
  * AtEOSubXact_Inval
  *		Process queued-up invalidation messages at end of subtransaction.
  *
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 3390e7a..299cd75 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -30,6 +30,7 @@ extern void AtEOXact_Inval(bool isCommit);
 
 extern void PreInplace_Inval(void);
 extern void AtInplace_Inval(void);
+extern void ForgetInplace_Inval(void);
 
 extern void AtEOSubXact_Inval(bool isCommit);
 
#23Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#22)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

On Fri, Nov 01, 2024 at 04:38:29PM -0700, Noah Misch wrote:

This was a near miss to having a worst-in-years regression in a minor release,
so I'm proposing this sequence:

- Revert from non-master branches commits 8e7e672 (inplace180, "WAL-log
inplace update before revealing it to other sessions.") and 243e9b4
(inplace160, "For inplace update, send nontransactional invalidations.").

- Back-patch inplace230-index_update_stats-io-before-buflock to harden commit
a07e03f (inplace110, "Fix data loss at inplace update after heap_update()").

- Push attached inplace240 to master.

- Make the commitfest entry a request for review of v17 inplace160+inplace240.
After some amount of additional review and master bake time, the reverted
patches would return to non-master branches.

If someone agrees or if nobody objects by 2024-11-02T15:00+0000, I'll make it
so. That's not much time, but I want to minimize buildfarm members hanging
and maximize inplace230 bake time before the release wrap.

Pushed as 0bada39.

Buildfarm member hornet REL_15_STABLE was in the same hang. Other buildfarm
runs 2024-10-25T13:51:02Z - 2024-11-02T16:04:56Z may hang the same way. It's
early to make a comprehensive list of hung buildfarm members, since many
reported infrequently even before this period. I'll wait a week or two and
then contact the likely-hung member owners. I regret the damage.

To make explicit something I didn't call out above, v12-v17 commit a4668c9 "At
end of recovery, reset all sinval-managed caches." is still in the tree.
That's despite it originating to serve the now-reverted back branch versions
of 243e9b4. It could prevent post-recovery corruption, and its risks are
orthogonal to the risks of the reverted code.

I'm attaching the v17 patch that now becomes the commitfest submission
associated with this thread.

Attachments:

inplace160-inval-durability-inplace-v7.patch_v17text/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Back-patch to v17 - v13 (all supported versions).  This is a back-patch
    of commits 243e9b40f1b2dd09d6e5bf91ebf6e822a2cd3704 and
    0bada39c83a150079567a6e97b1a25a198f30ea3.  It reverses commit
    c1099dd745b0135960895caa8892a1873ac6cbe5, my revert of the original
    back-patch of 243e9b4.  Like the original back-patch, this doesn't
    change WAL, because these branches use end-of-recovery SIResetAll().
    All branches change the ABI of extern function
    PrepareToInvalidateCacheTuple().  No PGXN extension calls that, and
    there's no apparent use case in extensions.
    
    Reviewed by FIXME, (in earlier versions) Nitin Motiani, and (in earlier
    versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index bbe64b1..bf2bdac 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6232,6 +6232,17 @@ heap_inplace_lock(Relation relation,
 
 	Assert(BufferIsValid(buffer));
 
+	/*
+	 * Construct shared cache inval if necessary.  Because we pass a tuple
+	 * version without our own inplace changes or inplace changes other
+	 * sessions complete while we wait for locks, inplace update mustn't
+	 * change catcache lookup keys.  But we aren't bothering with index
+	 * updates either, so that's true a fortiori.  After LockBuffer(), it
+	 * would be too late, because this might reach a
+	 * CatalogCacheInitializeCache() that locks "buffer".
+	 */
+	CacheInvalidateHeapTupleInplace(relation, oldtup_ptr, NULL);
+
 	LockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
 	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
 
@@ -6327,6 +6338,7 @@ heap_inplace_lock(Relation relation,
 	if (!ret)
 	{
 		UnlockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
+		ForgetInplace_Inval();
 		InvalidateCatalogSnapshot();
 	}
 	return ret;
@@ -6355,6 +6367,16 @@ heap_inplace_update_and_unlock(Relation relation,
 	if (oldlen != newlen || htup->t_hoff != tuple->t_data->t_hoff)
 		elog(ERROR, "wrong tuple length");
 
+	/*
+	 * 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();
 
@@ -6398,17 +6420,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);
@@ -6423,6 +6456,7 @@ heap_inplace_unlock(Relation relation,
 {
 	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 	UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
+	ForgetInplace_Inval();
 }
 
 /*
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4cecf63..053a200 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1358,14 +1358,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 6c47b3c..5a1de05 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2905,12 +2905,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 05a6de6..a586d24 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 8ec5adf..03f1617 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -508,23 +508,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 111d8a2..ea8ca0e 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2288,7 +2288,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;
@@ -2329,7 +2330,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)
 		{
@@ -2338,7 +2339,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 603aa41..8e311ca 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,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 										msgs,
 										n * sizeof(SharedInvalidationMessage)),
 								 nmsgs += n));
-	ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 								RelCacheMsgs,
 								(memcpy(msgarray + nmsgs,
 										msgs,
@@ -1024,7 +1081,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 +1097,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 +1120,56 @@ 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;
+}
+
+/*
+ * ForgetInplace_Inval
+ *		Alternative to PreInplace_Inval()+AtInplace_Inval(): discard queued-up
+ *		invalidations.  This lets inplace update enumerate invalidations
+ *		optimistically, before locking the buffer.
+ */
+void
+ForgetInplace_Inval(void)
+{
+	inplaceInvalInfo = NULL;
+}
+
+/*
  * AtEOSubXact_Inval
  *		Process queued-up invalidation messages at end of subtransaction.
  *
@@ -1082,9 +1191,20 @@ void
 AtEOSubXact_Inval(bool isCommit)
 {
 	int			my_level;
-	TransInvalidationInfo *myInfo = transInvalInfo;
+	TransInvalidationInfo *myInfo;
 
-	/* Quick exit if no messages. */
+	/*
+	 * 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 transactional messages. */
+	myInfo = transInvalInfo;
 	if (myInfo == NULL)
 		return;
 
@@ -1125,12 +1245,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 +1297,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 +1305,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 +1343,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 +1353,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 +1431,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 +1487,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1362,15 +1511,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 +1531,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1399,14 +1546,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 +1566,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 +1755,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 50c9440..f41b1c2 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".
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..299cd75 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,10 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+extern void ForgetInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +41,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 5628b3f..3b57e78 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1253,6 +1253,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
#24Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#23)
Re: Inval reliability, especially for inplace updates

On Sun, Nov 03, 2024 at 10:29:25AM -0800, Noah Misch wrote:

Pushed as 0bada39.

Buildfarm member hornet REL_15_STABLE was in the same hang. Other buildfarm
runs 2024-10-25T13:51:02Z - 2024-11-02T16:04:56Z may hang the same way. It's
early to make a comprehensive list of hung buildfarm members, since many
reported infrequently even before this period. I'll wait a week or two and
then contact the likely-hung member owners. I regret the damage.

Buildfarm members plover and (less likely) sevengill might be hung in this
way. Owners (bcc'd): if convenient, please check whether a buildfarm run that
started before 2024-11-02T16:04:56Z is still ongoing. If it is, please use
"kill -QUIT" on one of the long-running autovacuum workers.

If it's the same hang, one autovacuum worker will have stack frames like:

#37 0x0000555e8a4fabcb in CacheInvalidateHeapTupleCommon (prepare_callback=0x555e8a4fa320 <PrepareInplaceInvalidationState>, newtuple=newtuple@entry=0x0,
tuple=tuple@entry=0x7f168e9a74e8, relation=relation@entry=0x7f168f1738c8) at inval.c:1504
#38 0x0000555e8a094f7c in heap_inplace_update_and_unlock (relation=0x7f168f1738c8, oldtup=0x555eb277ea58, tuple=0x7f168e9a74e8, buffer=3) at heapam.c:6354

#25Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#23)
3 attachment(s)
Re: Inval reliability, especially for inplace updates

On Sun, Nov 03, 2024 at 10:29:25AM -0800, Noah Misch wrote:

On Fri, Nov 01, 2024 at 04:38:29PM -0700, Noah Misch wrote:

This was a near miss to having a worst-in-years regression in a minor release,
so I'm proposing this sequence:

- Revert from non-master branches commits 8e7e672 (inplace180, "WAL-log
inplace update before revealing it to other sessions.") and 243e9b4
(inplace160, "For inplace update, send nontransactional invalidations.").

- Back-patch inplace230-index_update_stats-io-before-buflock to harden commit
a07e03f (inplace110, "Fix data loss at inplace update after heap_update()").

- Push attached inplace240 to master.

- Make the commitfest entry a request for review of v17 inplace160+inplace240.
After some amount of additional review and master bake time, the reverted
patches would return to non-master branches.

Pushed as 0bada39.

I'm attaching the v17 patch that now becomes the commitfest submission

That still applies cleanly. I'm adding a back-patch of commit f4ece89, which
adds assertions intended to build confidence about the main patch. The
commitfest entry requests a review of the first patch only, but the second
patch may answer questions that a review of the first would otherwise raise.

Attachments:

inplace160-inval-durability-inplace-v7.patch_v17text/plain; charset=us-asciiDownload
Author:     Noah Misch <noah@leadboat.com>
Commit:     Noah Misch <noah@leadboat.com>

    For inplace update, send nontransactional invalidations.
    
    The inplace update survives ROLLBACK.  The inval didn't, so another
    backend's DDL could then update the row without incorporating the
    inplace update.  In the test this fixes, a mix of CREATE INDEX and ALTER
    TABLE resulted in a table with an index, yet relhasindex=f.  That is a
    source of index corruption.
    
    Back-patch to v17 - v13 (all supported versions).  This is a back-patch
    of commits 243e9b40f1b2dd09d6e5bf91ebf6e822a2cd3704 and
    0bada39c83a150079567a6e97b1a25a198f30ea3.  It reverses commit
    c1099dd745b0135960895caa8892a1873ac6cbe5, my revert of the original
    back-patch of 243e9b4.  Like the original back-patch, this doesn't
    change WAL, because these branches use end-of-recovery SIResetAll().
    All branches change the ABI of extern function
    PrepareToInvalidateCacheTuple().  No PGXN extension calls that, and
    there's no apparent use case in extensions.
    
    Reviewed by FIXME, (in earlier versions) Nitin Motiani, and (in earlier
    versions) Andres Freund.
    
    Discussion: https://postgr.es/m/20240523000548.58.nmisch@google.com

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index bbe64b1..bf2bdac 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6232,6 +6232,17 @@ heap_inplace_lock(Relation relation,
 
 	Assert(BufferIsValid(buffer));
 
+	/*
+	 * Construct shared cache inval if necessary.  Because we pass a tuple
+	 * version without our own inplace changes or inplace changes other
+	 * sessions complete while we wait for locks, inplace update mustn't
+	 * change catcache lookup keys.  But we aren't bothering with index
+	 * updates either, so that's true a fortiori.  After LockBuffer(), it
+	 * would be too late, because this might reach a
+	 * CatalogCacheInitializeCache() that locks "buffer".
+	 */
+	CacheInvalidateHeapTupleInplace(relation, oldtup_ptr, NULL);
+
 	LockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
 	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
 
@@ -6327,6 +6338,7 @@ heap_inplace_lock(Relation relation,
 	if (!ret)
 	{
 		UnlockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
+		ForgetInplace_Inval();
 		InvalidateCatalogSnapshot();
 	}
 	return ret;
@@ -6355,6 +6367,16 @@ heap_inplace_update_and_unlock(Relation relation,
 	if (oldlen != newlen || htup->t_hoff != tuple->t_data->t_hoff)
 		elog(ERROR, "wrong tuple length");
 
+	/*
+	 * 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();
 
@@ -6398,17 +6420,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);
@@ -6423,6 +6456,7 @@ heap_inplace_unlock(Relation relation,
 {
 	LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 	UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
+	ForgetInplace_Inval();
 }
 
 /*
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4cecf63..053a200 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1358,14 +1358,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 6c47b3c..5a1de05 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2905,12 +2905,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 05a6de6..a586d24 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 8ec5adf..03f1617 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -508,23 +508,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 111d8a2..ea8ca0e 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -2288,7 +2288,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;
@@ -2329,7 +2330,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)
 		{
@@ -2338,7 +2339,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 603aa41..8e311ca 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,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
 										msgs,
 										n * sizeof(SharedInvalidationMessage)),
 								 nmsgs += n));
-	ProcessMessageSubGroupMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
+	ProcessMessageSubGroupMulti(&transInvalInfo->ii.CurrentCmdInvalidMsgs,
 								RelCacheMsgs,
 								(memcpy(msgarray + nmsgs,
 										msgs,
@@ -1024,7 +1081,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 +1097,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 +1120,56 @@ 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;
+}
+
+/*
+ * ForgetInplace_Inval
+ *		Alternative to PreInplace_Inval()+AtInplace_Inval(): discard queued-up
+ *		invalidations.  This lets inplace update enumerate invalidations
+ *		optimistically, before locking the buffer.
+ */
+void
+ForgetInplace_Inval(void)
+{
+	inplaceInvalInfo = NULL;
+}
+
+/*
  * AtEOSubXact_Inval
  *		Process queued-up invalidation messages at end of subtransaction.
  *
@@ -1082,9 +1191,20 @@ void
 AtEOSubXact_Inval(bool isCommit)
 {
 	int			my_level;
-	TransInvalidationInfo *myInfo = transInvalInfo;
+	TransInvalidationInfo *myInfo;
 
-	/* Quick exit if no messages. */
+	/*
+	 * 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 transactional messages. */
+	myInfo = transInvalInfo;
 	if (myInfo == NULL)
 		return;
 
@@ -1125,12 +1245,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 +1297,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 +1305,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 +1343,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 +1353,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 +1431,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 +1487,13 @@ CacheInvalidateCatalog(Oid catalogId)
 {
 	Oid			databaseId;
 
-	PrepareInvalidationState();
-
 	if (IsSharedRelation(catalogId))
 		databaseId = InvalidOid;
 	else
 		databaseId = MyDatabaseId;
 
-	RegisterCatalogInvalidation(databaseId, catalogId);
+	RegisterCatalogInvalidation(PrepareInvalidationState(),
+								databaseId, catalogId);
 }
 
 /*
@@ -1362,15 +1511,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 +1531,8 @@ CacheInvalidateRelcache(Relation relation)
 void
 CacheInvalidateRelcacheAll(void)
 {
-	PrepareInvalidationState();
-
-	RegisterRelcacheInvalidation(InvalidOid, InvalidOid);
+	RegisterRelcacheInvalidation(PrepareInvalidationState(),
+								 InvalidOid, InvalidOid);
 }
 
 /*
@@ -1399,14 +1546,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 +1566,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 +1755,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 50c9440..f41b1c2 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".
diff --git a/src/include/utils/catcache.h b/src/include/utils/catcache.h
index 3fb9647..8f04bb8 100644
--- a/src/include/utils/catcache.h
+++ b/src/include/utils/catcache.h
@@ -225,6 +225,7 @@ extern void CatCacheInvalidate(CatCache *cache, uint32 hashValue);
 extern void PrepareToInvalidateCacheTuple(Relation relation,
 										  HeapTuple tuple,
 										  HeapTuple newtuple,
-										  void (*function) (int, uint32, Oid));
+										  void (*function) (int, uint32, Oid, void *),
+										  void *context);
 
 #endif							/* CATCACHE_H */
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 24695fa..299cd75 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -28,6 +28,10 @@ extern void AcceptInvalidationMessages(void);
 
 extern void AtEOXact_Inval(bool isCommit);
 
+extern void PreInplace_Inval(void);
+extern void AtInplace_Inval(void);
+extern void ForgetInplace_Inval(void);
+
 extern void AtEOSubXact_Inval(bool isCommit);
 
 extern void PostPrepare_Inval(void);
@@ -37,6 +41,9 @@ extern void CommandEndInvalidationMessages(void);
 extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
+extern void CacheInvalidateHeapTupleInplace(Relation relation,
+											HeapTuple tuple,
+											HeapTuple newtuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
diff --git a/src/test/isolation/expected/inplace-inval.out b/src/test/isolation/expected/inplace-inval.out
index e68eca5..c35895a 100644
--- a/src/test/isolation/expected/inplace-inval.out
+++ b/src/test/isolation/expected/inplace-inval.out
@@ -1,6 +1,6 @@
 Parsed test spec with 3 sessions
 
-starting permutation: cachefill3 cir1 cic2 ddl3
+starting permutation: cachefill3 cir1 cic2 ddl3 read1
 step cachefill3: TABLE newly_indexed;
 c
 -
@@ -9,6 +9,14 @@ c
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
 step cic2: CREATE INDEX i2 ON newly_indexed (c);
 step ddl3: ALTER TABLE newly_indexed ADD extra int;
+step read1: 
+	SELECT relhasindex FROM pg_class WHERE oid = 'newly_indexed'::regclass;
+
+relhasindex
+-----------
+t          
+(1 row)
+
 
 starting permutation: cir1 cic2 ddl3 read1
 step cir1: BEGIN; CREATE INDEX i1 ON newly_indexed (c); ROLLBACK;
diff --git a/src/test/isolation/specs/inplace-inval.spec b/src/test/isolation/specs/inplace-inval.spec
index 96954fd..b99112d 100644
--- a/src/test/isolation/specs/inplace-inval.spec
+++ b/src/test/isolation/specs/inplace-inval.spec
@@ -1,7 +1,7 @@
-# If a heap_update() caller retrieves its oldtup from a cache, it's possible
-# for that cache entry to predate an inplace update, causing loss of that
-# inplace update.  This arises because the transaction may abort before
-# sending the inplace invalidation message to the shared queue.
+# An inplace update had been able to abort before sending the inplace
+# invalidation message to the shared queue.  If a heap_update() caller then
+# retrieved its oldtup from a cache, the heap_update() could revert the
+# inplace update.
 
 setup
 {
@@ -27,14 +27,12 @@ step cachefill3	{ TABLE newly_indexed; }
 step ddl3		{ ALTER TABLE newly_indexed ADD extra int; }
 
 
-# XXX shows an extant bug.  Adding step read1 at the end would usually print
-# relhasindex=f (not wanted).  This does not reach the unwanted behavior under
-# -DCATCACHE_FORCE_RELEASE and friends.
 permutation
 	cachefill3	# populates the pg_class row in the catcache
 	cir1	# sets relhasindex=true; rollback discards cache inval
 	cic2	# sees relhasindex=true, skips changing it (so no inval)
 	ddl3	# cached row as the oldtup of an update, losing relhasindex
+	read1	# observe damage
 
 # without cachefill3, no bug
 permutation cir1 cic2 ddl3 read1
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 5628b3f..3b57e78 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1253,6 +1253,7 @@ Interval
 IntervalAggState
 IntoClause
 InvalMessageArray
+InvalidationInfo
 InvalidationMsgsGroup
 IpcMemoryId
 IpcMemoryKey
inplace900-assert-catcache-vs-buflock-v1.patch_v17text/plain; charset=us-asciiDownload
From: Noah Misch <noah@leadboat.com>

Assert lack of hazardous buffer locks before possible catalog read.

Commit 0bada39c83a150079567a6e97b1a25a198f30ea3 fixed a bug of this kind,
which existed in all branches for six days before detection.  While the
probability of reaching the trouble was low, the disruption was extreme.  No
new backends could start, and service restoration needed an immediate
shutdown.  Hence, add this to catch the next bug like it.

The new check in RelationIdGetRelation() suffices to make autovacuum detect
the bug in commit 243e9b40f1b2dd09d6e5bf91ebf6e822a2cd3704 that led to commit
0bada39.  This also checks in a number of similar places.  It replaces each
Assert(IsTransactionState()) that pertained to a conditional catalog read.

This a back-patch of commit f4ece891fc2f3f96f0571720a1ae30db8030681b to
all supported branches, to accompany the back-patch of commits 243e9b4
and 0bada39.  For catalog indexes, the bttextcmp() behavior that
motivated IsCatalogTextUniqueIndexOid() was v18-specific, so this
back-patch doesn't need that or the correction of that from commit
4a4ee0c2c1e53401924101945ac3d517c0a8a559.

Reported-by: Alexander Lakhin <exclusion@gmail.com>
Discussion: https://postgr.es/m/20250410191830.0e.nmisch@google.com
Discussion: https://postgr.es/m/10ec0bc3-5933-1189-6bb8-5dec4114558e@gmail.com
Backpatch-through: 13-17

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index f8d30bf..7daf1be 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -40,6 +40,9 @@
 #include "access/tableam.h"
 #include "access/xloginsert.h"
 #include "access/xlogutils.h"
+#ifdef USE_ASSERT_CHECKING
+#include "catalog/pg_tablespace_d.h"
+#endif
 #include "catalog/storage.h"
 #include "catalog/storage_xlog.h"
 #include "executor/instrument.h"
@@ -535,6 +538,10 @@ static void RelationCopyStorageUsingBuffer(RelFileLocator srclocator,
 										   ForkNumber forkNum, bool permanent);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
+#ifdef USE_ASSERT_CHECKING
+static void AssertNotCatalogBufferLock(LWLock *lock, LWLockMode mode,
+									   void *unused_context);
+#endif
 static int	rlocator_comparator(const void *p1, const void *p2);
 static inline int buffertag_comparator(const BufferTag *ba, const BufferTag *bb);
 static inline int ckpt_buforder_comparator(const CkptSortItem *a, const CkptSortItem *b);
@@ -3647,6 +3654,66 @@ CheckForBufferLeaks(void)
 #endif
 }
 
+#ifdef USE_ASSERT_CHECKING
+/*
+ * Check for exclusive-locked catalog buffers.  This is the core of
+ * AssertCouldGetRelation().
+ *
+ * A backend would self-deadlock on LWLocks if the catalog scan read the
+ * exclusive-locked buffer.  The main threat is exclusive-locked buffers of
+ * catalogs used in relcache, because a catcache search on any catalog may
+ * build that catalog's relcache entry.  We don't have an inventory of
+ * catalogs relcache uses, so just check buffers of most catalogs.
+ *
+ * It's better to minimize waits while holding an exclusive buffer lock, so it
+ * would be nice to broaden this check not to be catalog-specific.  However,
+ * bttextcmp() accesses pg_collation, and non-core opclasses might similarly
+ * read tables.  That is deadlock-free as long as there's no loop in the
+ * dependency graph: modifying table A may cause an opclass to read table B,
+ * but it must not cause a read of table A.
+ */
+void
+AssertBufferLocksPermitCatalogRead(void)
+{
+	ForEachLWLockHeldByMe(AssertNotCatalogBufferLock, NULL);
+}
+
+static void
+AssertNotCatalogBufferLock(LWLock *lock, LWLockMode mode,
+						   void *unused_context)
+{
+	BufferDesc *bufHdr;
+	BufferTag	tag;
+	Oid			relid;
+
+	if (mode != LW_EXCLUSIVE)
+		return;
+
+	if (!((BufferDescPadded *) lock > BufferDescriptors &&
+		  (BufferDescPadded *) lock < BufferDescriptors + NBuffers))
+		return;					/* not a buffer lock */
+
+	bufHdr = (BufferDesc *)
+		((char *) lock - offsetof(BufferDesc, content_lock));
+	tag = bufHdr->tag;
+
+	/*
+	 * This relNumber==relid assumption holds until a catalog experiences
+	 * VACUUM FULL or similar.  After a command like that, relNumber will be
+	 * in the normal (non-catalog) range, and we lose the ability to detect
+	 * hazardous access to that catalog.  Calling RelidByRelfilenumber() would
+	 * close that gap, but RelidByRelfilenumber() might then deadlock with a
+	 * held lock.
+	 */
+	relid = tag.relNumber;
+
+	Assert(!IsCatalogRelationOid(relid));
+	/* Shared rels are always catalogs: detect even after VACUUM FULL. */
+	Assert(tag.spcOid != GLOBALTABLESPACE_OID);
+}
+#endif
+
+
 /*
  * Helper routine to issue warnings when a buffer is unexpectedly pinned
  */
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 5df62fa..ea83110 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -1887,6 +1887,21 @@ LWLockReleaseAll(void)
 
 
 /*
+ * ForEachLWLockHeldByMe - run a callback for each held lock
+ *
+ * This is meant as debug support only.
+ */
+void
+ForEachLWLockHeldByMe(void (*callback) (LWLock *, LWLockMode, void *),
+					  void *context)
+{
+	int			i;
+
+	for (i = 0; i < num_held_lwlocks; i++)
+		callback(held_lwlocks[i].lock, held_lwlocks[i].mode, context);
+}
+
+/*
  * LWLockHeldByMe - test whether my process holds a lock in any mode
  *
  * This is meant as debug support only.
diff --git a/src/backend/utils/adt/pg_locale.c b/src/backend/utils/adt/pg_locale.c
index 4c85a01..d2b16d8 100644
--- a/src/backend/utils/adt/pg_locale.c
+++ b/src/backend/utils/adt/pg_locale.c
@@ -66,6 +66,7 @@
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/pg_locale.h"
+#include "utils/relcache.h"
 #include "utils/syscache.h"
 
 #ifdef USE_ICU
@@ -1258,6 +1259,8 @@ lookup_collation_cache(Oid collation, bool set_flags)
 	Assert(OidIsValid(collation));
 	Assert(collation != DEFAULT_COLLATION_OID);
 
+	AssertCouldGetRelation();
+
 	if (collation_cache == NULL)
 	{
 		/* First time through, initialize the hash table */
diff --git a/src/backend/utils/cache/catcache.c b/src/backend/utils/cache/catcache.c
index aab29f5..3d4228b 100644
--- a/src/backend/utils/cache/catcache.c
+++ b/src/backend/utils/cache/catcache.c
@@ -1055,11 +1055,40 @@ RehashCatCacheLists(CatCache *cp)
 }
 
 /*
+ *		ConditionalCatalogCacheInitializeCache
+ *
+ * Call CatalogCacheInitializeCache() if not yet done.
+ */
+pg_attribute_always_inline
+static void
+ConditionalCatalogCacheInitializeCache(CatCache *cache)
+{
+#ifdef USE_ASSERT_CHECKING
+	/*
+	 * TypeCacheRelCallback() runs outside transactions and relies on TYPEOID
+	 * for hashing.  This isn't ideal.  Since lookup_type_cache() both
+	 * registers the callback and searches TYPEOID, reaching trouble likely
+	 * requires OOM at an unlucky moment.
+	 *
+	 * InvalidateAttoptCacheCallback() runs outside transactions and likewise
+	 * relies on ATTNUM.  InitPostgres() initializes ATTNUM, so it's reliable.
+	 */
+	if (!(cache->id == TYPEOID || cache->id == ATTNUM) ||
+		IsTransactionState())
+		AssertCouldGetRelation();
+	else
+		Assert(cache->cc_tupdesc != NULL);
+#endif
+
+	if (unlikely(cache->cc_tupdesc == NULL))
+		CatalogCacheInitializeCache(cache);
+}
+
+/*
  *		CatalogCacheInitializeCache
  *
  * This function does final initialization of a catcache: obtain the tuple
- * descriptor and set up the hash and equality function links.  We assume
- * that the relcache entry can be opened at this point!
+ * descriptor and set up the hash and equality function links.
  */
 #ifdef CACHEDEBUG
 #define CatalogCacheInitializeCache_DEBUG1 \
@@ -1194,8 +1223,7 @@ CatalogCacheInitializeCache(CatCache *cache)
 void
 InitCatCachePhase2(CatCache *cache, bool touch_index)
 {
-	if (cache->cc_tupdesc == NULL)
-		CatalogCacheInitializeCache(cache);
+	ConditionalCatalogCacheInitializeCache(cache);
 
 	if (touch_index &&
 		cache->id != AMOID &&
@@ -1374,16 +1402,12 @@ SearchCatCacheInternal(CatCache *cache,
 	dlist_head *bucket;
 	CatCTup    *ct;
 
-	/* Make sure we're in an xact, even if this ends up being a cache hit */
-	Assert(IsTransactionState());
-
 	Assert(cache->cc_nkeys == nkeys);
 
 	/*
 	 * one-time startup overhead for each cache
 	 */
-	if (unlikely(cache->cc_tupdesc == NULL))
-		CatalogCacheInitializeCache(cache);
+	ConditionalCatalogCacheInitializeCache(cache);
 
 #ifdef CATCACHE_STATS
 	cache->cc_searches++;
@@ -1669,8 +1693,7 @@ GetCatCacheHashValue(CatCache *cache,
 	/*
 	 * one-time startup overhead for each cache
 	 */
-	if (cache->cc_tupdesc == NULL)
-		CatalogCacheInitializeCache(cache);
+	ConditionalCatalogCacheInitializeCache(cache);
 
 	/*
 	 * calculate the hash value
@@ -1721,8 +1744,7 @@ SearchCatCacheList(CatCache *cache,
 	/*
 	 * one-time startup overhead for each cache
 	 */
-	if (unlikely(cache->cc_tupdesc == NULL))
-		CatalogCacheInitializeCache(cache);
+	ConditionalCatalogCacheInitializeCache(cache);
 
 	Assert(nkeys > 0 && nkeys < cache->cc_nkeys);
 
@@ -2392,8 +2414,7 @@ PrepareToInvalidateCacheTuple(Relation relation,
 			continue;
 
 		/* Just in case cache hasn't finished initialization yet... */
-		if (ccp->cc_tupdesc == NULL)
-			CatalogCacheInitializeCache(ccp);
+		ConditionalCatalogCacheInitializeCache(ccp);
 
 		hashvalue = CatalogCacheComputeTupleHashValue(ccp, ccp->cc_nkeys, tuple);
 		dbid = ccp->cc_relisshared ? (Oid) 0 : MyDatabaseId;
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index 21ac3c2..82297fa 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -631,7 +631,8 @@ PrepareInvalidationState(void)
 {
 	TransInvalidationInfo *myInfo;
 
-	Assert(IsTransactionState());
+	/* PrepareToInvalidateCacheTuple() needs relcache */
+	AssertCouldGetRelation();
 	/* Can't queue transactional message while collecting inplace messages. */
 	Assert(inplaceInvalInfo == NULL);
 
@@ -700,7 +701,7 @@ PrepareInplaceInvalidationState(void)
 {
 	InvalidationInfo *myInfo;
 
-	Assert(IsTransactionState());
+	AssertCouldGetRelation();
 	/* limit of one inplace update under assembly */
 	Assert(inplaceInvalInfo == NULL);
 
@@ -863,6 +864,12 @@ InvalidateSystemCaches(void)
 void
 AcceptInvalidationMessages(void)
 {
+#ifdef USE_ASSERT_CHECKING
+	/* message handlers shall access catalogs only during transactions */
+	if (IsTransactionState())
+		AssertCouldGetRelation();
+#endif
+
 	ReceiveSharedInvalidMessages(LocalExecuteInvalidationMessage,
 								 InvalidateSystemCaches);
 
@@ -1327,6 +1334,9 @@ CacheInvalidateHeapTupleCommon(Relation relation,
 	Oid			databaseId;
 	Oid			relationId;
 
+	/* PrepareToInvalidateCacheTuple() needs relcache */
+	AssertCouldGetRelation();
+
 	/* Do nothing during bootstrap */
 	if (IsBootstrapProcessingMode())
 		return;
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 3f1e8ce..9e0519c 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -2037,6 +2037,23 @@ formrdesc(const char *relationName, Oid relationReltype,
 	relation->rd_isvalid = true;
 }
 
+#ifdef USE_ASSERT_CHECKING
+/*
+ *		AssertCouldGetRelation
+ *
+ *		Check safety of calling RelationIdGetRelation().
+ *
+ *		In code that reads catalogs in the event of a cache miss, call this
+ *		before checking the cache.
+ */
+void
+AssertCouldGetRelation(void)
+{
+	Assert(IsTransactionState());
+	AssertBufferLocksPermitCatalogRead();
+}
+#endif
+
 
 /* ----------------------------------------------------------------
  *				 Relation Descriptor Lookup Interface
@@ -2064,8 +2081,7 @@ RelationIdGetRelation(Oid relationId)
 {
 	Relation	rd;
 
-	/* Make sure we're in an xact, even if this ends up being a cache hit */
-	Assert(IsTransactionState());
+	AssertCouldGetRelation();
 
 	/*
 	 * first try to find reldesc in the cache
diff --git a/src/backend/utils/mb/mbutils.c b/src/backend/utils/mb/mbutils.c
index 97a4d69..6af3b8e 100644
--- a/src/backend/utils/mb/mbutils.c
+++ b/src/backend/utils/mb/mbutils.c
@@ -39,6 +39,7 @@
 #include "mb/pg_wchar.h"
 #include "utils/fmgrprotos.h"
 #include "utils/memutils.h"
+#include "utils/relcache.h"
 #include "varatt.h"
 
 /*
@@ -310,7 +311,7 @@ InitializeClientEncoding(void)
 	{
 		Oid			utf8_to_server_proc;
 
-		Assert(IsTransactionState());
+		AssertCouldGetRelation();
 		utf8_to_server_proc =
 			FindDefaultConversionProc(PG_UTF8,
 									  current_server_encoding);
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index a1e7101..09c43dd 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -255,6 +255,9 @@ extern Buffer ExtendBufferedRelTo(BufferManagerRelation bmr,
 
 extern void InitBufferPoolAccess(void);
 extern void AtEOXact_Buffers(bool isCommit);
+#ifdef USE_ASSERT_CHECKING
+extern void AssertBufferLocksPermitCatalogRead(void);
+#endif
 extern char *DebugPrintBufferRefcount(Buffer buffer);
 extern void CheckPointBuffers(int flags);
 extern BlockNumber BufferGetBlockNumber(Buffer buffer);
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index d70e6d3..5435948 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -129,6 +129,8 @@ extern bool LWLockAcquireOrWait(LWLock *lock, LWLockMode mode);
 extern void LWLockRelease(LWLock *lock);
 extern void LWLockReleaseClearVar(LWLock *lock, pg_atomic_uint64 *valptr, uint64 val);
 extern void LWLockReleaseAll(void);
+extern void ForEachLWLockHeldByMe(void (*callback) (LWLock *, LWLockMode, void *),
+								  void *context);
 extern bool LWLockHeldByMe(LWLock *lock);
 extern bool LWLockAnyHeldByMe(LWLock *lock, int nlocks, size_t stride);
 extern bool LWLockHeldByMeInMode(LWLock *lock, LWLockMode mode);
diff --git a/src/include/utils/relcache.h b/src/include/utils/relcache.h
index 18c32ea..6cb829e 100644
--- a/src/include/utils/relcache.h
+++ b/src/include/utils/relcache.h
@@ -37,6 +37,14 @@ typedef Relation *RelationPtr;
 /*
  * Routines to open (lookup) and close a relcache entry
  */
+#ifdef USE_ASSERT_CHECKING
+extern void AssertCouldGetRelation(void);
+#else
+static inline void
+AssertCouldGetRelation(void)
+{
+}
+#endif
 extern Relation RelationIdGetRelation(Oid relationId);
 extern void RelationClose(Relation relation);
 
non-master-commitfest-entry-v1.patchtext/plain; charset=us-asciiDownload
From: Noah Misch <noah@leadboat.com>



diff --git a/NON-MASTER-COMMITFEST-ENTRY b/NON-MASTER-COMMITFEST-ENTRY
new file mode 100644
index 0000000..b942c35
--- /dev/null
+++ b/NON-MASTER-COMMITFEST-ENTRY
@@ -0,0 +1,3 @@
+This commitfest entry proposes to change only non-master branches.  The cfbot
+is specific to the master branch.  Hence, this patch file exists to placate
+the cfbot.  See the other attachments for the real patches needing review.
#26Paul A Jungwirth
pj@illuminatedcomputing.com
In reply to: Noah Misch (#25)
Re: Inval reliability, especially for inplace updates

Ian Ilyasov and I reviewed this patch. We think it is ready to commit
to back branches.

The attached patch applies to REL_17_STABLE but not other stable
branches, so we assume Noah will adjust it as needed.

We were able to reproduce Alexander Lakhin's hang when we tested
against 0a0a0f2c59 (i.e. before the previous version was reverted),
although adding the delay was necessary. With this patch applied, we
don't see the hang (even with the delay).

We agree that the new assertions are a good idea to prevent similar
errors in the future.

We couldn't devise other ways to break this patch. Surely more
experienced hackers could be more creative, but nonetheless it's
reassuring that the patch's twin has been in v18devel since November.

We assume you will also un-revert 8e7e672cda ("WAL-log inplace update
before revealing it to other sessions.")? We didn't look closely at
that patch, but it seems like there are no known problems with it. It
was just reverted because it depends on this patch.

Since this is a backpatch, it doesn't seem right to give non-essential
feedback. Here are a few thoughts anyway. Consider them notes for
future work rather than reasons to delay backpatching or drift from
the patch on master.

Is there any way to add more testing around non-transactional
invalidations? It is a new "feature" but it is not really tested
anywhere. I don't think we could do this with regress tests, but
perhaps isolation tests would be suitable.

Some of the comments felt a bit compressed. They make sense in the
context of this fix, but reading them cold seems like it will be
challenging. For example this took a lot of thinking to follow:

* Construct shared cache inval if necessary. Because we pass a
tuple
* version without our own inplace changes or inplace changes
other
* sessions complete while we wait for locks, inplace update
mustn't
* change catcache lookup keys. But we aren't bothering with
index
* updates either, so that's true a fortiori.

Or this:

* WAL contains likely-unnecessary commit-time invals from the
* CacheInvalidateHeapTuple() call in heap_inplace_update().

Why likely-unnecessary? I know you explain it at that callsite, but
some hint might help here.

It's a bit surprising that wrongly leaving relhasindex=t is safe (for
example after BEGIN; CREATE INDEX; ROLLBACK;). I guess this column is
just to save us a lookup for tables with no index, and no harm is done
if we do the lookup needlessly but find no indexes. And vacuum can
repair it later. Still it's a little unnerving.

On Thu, Oct 31, 2024 at 09:20:52PM -0700, Noah Misch wrote:

Here, one of the autovacuum workers had the guilty stack trace, appearing at
the end of this message. heap_inplace_update_and_unlock() calls
CacheInvalidateHeapTupleInplace() while holding BUFFER_LOCK_EXCLUSIVE on a
buffer of pg_class. CacheInvalidateHeapTupleInplace() may call
CatalogCacheInitializeCache(), which opens the cache's rel. If there's not a
valid relcache entry for the catcache's rel, we scan pg_class to make a valid
relcache entry. The ensuing hang makes sense.

Personally I never expected that catcache could depend on relcache,
since it seems lower-level. But it makes sense that you need a
relcache of pg_class at least, so their relationship is more
complicated than just layers.

I'm struggling to understand how your explanation incorporates
*concurrency*, since a self-deadlock only involves locks from one
backend. But the point is that a concurrent invalidation causes the
relcache entry to vanish, so that we need to rebuild it. (We can't get
this far without having built the relcache for pg_class once already.)

Specifically, we drop the table while autovacuum is updating its
statistics. But how is that possible? Don't both those things
exclusive-lock the row in pg_class? I must be misunderstanding.

Tomorrow, I'll think more about fixes. Two that might work:

1. Call CacheInvalidateHeapTupleInplace() before locking the buffer. Each
time we need to re-find the tuple, discard the previous try's inplace
invals and redo CacheInvalidateHeapTupleInplace(). That's because
concurrent activity may have changed cache key fields like relname.

We agree that choice 1 is a good approach.

 PrepareToInvalidateCacheTuple(Relation relation,
                               HeapTuple tuple,
                               HeapTuple newtuple,
-                              void (*function) (int, uint32, Oid))
+                              void (*function) (int, uint32, Oid, void *),
+                              void *context)

It's a little odd that PrepareToInvalidateCacheTuple takes a callback
function when it only has one caller, so it always calls
RegisterCatcacheInvalidation. Is it just to avoid adding dependencies
to inval.c? But it already #includes catcache.h and contains lots of
knowledge about catcache specifics. Maybe originally
PrepareToInvalidateCacheTuple was built to take
RegisterRelcacheInvalidation as well? Is it worth still passing the
callback?

@@ -6511,6 +6544,7 @@ heap_inplace_unlock(Relation relation,
{
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
+ ForgetInplace_Inval();
}

Is this the right place to add this? We think on balance yes, but the
question crossed my mind: Clearing the invals seems like a separate
responsibility from unlocking the buffer & tuple. After this patch,
our only remaining caller of heap_inplace_unlock is
systable_inplace_update_cancel, so perhaps it should call
ForgetInplace_Inval itself? OTOH we like that putting it here
guarantees it gets called, as a complement to building the invals in
heap_inplace_lock.

Yours,

--
Paul ~{:-)
pj@illuminatedcomputing.com

#27Paul A Jungwirth
pj@illuminatedcomputing.com
In reply to: Paul A Jungwirth (#26)
Re: Inval reliability, especially for inplace updates

On Thu, Jul 31, 2025 at 9:53 AM Paul A Jungwirth
<pj@illuminatedcomputing.com> wrote:

On Thu, Oct 31, 2024 at 09:20:52PM -0700, Noah Misch wrote:

Here, one of the autovacuum workers had the guilty stack trace, appearing at
the end of this message. heap_inplace_update_and_unlock() calls
CacheInvalidateHeapTupleInplace() while holding BUFFER_LOCK_EXCLUSIVE on a
buffer of pg_class. CacheInvalidateHeapTupleInplace() may call
CatalogCacheInitializeCache(), which opens the cache's rel. If there's not a
valid relcache entry for the catcache's rel, we scan pg_class to make a valid
relcache entry. The ensuing hang makes sense.

Personally I never expected that catcache could depend on relcache,
since it seems lower-level. But it makes sense that you need a
relcache of pg_class at least, so their relationship is more
complicated than just layers.

I'm struggling to understand how your explanation incorporates
*concurrency*, since a self-deadlock only involves locks from one
backend. But the point is that a concurrent invalidation causes the
relcache entry to vanish, so that we need to rebuild it. (We can't get
this far without having built the relcache for pg_class once already.)

Specifically, we drop the table while autovacuum is updating its
statistics. But how is that possible? Don't both those things
exclusive-lock the row in pg_class? I must be misunderstanding.

I was curious so I decided to look into this some more. We have a
self-deadlock, but it's rare. Concurrency is necessary to trigger it.
The patch fixes the double-locking, but what was the sequence that
caused the problem?

The only thing reproi.sh does is add & drop tables over and over.

I looked at a few reproductions, and the situation was always the
same: one of the autovacuum workers was stuck analyzing pg_attribute,
with a stack trace like this (no different than Noah's):

#0 __futex_abstimed_wait_common
(futex_word=futex_word@entry=0x7fa76af77238,
expected=expected@entry=0, clockid=clockid@entry=0,
abstime=abstime@entry=0x0, private=<optimized out>,
cancel=cancel@entry=true) at ./nptl/futex-internal.c:103
#1 0x00007fa7742d5f7b in __GI___futex_abstimed_wait_cancelable64
(futex_word=futex_word@entry=0x7fa76af77238,
expected=expected@entry=0,
clockid=clockid@entry=0, abstime=abstime@entry=0x0,
private=<optimized out>) at ./nptl/futex-internal.c:139
#2 0x00007fa7742e0cff in do_futex_wait (sem=sem@entry=0x7fa76af77238,
abstime=0x0, clockid=0) at ./nptl/sem_waitcommon.c:111
#3 0x00007fa7742e0d90 in __new_sem_wait_slow64
(sem=sem@entry=0x7fa76af77238, abstime=0x0, clockid=0) at
./nptl/sem_waitcommon.c:183
#4 0x00007fa7742e0df9 in __new_sem_wait
(sem=sem@entry=0x7fa76af77238) at ./nptl/sem_wait.c:42
#5 0x00005583c641a6a8 in PGSemaphoreLock (sema=0x7fa76af77238) at
pg_sema.c:327
#6 0x00005583c652e1da in LWLockAcquire (lock=0x7fa76b4fb4e4,
mode=mode@entry=LW_SHARED) at lwlock.c:1318
#7 0x00005583c64df801 in LockBuffer (buffer=36, mode=mode@entry=1) at
bufmgr.c:4182
#8 0x00005583c5eda4b3 in heapam_index_fetch_tuple
(scan=0x7fa76adc1b30, tid=0x7fa76adc3fb8, snapshot=0x5583e8ede3d8,
slot=0x7fa76adc1030,
call_again=0x7fa76adc3fbe, all_dead=0x7ffebf643e6f) at
heapam_handler.c:146
#9 0x00005583c5efb548 in table_index_fetch_tuple (scan=<optimized
out>, tid=tid@entry=0x7fa76adc3fb8, snapshot=<optimized out>,
slot=slot@entry=0x7fa76adc1030,
call_again=call_again@entry=0x7fa76adc3fbe,
all_dead=all_dead@entry=0x7ffebf643e6f)
at ../../../../src/include/access/tableam.h:1226
#10 0x00005583c5efd1a7 in index_fetch_heap
(scan=scan@entry=0x7fa76adc3f58, slot=slot@entry=0x7fa76adc1030) at
indexam.c:622
#11 0x00005583c5efd362 in index_getnext_slot (scan=0x7fa76adc3f58,
direction=direction@entry=ForwardScanDirection, slot=0x7fa76adc1030)
at indexam.c:682
#12 0x00005583c5efa32a in systable_getnext
(sysscan=sysscan@entry=0x7fa76adc3dd0) at genam.c:512
#13 0x00005583c677fd20 in ScanPgRelation (targetRelId=<optimized out>,
indexOK=indexOK@entry=true,
force_non_historic=force_non_historic@entry=false) at
relcache.c:385
#14 0x00005583c6780cf8 in RelationReloadNailed
(relation=relation@entry=0x7fa7740fd698) at relcache.c:2381
#15 0x00005583c678b67e in RelationClearRelation
(relation=relation@entry=0x7fa7740fd698, rebuild=<optimized out>) at
relcache.c:2527
#16 0x00005583c678ce85 in RelationFlushRelation
(relation=0x7fa7740fd698) at relcache.c:2839
#17 0x00005583c678cf00 in RelationCacheInvalidateEntry
(relationId=<optimized out>) at relcache.c:2900
#18 0x00005583c67714fe in LocalExecuteInvalidationMessage
(msg=0x7ffebf644220) at inval.c:666
#19 0x00005583c651585b in ReceiveSharedInvalidMessages (
invalFunction=invalFunction@entry=0x5583c677130d
<LocalExecuteInvalidationMessage>,
resetFunction=resetFunction@entry=0x5583c67700cf
<InvalidateSystemCaches>) at sinval.c:121
#20 0x00005583c677012c in AcceptInvalidationMessages () at inval.c:766
#21 0x00005583c651f8d2 in LockRelationOid (relid=<optimized out>,
lockmode=1) at lmgr.c:137
#22 0x00005583c5e503db in relation_open
(relationId=relationId@entry=1259, lockmode=lockmode@entry=1) at
relation.c:56
#23 0x00005583c5f595e5 in table_open
(relationId=relationId@entry=1259, lockmode=lockmode@entry=1) at
table.c:43
#24 0x00005583c677fc8c in ScanPgRelation (targetRelId=<optimized out>,
indexOK=<optimized out>,
force_non_historic=force_non_historic@entry=false) at
relcache.c:368
#25 0x00005583c67807c1 in RelationReloadIndexInfo
(relation=relation@entry=0x7fa774141028) at relcache.c:2257
#26 0x00005583c6780dff in RelationReloadNailed
(relation=relation@entry=0x7fa774141028) at relcache.c:2359
#27 0x00005583c678b67e in RelationClearRelation
(relation=relation@entry=0x7fa774141028, rebuild=<optimized out>) at
relcache.c:2527
#28 0x00005583c678ce85 in RelationFlushRelation
(relation=0x7fa774141028) at relcache.c:2839
#29 0x00005583c678cf00 in RelationCacheInvalidateEntry
(relationId=<optimized out>) at relcache.c:2900
#30 0x00005583c67714fe in LocalExecuteInvalidationMessage
(msg=0x7ffebf644640) at inval.c:666
#31 0x00005583c651585b in ReceiveSharedInvalidMessages (
invalFunction=invalFunction@entry=0x5583c677130d
<LocalExecuteInvalidationMessage>,
resetFunction=resetFunction@entry=0x5583c67700cf
<InvalidateSystemCaches>) at sinval.c:121
#32 0x00005583c677012c in AcceptInvalidationMessages () at inval.c:766
#33 0x00005583c651f8d2 in LockRelationOid (relid=<optimized out>,
lockmode=1) at lmgr.c:137
#34 0x00005583c5e503db in relation_open
(relationId=relationId@entry=1259, lockmode=lockmode@entry=1) at
relation.c:56
#35 0x00005583c5f595e5 in table_open
(relationId=relationId@entry=1259, lockmode=lockmode@entry=1) at
table.c:43
#36 0x00005583c677fc8c in ScanPgRelation (targetRelId=<optimized out>,
indexOK=indexOK@entry=true,
force_non_historic=force_non_historic@entry=false) at relcache.c:368
#37 0x00005583c6780cf8 in RelationReloadNailed
(relation=relation@entry=0x7fa7740fd698) at relcache.c:2381
#38 0x00005583c678b67e in RelationClearRelation
(relation=relation@entry=0x7fa7740fd698, rebuild=<optimized out>) at
relcache.c:2527
#39 0x00005583c678ce85 in RelationFlushRelation
(relation=0x7fa7740fd698) at relcache.c:2839
#40 0x00005583c678cf00 in RelationCacheInvalidateEntry
(relationId=<optimized out>) at relcache.c:2900
#41 0x00005583c67714fe in LocalExecuteInvalidationMessage
(msg=0x7ffebf644a40) at inval.c:666
#42 0x00005583c651585b in ReceiveSharedInvalidMessages (
invalFunction=invalFunction@entry=0x5583c677130d
<LocalExecuteInvalidationMessage>,
resetFunction=resetFunction@entry=0x5583c67700cf
<InvalidateSystemCaches>) at sinval.c:121
#43 0x00005583c677012c in AcceptInvalidationMessages () at inval.c:766
#44 0x00005583c651f8d2 in LockRelationOid (relid=<optimized out>,
lockmode=1) at lmgr.c:137
#45 0x00005583c5e503db in relation_open (relationId=1259,
lockmode=lockmode@entry=1) at relation.c:56
#46 0x00005583c5f595e5 in table_open (relationId=<optimized out>,
lockmode=lockmode@entry=1) at table.c:43
#47 0x00005583c676b11a in CatalogCacheInitializeCache
(cache=cache@entry=0x5583e8e79f80) at catcache.c:929
#48 0x00005583c676e3c2 in PrepareToInvalidateCacheTuple
(relation=relation@entry=0x7fa7740fd698,
tuple=tuple@entry=0x7fa76adc3a88,
newtuple=newtuple@entry=0x0,
function=function@entry=0x5583c676fac8 <RegisterCatcacheInvalidation>,
context=context@entry=0x7fa76adc38b8) at catcache.c:2164
#49 0x00005583c676fdc3 in CacheInvalidateHeapTupleCommon
(relation=relation@entry=0x7fa7740fd698,
tuple=tuple@entry=0x7fa76adc3a88,
newtuple=newtuple@entry=0x0,
prepare_callback=prepare_callback@entry=0x5583c676fe93
<PrepareInplaceInvalidationState>) at inval.c:1350
#50 0x00005583c6770ab9 in CacheInvalidateHeapTupleInplace
(relation=relation@entry=0x7fa7740fd698,
tuple=tuple@entry=0x7fa76adc3a88,
newtuple=newtuple@entry=0x0) at inval.c:1461
#51 0x00005583c5ec8dc9 in heap_inplace_update_and_unlock
(relation=0x7fa7740fd698, oldtup=<optimized out>,
tuple=0x7fa76adc3a88, buffer=36)
at heapam.c:6311
#52 0x00005583c5efb33a in systable_inplace_update_finish
(state=0x7fa76adc3748, tuple=<optimized out>) at genam.c:884
#53 0x00005583c61fbc92 in vac_update_relstats
(relation=relation@entry=0x7fa774102b20, num_pages=num_pages@entry=89,
num_tuples=4460,
num_all_visible_pages=<optimized out>,
hasindex=hasindex@entry=true, frozenxid=frozenxid@entry=0,
minmulti=minmulti@entry=0,
frozenxid_updated=0x0, minmulti_updated=0x0, in_outer_xact=false)
at vacuum.c:1455
#54 0x00005583c60ec3f2 in do_analyze_rel
(onerel=onerel@entry=0x7fa774102b20,
params=params@entry=0x5583e8e9bda4,
va_cols=va_cols@entry=0x0, acquirefunc=<optimized out>,
relpages=89, inh=inh@entry=false, in_outer_xact=false, elevel=13)
at analyze.c:645
#55 0x00005583c60ed155 in analyze_rel (relid=<optimized out>,
relation=0x5583e8ef9fa0, params=params@entry=0x5583e8e9bda4,
va_cols=0x0,
in_outer_xact=<optimized out>, bstrategy=<optimized out>) at analyze.c:262
#56 0x00005583c61fd2d4 in vacuum (relations=0x5583e8efc008,
params=params@entry=0x5583e8e9bda4, bstrategy=<optimized out>,
bstrategy@entry=0x5583e8ef2e00, isTopLevel=isTopLevel@entry=true)
at vacuum.c:493
#57 0x00005583c641e58b in autovacuum_do_vac_analyze
(tab=tab@entry=0x5583e8e9bda0,
bstrategy=bstrategy@entry=0x5583e8ef2e00)
at autovacuum.c:3180
#58 0x00005583c6420f84 in do_autovacuum () at autovacuum.c:2503
#59 0x00005583c64219ae in AutoVacWorkerMain (argc=argc@entry=0,
argv=argv@entry=0x0) at autovacuum.c:1716
#60 0x00005583c6421bb3 in StartAutoVacWorker () at autovacuum.c:1494
#61 0x00005583c6433c22 in StartAutovacuumWorker () at postmaster.c:5536
#62 0x00005583c64344ca in sigusr1_handler
(postgres_signal_arg=<optimized out>) at postmaster.c:5241
#63 <signal handler called>
#64 0x00007fa77434e904 in __GI___select (nfds=nfds@entry=8,
readfds=readfds@entry=0x7ffebf6461c0, writefds=writefds@entry=0x0,
exceptfds=exceptfds@entry=0x0,
timeout=timeout@entry=0x7ffebf6461b0) at
../sysdeps/unix/sysv/linux/select.c:69
#65 0x00005583c6435442 in ServerLoop () at postmaster.c:1773
#66 0x00005583c6437c5e in PostmasterMain (argc=argc@entry=1,
argv=argv@entry=0x5583e8e0fe00) at postmaster.c:1481
#67 0x00005583c62d57f5 in main (argc=1, argv=0x5583e8e0fe00) at main.c:202

In frame 55 we are analyzing relation 1249 (pg_attribute):

(gdb) f 55
#55 0x00005583c60ed155 in analyze_rel (relid=<optimized out>,
relation=0x5583e8ef9fa0, params=params@entry=0x5583e8e9bda4,
va_cols=0x0,
in_outer_xact=<optimized out>, bstrategy=<optimized out>) at analyze.c:262
262 do_analyze_rel(onerel, params, va_cols, acquirefunc,
(gdb) p *relation
$1 = {type = T_RangeVar, catalogname = 0x0, schemaname =
0x5583e8ef3308 "pg_catalog", relname = 0x5583e8ef32e0 "pg_attribute",
inh = true,
relpersistence = 112 'p', alias = 0x0, location = -1}
(gdb) p *onerel
$2 = {rd_node = {spcNode = 1663, dbNode = 5, relNode = 1249}, rd_smgr
= 0x5583e8eb98c0, rd_refcnt = 2, rd_backend = -1,
rd_islocaltemp = false, rd_isnailed = true, rd_isvalid = true,
rd_indexvalid = true, rd_statvalid = false, rd_createSubid = 0,
rd_newRelfilenodeSubid = 0, rd_firstRelfilenodeSubid = 0,
rd_droppedSubid = 0, rd_rel = 0x7fa774102d38, rd_att = 0x7fa774102e50,
rd_id = 1249, rd_lockInfo = {lockRelId = {relId = 1249, dbId = 5}},
rd_rules = 0x0, rd_rulescxt = 0x0, trigdesc = 0x0, rd_rsdesc = 0x0,
rd_fkeylist = 0x0, rd_fkeyvalid = false, rd_partkey = 0x0,
rd_partkeycxt = 0x0, rd_partdesc = 0x0, rd_pdcxt = 0x0,
rd_partdesc_nodetached = 0x0, rd_pddcxt = 0x0,
rd_partdesc_nodetached_xmin = 0, rd_partcheck = 0x0, rd_partcheckvalid
= false,
rd_partcheckcxt = 0x0, rd_indexlist = 0x7fa774120298, rd_pkindex =
2659, rd_replidindex = 0, rd_statlist = 0x0, rd_indexattr = 0x0,
rd_keyattr = 0x0, rd_pkattr = 0x0, rd_idattr = 0x0, rd_pubdesc =
0x0, rd_options = 0x0, rd_amhandler = 3,
rd_tableam = 0x5583c6b0bf00 <heapam_methods>, rd_index = 0x0,
rd_indextuple = 0x0, rd_indexcxt = 0x0, rd_indam = 0x0, rd_opfamily =
0x0,
rd_opcintype = 0x0, rd_support = 0x0, rd_supportinfo = 0x0,
rd_indoption = 0x0, rd_indexprs = 0x0, rd_indpred = 0x0, rd_exclops =
0x0,
rd_exclprocs = 0x0, rd_exclstrats = 0x0, rd_indcollation = 0x0,
rd_opcoptions = 0x0, rd_amcache = 0x0, rd_fdwroutine = 0x0,
rd_toastoid = 0, pgstat_enabled = true, pgstat_info = 0x5583e8eb0090}

It makes sense that pg_attribute would experience a lot of churn,
since we just add & drop tables.

We call vac_update_relstats, which does an in-place update to pg_class
(frame 53).

Within vac_update_relstats, we call system_inplace_update_begin, which
LWLocks the buffer we want to update. That's the first part of the
self-deadlock. But it's no problem yet. We proceed in
vac_update_relstats to sys_inplace_update_finish. . . .

That sends an inplace invalidation message for the pg_class tuple
(frame 50). So we need to replace the tuple in the pg_class catcache.
But before we do that, we make sure the catcache is initialized (frame
47):

/* Just in case cache hasn't finished initialization yet... */
if (ccp->cc_tupdesc == NULL)
CatalogCacheInitializeCache(ccp);

That locks pg_class with AccessShareLock, which looks for pending
invalidation messages (frame 43).

It finds one invalidating the relcache for pg_class:

(gdb) f 41
#41 0x00005583c67714fe in LocalExecuteInvalidationMessage
(msg=0x7ffebf644a40) at inval.c:666
666
RelationCacheInvalidateEntry(msg->rc.relId);
(gdb) p *msg
$9 = {id = -2 '\376', cc = {id = -2 '\376', dbId = 5, hashValue =
1259}, cat = {id = -2 '\376', dbId = 5, catId = 1259}, rc = {
id = -2 '\376', dbId = 5, relId = 1259}, sm = {id = -2 '\376',
backend_hi = -47 '\321', backend_lo = 59642, rnode = {spcNode = 5,
dbNode = 1259, relNode = 32679}}, rm = {id = -2 '\376', dbId =
5}, sn = {id = -2 '\376', dbId = 5, relId = 1259}}

(Btw where did this message come from?)

So we have to rebuild the relcache for pg_class. Obviously we need to
look up its entry in (heh) pg_class (frame 37). So we open that table
with an AccessShareLock, which looks for pending invalidation messages
(frame 32). We find one telling us to invalidate 2662
(pg_class_oid_index):

(gdb) f 30
#30 0x00005583c67714fe in LocalExecuteInvalidationMessage
(msg=0x7ffebf644640) at inval.c:666
666
RelationCacheInvalidateEntry(msg->rc.relId);
(gdb) p *msg
$12 = {id = -2 '\376', cc = {id = -2 '\376', dbId = 5, hashValue =
2662}, cat = {id = -2 '\376', dbId = 5, catId = 2662}, rc = {
id = -2 '\376', dbId = 5, relId = 2662}, sm = {id = -2 '\376',
backend_hi = -47 '\321', backend_lo = 59642, rnode = {spcNode = 5,
dbNode = 2662, relNode = 32679}}, rm = {id = -2 '\376', dbId =
5}, sn = {id = -2 '\376', dbId = 5, relId = 2662}}

I guess that is coming from the add & drop table commands.

So we reload the relcache entry for 2662 (frame 26). That opens
pg_class with AccessShareLock, which looks for pending invalidation
messages (frame 20). We find one telling us to invalidate pg_class:

(gdb) f 18
#18 0x00005583c67714fe in LocalExecuteInvalidationMessage
(msg=0x7ffebf644220) at inval.c:666
666
RelationCacheInvalidateEntry(msg->rc.relId);
(gdb) p *msg
$14 = {id = -2 '\376', cc = {id = -2 '\376', dbId = 5, hashValue =
1259}, cat = {id = -2 '\376', dbId = 5, catId = 1259}, rc = {
id = -2 '\376', dbId = 5, relId = 1259}, sm = {id = -2 '\376',
backend_hi = 75 'K', backend_lo = 48996, rnode = {spcNode = 5,
dbNode = 1259, relNode = 32679}}, rm = {id = -2 '\376', dbId =
5}, sn = {id = -2 '\376', dbId = 5, relId = 1259}}

So we rebuild the relcache for pg_class, which means scanning pg_class
(frame 14). We try to lock one of its buffers (frame 7), and we
self-deadlock.

So the problem is triggered by an invalidation message for the
pg_class relcache entry. But what sends that message? (Actually it
seems we got two of them.) Why does anything we're doing invalidate
pg_class? Normally an inplace update to pg_class sends an invalidation
message for the relation described by that tuple, not all of pg_class
itself (see CacheInvalidateHeapTupleCommon).

I couldn't figure that out, until I put a breakpoint at the top of
RegisterRelcacheInvalidation and ran `vacuum analyze`. When we analyze
pg_class (and btw the repro script gives it a lot of churn), we call
vac_update_relstats on it, so we are doing an inplace update of
pg_class on the tuple *about* pg_class. So that's why we send the
invalidation about the whole table.

The concurrency, then, is analyzing pg_attribute in one worker and
pg_class in another. No locks would prevent that. It doesn't have to
be pg_attribute; it could be some user table. But since the repro
script adds & drops tables, pg_attribute is getting churned.

Yours,

--
Paul ~{:-)
pj@illuminatedcomputing.com

#28Noah Misch
noah@leadboat.com
In reply to: Paul A Jungwirth (#27)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

On Thu, Jul 31, 2025 at 09:53:20AM -0700, Paul A Jungwirth wrote:

Ian Ilyasov and I reviewed this patch. We think it is ready to commit
to back branches.

Thank you!

We assume you will also un-revert 8e7e672cda ("WAL-log inplace update
before revealing it to other sessions.")? We didn't look closely at
that patch, but it seems like there are no known problems with it. It
was just reverted because it depends on this patch.

Right. I'm fine self-certifying that one.

Is there any way to add more testing around non-transactional
invalidations? It is a new "feature" but it is not really tested
anywhere. I don't think we could do this with regress tests, but
perhaps isolation tests would be suitable.

I think src/test/isolation/specs/inplace-inval.spec is testing it.

Some of the comments felt a bit compressed. They make sense in the
context of this fix, but reading them cold seems like it will be
challenging. For example this took a lot of thinking to follow:

* Construct shared cache inval if necessary. Because we pass a tuple
* version without our own inplace changes or inplace changes other
* sessions complete while we wait for locks, inplace update mustn't
* change catcache lookup keys. But we aren't bothering with index
* updates either, so that's true a fortiori.

What do you think of the attached rewrite? I also removed this part:

- * 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.

That would have needed s/this transaction/this command/ to be correct, and I
didn't feel it was saying something important enough to keep. There are
plenty of ways for invals to be redundant.

Or this:

* WAL contains likely-unnecessary commit-time invals from the
* CacheInvalidateHeapTuple() call in heap_inplace_update().

Why likely-unnecessary? I know you explain it at that callsite, but
some hint might help here.

On further study, I think one could construct a logical decoding output plugin
for which it's necessary. I've detailed that in the attached edits. This was
in the heap_decode() changes, which is the part of the patch I understand the
least. I likely should have made it a separate patch. Here's the surrounding
change I made in 2024, in context diff format:

*** 508,530 **** 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.
*
! * 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.
*/
- if (!TransactionIdIsValid(xid))
- break;
-
- (void) SnapBuildProcessChange(builder, xid, buf->origptr);
- ReorderBufferXidSetCatalogChanges(ctx->reorder, xid, buf->origptr);
break;

  		case XLOG_HEAP_CONFIRM:
--- 508,526 ----

/*
* Inplace updates are only ever performed on catalog tuples and
! * 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.
*
! * WAL contains likely-unnecessary commit-time invals from the
! * CacheInvalidateHeapTuple() call in heap_inplace_update().
! * Excess invalidation is safe.
*/
break;

case XLOG_HEAP_CONFIRM:

This code had been unchanged since commit b89e1510 (v9.4) introduced logical
decoding. I'm making the following assumptions about the old code:

- I guessed "decoding in-progress relations" was a typo for "decoding
in-progress transactions", something we've supported since at least commit
4648243 "Add support for streaming to built-in logical replication"
(2020-09, v14). If it's not a typo, I don't know what "in-progress
relations" denoted here.

- It said "redundant because the commit will do that as well", but I didn't
find such code. I bet that it referenced the DecodeCommit() lines removed
in commit c55040c "WAL Log invalidations at command end with
wal_level=logical" (2020-07, v14). The same commit likely made the
ReorderBufferXidSetCatalogChanges() call obsolete.

- I had no idea why we call SnapBuildProcessChange(). Every other caller uses
its return value. I guess there was a desire for its snapshot side effects,
but I didn't follow that. Nothing snapshot-relevant happens at
XLOG_HEAP_INPLACE. Removing this call doesn't break any test on v13 or on
v9.4. Similarly, no test fails after removing both this and the
ReorderBufferXidSetCatalogChanges() call.

- In v9.4, this area of code (per-heapam-record-type decoding) had inval
responsibilities. That went away in v14, so there's no need for the comment
here to keep discussing invals.

I now think it would be prudent to omit from back-patch the non-comment
heap_decode() changes. While the above assumptions argue against needing the
removed ReorderBufferXidSetCatalogChanges(), that's the sort of speculation I
should keep out back-branch changes.

It's a bit surprising that wrongly leaving relhasindex=t is safe (for
example after BEGIN; CREATE INDEX; ROLLBACK;). I guess this column is
just to save us a lookup for tables with no index, and no harm is done
if we do the lookup needlessly but find no indexes.

And vacuum can
repair it later. Still it's a little unnerving.

DROP INDEX doesn't clear it, either. That's longstanding, and it doesn't
involve narrow race conditions. Hence, I'm not worrying about it. If it were
broken, we'd have heard by now.

On Thu, Oct 31, 2024 at 09:20:52PM -0700, Noah Misch wrote:

Here, one of the autovacuum workers had the guilty stack trace, appearing at
the end of this message. heap_inplace_update_and_unlock() calls
CacheInvalidateHeapTupleInplace() while holding BUFFER_LOCK_EXCLUSIVE on a
buffer of pg_class. CacheInvalidateHeapTupleInplace() may call
CatalogCacheInitializeCache(), which opens the cache's rel. If there's not a
valid relcache entry for the catcache's rel, we scan pg_class to make a valid
relcache entry. The ensuing hang makes sense.

Personally I never expected that catcache could depend on relcache,
since it seems lower-level. But it makes sense that you need a
relcache of pg_class at least, so their relationship is more
complicated than just layers.

Yes, relcache.c is indeed ... eclectic.

PrepareToInvalidateCacheTuple(Relation relation,
HeapTuple tuple,
HeapTuple newtuple,
-                              void (*function) (int, uint32, Oid))
+                              void (*function) (int, uint32, Oid, void *),
+                              void *context)

It's a little odd that PrepareToInvalidateCacheTuple takes a callback
function when it only has one caller, so it always calls
RegisterCatcacheInvalidation. Is it just to avoid adding dependencies
to inval.c? But it already #includes catcache.h and contains lots of
knowledge about catcache specifics. Maybe originally
PrepareToInvalidateCacheTuple was built to take
RegisterRelcacheInvalidation as well? Is it worth still passing the
callback?

Looking at the history, it did have two callbacks in Postgres95 1.01. It was
down to one callback when it got the name PrepareToInvalidateCacheTuple() in
commit 81d08fc of 2001-01. I think the main alternative to today's callback
would be to make RegisterCatcacheInvalidation() an extern for catcache.c to
call. All the other Register* functions would remain static. I left things
unchanged since that would be cleaner in one way and dirtier in another.

@@ -6511,6 +6544,7 @@ heap_inplace_unlock(Relation relation,
{
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
+ ForgetInplace_Inval();
}

Is this the right place to add this? We think on balance yes, but the
question crossed my mind: Clearing the invals seems like a separate
responsibility from unlocking the buffer & tuple. After this patch,
our only remaining caller of heap_inplace_unlock is
systable_inplace_update_cancel, so perhaps it should call
ForgetInplace_Inval itself? OTOH we like that putting it here
guarantees it gets called, as a complement to building the invals in
heap_inplace_lock.

Style principles behind the placement:

- Inplace update puts some responsibilities in genam.c and others in heapam.c.
A given task, e.g. draining the inval queue, should consistently appear on
the same side of that boundary.

- The heapam.c side of inplace update should cover roughly as much as the
heapam.c side of heap_update(). Since heap_update() handles invals and the
critical section, so should the heapam.c side of inplace update.

It wouldn't take a strong reason to override those principles.

postgr.es/m/20240831011043.2b.nmisch@google.com has an inventory of the ways
to assign inval responsibility to heapam.c vs. genam.c.
postgr.es/m/20241013004511.9c.nmisch@google.com has a related discussion,
about the need for AtInplace_Inval() to be in the critical section.

On Sun, Aug 03, 2025 at 12:32:31PM -0700, Paul A Jungwirth wrote:

[details of the hang]

The concurrency, then, is analyzing pg_attribute in one worker and
pg_class in another. No locks would prevent that. It doesn't have to
be pg_attribute; it could be some user table. But since the repro
script adds & drops tables, pg_attribute is getting churned.

Thanks for that detailed write-up! I agree with your findings.

Attachments:

inplace290-comments202508-v1.patchtext/plain; charset=us-asciiDownload
From: Noah Misch <noah@leadboat.com>



diff --git a/src/backend/access/heap/README.tuplock b/src/backend/access/heap/README.tuplock
index 843c2e5..16f7d78 100644
--- a/src/backend/access/heap/README.tuplock
+++ b/src/backend/access/heap/README.tuplock
@@ -199,3 +199,35 @@ under a reader holding a pin.  A reader of a heap_fetch() result tuple may
 witness a torn read.  Current inplace-updated fields are aligned and are no
 wider than four bytes, and current readers don't need consistency across
 fields.  Hence, they get by with just fetching each field once.
+
+During logical decoding, caches reflect an inplace update no later than the
+next XLOG_XACT_INVALIDATIONS.  That record witnesses the end of a command.
+Tuples of its cmin are then visible to decoding, as are inplace updates of any
+lower LSN.  Inplace updates of a higher LSN may also be visible, even if those
+updates would have been invisible to a non-historic snapshot matching
+decoding's historic snapshot.  (In other words, decoding may see inplace
+updates that were not visible to a similar snapshot taken during original
+transaction processing.)  That's a consequence of inplace update violating
+MVCC: there are no snapshot-specific versions of inplace-updated values.  This
+all makes it hard to reason about inplace-updated column reads during logical
+decoding, but the behavior does suffice for relhasindex.  A relhasindex=t in
+CREATE INDEX becomes visible no later than the new pg_index row.  While it may
+be visible earlier, that's harmless.  Finding zero indexes despite
+relhasindex=t is normal in more cases than this, e.g. after DROP INDEX.
+Example of a case that meaningfully reacts to the inplace inval:
+
+CREATE TABLE cat (c int) WITH (user_catalog_table = true);
+CREATE TABLE normal (d int);
+...
+CREATE INDEX ON cat (c)\; INSERT INTO normal VALUES (1);
+
+If the output plugin reads "cat" during decoding of the INSERT, it's fair to
+want that read to see relhasindex=t and use the new index.
+
+An alternative would be to have decoding of XLOG_HEAP_INPLACE immediately
+execute its invals.  That would behave more like invals during original
+transaction processing.  It would remove the decoding-specific delay in e.g. a
+decoding plugin witnessing a relfrozenxid change.  However, a good use case
+for that is unlikely, since the plugin would still witness relfrozenxid
+changes prematurely.  Hence, inplace update takes the trivial approach of
+delegating to XLOG_XACT_INVALIDATIONS.
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 7491cc3..a1ef191 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6371,13 +6371,28 @@ heap_inplace_lock(Relation relation,
 	Assert(BufferIsValid(buffer));
 
 	/*
-	 * Construct shared cache inval if necessary.  Because we pass a tuple
-	 * version without our own inplace changes or inplace changes other
-	 * sessions complete while we wait for locks, inplace update mustn't
-	 * change catcache lookup keys.  But we aren't bothering with index
-	 * updates either, so that's true a fortiori.  After LockBuffer(), it
-	 * would be too late, because this might reach a
-	 * CatalogCacheInitializeCache() that locks "buffer".
+	 * Register shared cache invals if necessary.  Our input to inval can be
+	 * weaker than heap_update() input to inval in these ways:
+	 *
+	 * - This passes only the old version of the tuple.  Inval reacts only to
+	 * catcache lookup key columns and pg_class.oid values stored in
+	 * relcache-relevant catalog columns.  All of those columns are indexed.
+	 * Inplace update mustn't be used for any operations that could change
+	 * those.  Hence, the new tuple would provide no additional inval-relevant
+	 * information.  Those facts also make it fine to skip updating indexes.
+	 *
+	 * - Other sessions may finish inplace updates of this tuple between this
+	 * step and LockTuple().  That's fine for the same reason: those inplace
+	 * updates mustn't be changing columns that affect inval decisions.
+	 *
+	 * - The xwait found below may COMMIT between now and this function
+	 * returning, making the tuple dead.  That can change inval decisions, so
+	 * we'll later react to it by forgetting the inval before returning. While
+	 * it's tempting to just register invals after we've confirmed no xwait
+	 * will COMMIT, the following obstacle precludes reordering steps that
+	 * way.  Registering invals might reach a CatalogCacheInitializeCache()
+	 * that locks "buffer".  That would hang indefinitely if running after our
+	 * own LockBuffer().  Hence, we must register invals before LockBuffer().
 	 */
 	CacheInvalidateHeapTupleInplace(relation, oldtup_ptr, NULL);
 
@@ -6617,10 +6632,6 @@ heap_inplace_update_and_unlock(Relation relation,
 	/*
 	 * 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();
 
@@ -6631,10 +6642,10 @@ heap_inplace_update_and_unlock(Relation relation,
 	AcceptInvalidationMessages();	/* local processing of just-sent inval */
 
 	/*
-	 * 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.
+	 * Queue a transactional inval, for logical decoding and for third-party
+	 * code that might have been relying on it since long before inplace
+	 * update adopted immediate invalidation.  See README.tuplock section
+	 * "Reading inplace-updated columns" for logical decoding details.
 	 */
 	if (!IsBootstrapProcessingMode())
 		CacheInvalidateHeapTuple(relation, tuple, NULL);
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index cc03f07..5e15cb1 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -521,18 +521,9 @@ heap_decode(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 			/*
 			 * Inplace updates are only ever performed on catalog tuples and
-			 * 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.
-			 *
-			 * WAL contains likely-unnecessary commit-time invals from the
-			 * CacheInvalidateHeapTuple() call in
-			 * heap_inplace_update_and_unlock(). Excess invalidation is safe.
+			 * can, per definition, not change tuple visibility.  Since we
+			 * also don't decode catalog tuples, we're not interested in the
+			 * record's contents.
 			 */
 			break;
 
#29Paul A Jungwirth
pj@illuminatedcomputing.com
In reply to: Noah Misch (#28)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

Surya Poondla (cc'd) and I took another look at this as part of the
November Patch Review Workshop.

We think it looks good. But I couldn't get the latest patch to apply
on top of REL_17_STABLE until I did this:

```
git am inplace160-inval-durability-inplace-v7.patch_v17
git revert bc6bad8857 # revert the revert of "WAL-log inplace update"
git am inplace280-comment-fix-v1.patch.nocfbot # attached
git am inplace290-comments202508-v1.patch
```

The inplace280 step adds a small comment change that seems to be in
your git history, but I couldn't find it in the email chain. Also the
290 patch has context from reverting the WAL-log revert.

The patch avoids deadlocks by reordering invalidation prep before
buffer locking. While no explicit assertion exists to detect future
violations, would it be helpful to add a helper or macro that enforces
this lock ordering rule more visibly? Probably not for a backpatch,
but in master?

On Sun, Aug 24, 2025 at 4:39 PM Noah Misch <noah@leadboat.com> wrote:

Some of the comments felt a bit compressed. They make sense in the
context of this fix, but reading them cold seems like it will be
challenging. For example this took a lot of thinking to follow:

* Construct shared cache inval if necessary. Because we pass a tuple
* version without our own inplace changes or inplace changes other
* sessions complete while we wait for locks, inplace update mustn't
* change catcache lookup keys. But we aren't bothering with index
* updates either, so that's true a fortiori.

What do you think of the attached rewrite? I also removed this part:

- * 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.

That would have needed s/this transaction/this command/ to be correct, and I
didn't feel it was saying something important enough to keep. There are
plenty of ways for invals to be redundant.

Thanks for expanding on this! Here are some thoughts about the new comment:

+    * Register shared cache invals if necessary.  Our input to inval can be
+    * weaker than heap_update() input to inval in these ways:

Perhaps "than the heap_update() input" or "than heap_update()'s input"?

+    * - This passes only the old version of the tuple.  Inval reacts only to
+    * catcache lookup key columns and pg_class.oid values stored in
+    * relcache-relevant catalog columns.  All of those columns are indexed.
+    * Inplace update mustn't be used for any operations that could change
+    * those.  Hence, the new tuple would provide no additional inval-relevant
+    * information.  Those facts also make it fine to skip updating indexes.

This is confusing to me. "Inval only reacts": who is inval? Are you
talking about the other backends when they receive the message? After
spending a lot of time, I think you mean
CacheInvalidateHeapTupleCommon, how it decides whether to invalidate
first the catcache and then the relcache. Also I wondered, if an
inplace update never changes index keys, and [something] only cares
about inval messages that change index keys, why are we sending an
inval message at all? After a few months away from this patch, it was
hard for me to remember. I think the comment is a bit misleading. We
*do* send catcache invalidations if non-key columns change. What about
this?:

+ * - This passes only the old version of the tuple. Catcache
invalidation doesn't need newtuple because inplace updates never
change key columns, so it only needs to invalidate one hash value, not
two. [For the same reason, we don't need to update indexes.] Relcache
invalidation (in CacheInvalidateHeapTupleCommon) ignores newtuple
altogether, even for regular updates, because an update can never move
a tuple from one relcache entry to another.

I bracketed the line about indexes, because I don't understand why
we're talking about updating indexes here. I don't see anything about
that in CacheInvalidateHeapTupleCommon or PrepareInvalidationState
(which doesn't have access to newtuple anyway).

Also it feels like this comment (or something similar) really belongs
on CacheInvalidateHeapTupleInplace. And that function doesn't need its
newtuple parameter.

+    * - The xwait found below may COMMIT between now and this function
+    * returning, making the tuple dead.  That can change inval decisions, so

Is this third bullet point still explaining why an inplace update can
be looser about invalidating caches than heap_update? Or is it making
a separate point? It seems like it should be a paragraph, not a bullet
point.

Incidentally, this comment on heap_inplace_lock looks suspicious:

* One could modify this to return true for tuples with delete in progress,
* All inplace updaters take a lock that conflicts with DROP. If explicit
* "DELETE FROM pg_class" is in progress, we'll wait for it like we would an
* update.

I think it should be "While one could modify this . . . , all inplace
updaters . . . ." Something to consider for a non-backpatch commit
anyway.

Or this:

* WAL contains likely-unnecessary commit-time invals from the
* CacheInvalidateHeapTuple() call in heap_inplace_update().

Why likely-unnecessary? I know you explain it at that callsite, but
some hint might help here.

On further study, I think one could construct a logical decoding output plugin
for which it's necessary. I've detailed that in the attached edits. This was
in the heap_decode() changes, which is the part of the patch I understand the
least. I likely should have made it a separate patch. Here's the surrounding
change I made in 2024, in context diff format:

*** 508,530 **** 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.
*
! * 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.
*/
- if (!TransactionIdIsValid(xid))
- break;
-
- (void) SnapBuildProcessChange(builder, xid, buf->origptr);
- ReorderBufferXidSetCatalogChanges(ctx->reorder, xid, buf->origptr);
break;

case XLOG_HEAP_CONFIRM:
--- 508,526 ----

/*
* Inplace updates are only ever performed on catalog tuples and
! * 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.
*
! * WAL contains likely-unnecessary commit-time invals from the
! * CacheInvalidateHeapTuple() call in heap_inplace_update().
! * Excess invalidation is safe.
*/
break;

case XLOG_HEAP_CONFIRM:

This code had been unchanged since commit b89e1510 (v9.4) introduced logical
decoding. I'm making the following assumptions about the old code:

- I guessed "decoding in-progress relations" was a typo for "decoding
in-progress transactions", something we've supported since at least commit
4648243 "Add support for streaming to built-in logical replication"
(2020-09, v14). If it's not a typo, I don't know what "in-progress
relations" denoted here.

I agree it's probably a typo.

- It said "redundant because the commit will do that as well", but I didn't
find such code. I bet that it referenced the DecodeCommit() lines removed
in commit c55040c "WAL Log invalidations at command end with
wal_level=logical" (2020-07, v14). The same commit likely made the
ReorderBufferXidSetCatalogChanges() call obsolete.

Makes sense.

- I had no idea why we call SnapBuildProcessChange(). Every other caller uses
its return value. I guess there was a desire for its snapshot side effects,
but I didn't follow that. Nothing snapshot-relevant happens at
XLOG_HEAP_INPLACE. Removing this call doesn't break any test on v13 or on
v9.4. Similarly, no test fails after removing both this and the
ReorderBufferXidSetCatalogChanges() call.

See below.

- In v9.4, this area of code (per-heapam-record-type decoding) had inval
responsibilities. That went away in v14, so there's no need for the comment
here to keep discussing invals.

Okay.

I now think it would be prudent to omit from back-patch the non-comment
heap_decode() changes. While the above assumptions argue against needing the
removed ReorderBufferXidSetCatalogChanges(), that's the sort of speculation I
should keep out back-branch changes.

Okay. We didn't investigate whether/why this code is still needed. I
did run check-world without removing those lines, and it still passes.
So if they are unnecessary, at least they're not harmful.

It's a bit surprising that wrongly leaving relhasindex=t is safe (for
example after BEGIN; CREATE INDEX; ROLLBACK;). I guess this column is
just to save us a lookup for tables with no index, and no harm is done
if we do the lookup needlessly but find no indexes.

And vacuum can
repair it later. Still it's a little unnerving.

DROP INDEX doesn't clear it, either. That's longstanding, and it doesn't
involve narrow race conditions. Hence, I'm not worrying about it. If it were
broken, we'd have heard by now.

Yes, I've come to terms with it. :-)

PrepareToInvalidateCacheTuple(Relation relation,
HeapTuple tuple,
HeapTuple newtuple,
-                              void (*function) (int, uint32, Oid))
+                              void (*function) (int, uint32, Oid, void *),
+                              void *context)

It's a little odd that PrepareToInvalidateCacheTuple takes a callback
function when it only has one caller, so it always calls
RegisterCatcacheInvalidation. Is it just to avoid adding dependencies
to inval.c? But it already #includes catcache.h and contains lots of
knowledge about catcache specifics. Maybe originally
PrepareToInvalidateCacheTuple was built to take
RegisterRelcacheInvalidation as well? Is it worth still passing the
callback?

Looking at the history, it did have two callbacks in Postgres95 1.01. It was
down to one callback when it got the name PrepareToInvalidateCacheTuple() in
commit 81d08fc of 2001-01. I think the main alternative to today's callback
would be to make RegisterCatcacheInvalidation() an extern for catcache.c to
call. All the other Register* functions would remain static. I left things
unchanged since that would be cleaner in one way and dirtier in another.

Okay. It still feels more convoluted than the alternative, but a
backpatch isn't the place to change it.

@@ -6511,6 +6544,7 @@ heap_inplace_unlock(Relation relation,
{
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
UnlockTuple(relation, &oldtup->t_self, InplaceUpdateTupleLock);
+ ForgetInplace_Inval();
}

Is this the right place to add this? We think on balance yes, but the
question crossed my mind: Clearing the invals seems like a separate
responsibility from unlocking the buffer & tuple. After this patch,
our only remaining caller of heap_inplace_unlock is
systable_inplace_update_cancel, so perhaps it should call
ForgetInplace_Inval itself? OTOH we like that putting it here
guarantees it gets called, as a complement to building the invals in
heap_inplace_lock.

Style principles behind the placement:

- Inplace update puts some responsibilities in genam.c and others in heapam.c.
A given task, e.g. draining the inval queue, should consistently appear on
the same side of that boundary.

- The heapam.c side of inplace update should cover roughly as much as the
heapam.c side of heap_update(). Since heap_update() handles invals and the
critical section, so should the heapam.c side of inplace update.

It wouldn't take a strong reason to override those principles.

postgr.es/m/20240831011043.2b.nmisch@google.com has an inventory of the ways
to assign inval responsibility to heapam.c vs. genam.c.
postgr.es/m/20241013004511.9c.nmisch@google.com has a related discussion,
about the need for AtInplace_Inval() to be in the critical section.

Thanks for the explanation!

Yours,

--
Paul ~{:-)
pj@illuminatedcomputing.com

Attachments:

0001-inplace280-comment-fix.patch.nocfbotapplication/octet-stream; name=0001-inplace280-comment-fix.patch.nocfbotDownload
From adf34dd4a43ec4f9cc1ddcb7f4a47891465b7ae8 Mon Sep 17 00:00:00 2001
From: "Paul A. Jungwirth" <pj@illuminatedcomputing.com>
Date: Wed, 26 Nov 2025 13:17:40 -0800
Subject: [PATCH] inplace280-comment-fix

---
 src/backend/replication/logical/decode.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 65ba32b136a..efd0a837fe9 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -530,8 +530,8 @@ heap_decode(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 			 * tuples, we're not interested in the record's contents.
 			 *
 			 * WAL contains likely-unnecessary commit-time invals from the
-			 * CacheInvalidateHeapTuple() call in heap_inplace_update().
-			 * Excess invalidation is safe.
+			 * CacheInvalidateHeapTuple() call in
+			 * heap_inplace_update_and_unlock(). Excess invalidation is safe.
 			 */
 			break;
 
-- 
2.47.3

#30Noah Misch
noah@leadboat.com
In reply to: Paul A Jungwirth (#29)
1 attachment(s)
Re: Inval reliability, especially for inplace updates

Thanks for the review.

On Wed, Dec 03, 2025 at 12:45:58PM -0800, Paul A Jungwirth wrote:

Surya Poondla (cc'd) and I took another look at this as part of the
November Patch Review Workshop.

We think it looks good. But I couldn't get the latest patch to apply
on top of REL_17_STABLE until I did this:

```
git am inplace160-inval-durability-inplace-v7.patch_v17
git revert bc6bad8857 # revert the revert of "WAL-log inplace update"
git am inplace280-comment-fix-v1.patch.nocfbot # attached
git am inplace290-comments202508-v1.patch
```

The inplace280 step adds a small comment change that seems to be in
your git history, but I couldn't find it in the email chain.

inplace290 targets master. inplace280-comment-fix-v1 is doing
s/heap_inplace_update/heap_inplace_update_and_unlock/ on decode.c, like
11012c5 did on master. I'll incorporate inplace280-comment-fix-v1 when
back-patching.

Also the
290 patch has context from reverting the WAL-log revert.

That makes sense. I've not yet probed that level of detail. FYI, if it ends
up making the back-patch clearer, I may split inplace290 into two patches, one
for the inplace160 bits and one for the inplace180 ("WAL-log inplace update")
bits.

The patch avoids deadlocks by reordering invalidation prep before
buffer locking. While no explicit assertion exists to detect future
violations, would it be helpful to add a helper or macro that enforces
this lock ordering rule more visibly? Probably not for a backpatch,
but in master?

I think f4ece89 added that. I plan to back-patch it on the same day as
$SUBJECT.

On Sun, Aug 24, 2025 at 4:39 PM Noah Misch <noah@leadboat.com> wrote:

Some of the comments felt a bit compressed.

Thanks for expanding on this! Here are some thoughts about the new comment:

+    * Register shared cache invals if necessary.  Our input to inval can be
+    * weaker than heap_update() input to inval in these ways:

Perhaps "than the heap_update() input" or "than heap_update()'s input"?

I think adding "the" would make it less correct, since we're contemplating a
class of inputs, not a specific population. I think it's fine with or without
"'s", because there's no rule about treating a C function as a noun adjunct.

+    * - This passes only the old version of the tuple.  Inval reacts only to
+    * catcache lookup key columns and pg_class.oid values stored in
+    * relcache-relevant catalog columns.  All of those columns are indexed.
+    * Inplace update mustn't be used for any operations that could change
+    * those.  Hence, the new tuple would provide no additional inval-relevant
+    * information.  Those facts also make it fine to skip updating indexes.

This is confusing to me. "Inval only reacts": who is inval? Are you
talking about the other backends when they receive the message? After
spending a lot of time, I think you mean
CacheInvalidateHeapTupleCommon, how it decides whether to invalidate
first the catcache and then the relcache.

Right, the last interpretation.

Also I wondered, if an
inplace update never changes index keys, and [something] only cares
about inval messages that change index keys, why are we sending an
inval message at all?

An invalidation message contains only the key, not the value to cache. Hence,
any change to a cacheable tuple must queue an invalidation message, but the
keys suffice to compute which invalidation message to insert.

The cache value is the whole tuple, but the cache keys are a function of:
- list of columns in syscache_info.h
- indexed columns that reference pg_class.oid, e.g. pg_constraint.conrelid

After a few months away from this patch, it was
hard for me to remember. I think the comment is a bit misleading. We
*do* send catcache invalidations if non-key columns change. What about
this?:

+ * - This passes only the old version of the tuple. Catcache
invalidation doesn't need newtuple because inplace updates never
change key columns, so it only needs to invalidate one hash value, not
two. [For the same reason, we don't need to update indexes.] Relcache
invalidation (in CacheInvalidateHeapTupleCommon) ignores newtuple
altogether, even for regular updates, because an update can never move
a tuple from one relcache entry to another.

This influenced some of my new wording.

I bracketed the line about indexes, because I don't understand why
we're talking about updating indexes here. I don't see anything about
that in CacheInvalidateHeapTupleCommon or PrepareInvalidationState
(which doesn't have access to newtuple anyway).

It's basically saying "don't worry about cache key inplace updates until we
solve inplace updates of indexed cols". If we ever wanted to inplace-update a
cache key column, we'd first need to solve inplace-update of indexed columns,
since all cache key columns are indexed. Solving indexed columns is hard with
the buffer locks involved, and there's no reason to expect a wish to do it.

Also it feels like this comment (or something similar) really belongs
on CacheInvalidateHeapTupleInplace. And that function doesn't need its
newtuple parameter.

I like that, so I've made edits along those lines.

+    * - The xwait found below may COMMIT between now and this function
+    * returning, making the tuple dead.  That can change inval decisions, so

Is this third bullet point still explaining why an inplace update can
be looser about invalidating caches than heap_update? Or is it making
a separate point? It seems like it should be a paragraph, not a bullet
point.

It's a separate point. Fixed.

Incidentally, this comment on heap_inplace_lock looks suspicious:

* One could modify this to return true for tuples with delete in progress,
* All inplace updaters take a lock that conflicts with DROP. If explicit
* "DELETE FROM pg_class" is in progress, we'll wait for it like we would an
* update.

I think it should be "While one could modify this . . . , all inplace
updaters . . . ." Something to consider for a non-backpatch commit
anyway.

I did break English grammar there. But I now find the paragraph's argument
faulty, so I've rewritten it:

* heap_delete() is a rarer source of blocking transactions (xwait). We'll
* wait for such a transaction just like for the normal heap_update() case.
* Normal concurrent DROP commands won't cause that, because all inplace
* updaters take some lock that conflicts with DROP. An explicit SQL "DELETE
* FROM pg_class" can cause it. By waiting, if the concurrent transaction
* executed both "DELETE FROM pg_class" and "INSERT INTO pg_class", our caller
* can find the successor tuple.

I considered just deleting the paragraph as being too esoteric to be worth
hackers reading. But if you're reading heap_inplace_lock(), esoterica is
expected.

The attached version doesn't need a comprehensive re-review, but I'd
particularly value hearing about any places where you find it's reducing
comprehensibility rather than enhancing.

Attachments:

inplace290-comments202508-v2.patchtext/plain; charset=us-asciiDownload
From: Noah Misch <noah@leadboat.com>



diff --git a/src/backend/access/heap/README.tuplock b/src/backend/access/heap/README.tuplock
index 843c2e5..16f7d78 100644
--- a/src/backend/access/heap/README.tuplock
+++ b/src/backend/access/heap/README.tuplock
@@ -199,3 +199,35 @@ under a reader holding a pin.  A reader of a heap_fetch() result tuple may
 witness a torn read.  Current inplace-updated fields are aligned and are no
 wider than four bytes, and current readers don't need consistency across
 fields.  Hence, they get by with just fetching each field once.
+
+During logical decoding, caches reflect an inplace update no later than the
+next XLOG_XACT_INVALIDATIONS.  That record witnesses the end of a command.
+Tuples of its cmin are then visible to decoding, as are inplace updates of any
+lower LSN.  Inplace updates of a higher LSN may also be visible, even if those
+updates would have been invisible to a non-historic snapshot matching
+decoding's historic snapshot.  (In other words, decoding may see inplace
+updates that were not visible to a similar snapshot taken during original
+transaction processing.)  That's a consequence of inplace update violating
+MVCC: there are no snapshot-specific versions of inplace-updated values.  This
+all makes it hard to reason about inplace-updated column reads during logical
+decoding, but the behavior does suffice for relhasindex.  A relhasindex=t in
+CREATE INDEX becomes visible no later than the new pg_index row.  While it may
+be visible earlier, that's harmless.  Finding zero indexes despite
+relhasindex=t is normal in more cases than this, e.g. after DROP INDEX.
+Example of a case that meaningfully reacts to the inplace inval:
+
+CREATE TABLE cat (c int) WITH (user_catalog_table = true);
+CREATE TABLE normal (d int);
+...
+CREATE INDEX ON cat (c)\; INSERT INTO normal VALUES (1);
+
+If the output plugin reads "cat" during decoding of the INSERT, it's fair to
+want that read to see relhasindex=t and use the new index.
+
+An alternative would be to have decoding of XLOG_HEAP_INPLACE immediately
+execute its invals.  That would behave more like invals during original
+transaction processing.  It would remove the decoding-specific delay in e.g. a
+decoding plugin witnessing a relfrozenxid change.  However, a good use case
+for that is unlikely, since the plugin would still witness relfrozenxid
+changes prematurely.  Hence, inplace update takes the trivial approach of
+delegating to XLOG_XACT_INVALIDATIONS.
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 4d382a0..7858691 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6349,10 +6349,13 @@ heap_abort_speculative(Relation relation, const ItemPointerData *tid)
  * Since this is intended for system catalogs and SERIALIZABLE doesn't cover
  * DDL, this doesn't guarantee any particular predicate locking.
  *
- * One could modify this to return true for tuples with delete in progress,
- * All inplace updaters take a lock that conflicts with DROP.  If explicit
- * "DELETE FROM pg_class" is in progress, we'll wait for it like we would an
- * update.
+ * heap_delete() is a rarer source of blocking transactions (xwait).  We'll
+ * wait for such a transaction just like for the normal heap_update() case.
+ * Normal concurrent DROP commands won't cause that, because all inplace
+ * updaters take some lock that conflicts with DROP.  An explicit SQL "DELETE
+ * FROM pg_class" can cause it.  By waiting, if the concurrent transaction
+ * executed both "DELETE FROM pg_class" and "INSERT INTO pg_class", our caller
+ * can find the successor tuple.
  *
  * Readers of inplace-updated fields expect changes to those fields are
  * durable.  For example, vac_truncate_clog() reads datfrozenxid from
@@ -6393,15 +6396,17 @@ heap_inplace_lock(Relation relation,
 	Assert(BufferIsValid(buffer));
 
 	/*
-	 * Construct shared cache inval if necessary.  Because we pass a tuple
-	 * version without our own inplace changes or inplace changes other
-	 * sessions complete while we wait for locks, inplace update mustn't
-	 * change catcache lookup keys.  But we aren't bothering with index
-	 * updates either, so that's true a fortiori.  After LockBuffer(), it
-	 * would be too late, because this might reach a
-	 * CatalogCacheInitializeCache() that locks "buffer".
+	 * Register shared cache invals if necessary.  Other sessions may finish
+	 * inplace updates of this tuple between this step and LockTuple().  Since
+	 * inplace updates don't change cache keys, that's harmless.
+	 *
+	 * While it's tempting to register invals only after confirming we can
+	 * return true, the following obstacle precludes reordering steps that
+	 * way.  Registering invals might reach a CatalogCacheInitializeCache()
+	 * that locks "buffer".  That would hang indefinitely if running after our
+	 * own LockBuffer().  Hence, we must register invals before LockBuffer().
 	 */
-	CacheInvalidateHeapTupleInplace(relation, oldtup_ptr, NULL);
+	CacheInvalidateHeapTupleInplace(relation, oldtup_ptr);
 
 	LockTuple(relation, &oldtup.t_self, InplaceUpdateTupleLock);
 	LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
@@ -6639,10 +6644,6 @@ heap_inplace_update_and_unlock(Relation relation,
 	/*
 	 * 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();
 
@@ -6653,10 +6654,10 @@ heap_inplace_update_and_unlock(Relation relation,
 	AcceptInvalidationMessages();	/* local processing of just-sent inval */
 
 	/*
-	 * 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.
+	 * Queue a transactional inval, for logical decoding and for third-party
+	 * code that might have been relying on it since long before inplace
+	 * update adopted immediate invalidation.  See README.tuplock section
+	 * "Reading inplace-updated columns" for logical decoding details.
 	 */
 	if (!IsBootstrapProcessingMode())
 		CacheInvalidateHeapTuple(relation, tuple, NULL);
diff --git a/src/backend/access/index/genam.c b/src/backend/access/index/genam.c
index c969170..5f7b0ca 100644
--- a/src/backend/access/index/genam.c
+++ b/src/backend/access/index/genam.c
@@ -781,10 +781,11 @@ systable_endscan_ordered(SysScanDesc sysscan)
  * systable_inplace_update_begin --- update a row "in place" (overwrite it)
  *
  * Overwriting violates both MVCC and transactional safety, so the uses of
- * this function in Postgres are extremely limited.  Nonetheless we find some
- * places to use it.  See README.tuplock section "Locking to write
- * inplace-updated tables" and later sections for expectations of readers and
- * writers of a table that gets inplace updates.  Standard flow:
+ * this function in Postgres are extremely limited.  This makes no effort to
+ * support updating cache key columns or other indexed columns.  Nonetheless
+ * we find some places to use it.  See README.tuplock section "Locking to
+ * write inplace-updated tables" and later sections for expectations of
+ * readers and writers of a table that gets inplace updates.  Standard flow:
  *
  * ... [any slow preparation not requiring oldtup] ...
  * systable_inplace_update_begin([...], &tup, &inplace_state);
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index cc03f07..5e15cb1 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -521,18 +521,9 @@ heap_decode(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 			/*
 			 * Inplace updates are only ever performed on catalog tuples and
-			 * 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.
-			 *
-			 * WAL contains likely-unnecessary commit-time invals from the
-			 * CacheInvalidateHeapTuple() call in
-			 * heap_inplace_update_and_unlock(). Excess invalidation is safe.
+			 * can, per definition, not change tuple visibility.  Since we
+			 * also don't decode catalog tuples, we're not interested in the
+			 * record's contents.
 			 */
 			break;
 
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index 06f736c..f75c247 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -1583,13 +1583,17 @@ CacheInvalidateHeapTuple(Relation relation,
  *		implied.
  *
  * Like CacheInvalidateHeapTuple(), but for inplace updates.
+ *
+ * Just before and just after the inplace update, the tuple's cache keys must
+ * match those in key_equivalent_tuple.  Cache keys consist of catcache lookup
+ * key columns and columns referencing pg_class.oid values,
+ * e.g. pg_constraint.conrelid, which would trigger relcache inval.
  */
 void
 CacheInvalidateHeapTupleInplace(Relation relation,
-								HeapTuple tuple,
-								HeapTuple newtuple)
+								HeapTuple key_equivalent_tuple)
 {
-	CacheInvalidateHeapTupleCommon(relation, tuple, newtuple,
+	CacheInvalidateHeapTupleCommon(relation, key_equivalent_tuple, NULL,
 								   PrepareInplaceInvalidationState);
 }
 
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index af46625..345733d 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -61,8 +61,7 @@ extern void CacheInvalidateHeapTuple(Relation relation,
 									 HeapTuple tuple,
 									 HeapTuple newtuple);
 extern void CacheInvalidateHeapTupleInplace(Relation relation,
-											HeapTuple tuple,
-											HeapTuple newtuple);
+											HeapTuple key_equivalent_tuple);
 
 extern void CacheInvalidateCatalog(Oid catalogId);
 
#31Noah Misch
noah@leadboat.com
In reply to: Noah Misch (#30)
Re: Inval reliability, especially for inplace updates

On Thu, Dec 04, 2025 at 04:19:02PM -0800, Noah Misch wrote:

Thanks for the review.

The attached version doesn't need a comprehensive re-review, but I'd
particularly value hearing about any places where you find it's reducing
comprehensibility rather than enhancing.

I'd like to get this into the back branches well in advance of the 2026-02
releases, in case the buildfarm catches some defect at low probability. If
there are no objections in the next week, I'll proceed that way.

#32Paul A Jungwirth
pj@illuminatedcomputing.com
In reply to: Noah Misch (#31)
Re: Inval reliability, especially for inplace updates

On Thu, Dec 11, 2025 at 4:24 PM Noah Misch <noah@leadboat.com> wrote:

On Thu, Dec 04, 2025 at 04:19:02PM -0800, Noah Misch wrote:

Thanks for the review.

The attached version doesn't need a comprehensive re-review, but I'd
particularly value hearing about any places where you find it's reducing
comprehensibility rather than enhancing.

I'd like to get this into the back branches well in advance of the 2026-02
releases, in case the buildfarm catches some defect at low probability. If
there are no objections in the next week, I'll proceed that way.

I'm happy with these new comments. The explanation in
heap_inplace_lock before calling CacheInvalidateHeapTupleInplace is a
lot better I think. And removing the last param means there is less to
think about.

Thanks!

--
Paul ~{:-)
pj@illuminatedcomputing.com

#33Noah Misch
noah@leadboat.com
In reply to: Paul A Jungwirth (#32)
Re: Inval reliability, especially for inplace updates

On Fri, Dec 12, 2025 at 09:48:48AM -0800, Paul A Jungwirth wrote:

On Thu, Dec 11, 2025 at 4:24 PM Noah Misch <noah@leadboat.com> wrote:

On Thu, Dec 04, 2025 at 04:19:02PM -0800, Noah Misch wrote:

Thanks for the review.

The attached version doesn't need a comprehensive re-review, but I'd
particularly value hearing about any places where you find it's reducing
comprehensibility rather than enhancing.

I'd like to get this into the back branches well in advance of the 2026-02
releases, in case the buildfarm catches some defect at low probability. If
there are no objections in the next week, I'll proceed that way.

I'm happy with these new comments. The explanation in
heap_inplace_lock before calling CacheInvalidateHeapTupleInplace is a
lot better I think. And removing the last param means there is less to
think about.

I pushed the patch bundle to v17-v14. Thanks for the reviews! The
https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=dragonet&amp;dt=2025-12-17%2001%3A34%3A36
"double free or corruption (!prev)" likely witnessed a defect in how I
back-patched this to v14. I am looking into it.

#34Alexander Lakhin
exclusion@gmail.com
In reply to: Noah Misch (#33)
Re: Inval reliability, especially for inplace updates

Hello Noah,

17.12.2025 05:23, Noah Misch wrote:

I pushed the patch bundle to v17-v14. Thanks for the reviews! The
https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=dragonet&amp;dt=2025-12-17%2001%3A34%3A36
"double free or corruption (!prev)" likely witnessed a defect in how I
back-patched this to v14. I am looking into it.

Thank you for working on this!

Regarding that failure, it's more or less known issue, not related to your
commit:
https://wiki.postgresql.org/wiki/Known_Buildfarm_Test_Failures#check-pg_upgrade_fails_on_LLVM-enabled_animals_due_to_double_free_or_corruption
/messages/by-id/aPAl27_urHSODwRN@paquier.xyz

Best regards,
Alexander

#35Noah Misch
noah@leadboat.com
In reply to: Alexander Lakhin (#34)
Re: Inval reliability, especially for inplace updates

On Wed, Dec 17, 2025 at 07:00:00AM +0200, Alexander Lakhin wrote:

17.12.2025 05:23, Noah Misch wrote:

I pushed the patch bundle to v17-v14. Thanks for the reviews! The
https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=dragonet&amp;dt=2025-12-17%2001%3A34%3A36
"double free or corruption (!prev)" likely witnessed a defect in how I
back-patched this to v14. I am looking into it.

Thank you for working on this!

Regarding that failure, it's more or less known issue, not related to your
commit:
https://wiki.postgresql.org/wiki/Known_Buildfarm_Test_Failures#check-pg_upgrade_fails_on_LLVM-enabled_animals_due_to_double_free_or_corruption
/messages/by-id/aPAl27_urHSODwRN@paquier.xyz

Since v14 dragonet had never failed before, I was ready to assume it was new.
Thank you for highlighting the older issue. I agree.

Other buildfarm failures have been either abi-compliance-check (expected) or
unrelated, so I've closed https://commitfest.postgresql.org/patch/5091/. That
finishes the known-reachable inplace update corruption routes.