From 4acd07bf1639f06c7f85a4f98289622a3eb2ee6c Mon Sep 17 00:00:00 2001
From: amitlan <amitlangote09@gmail.com>
Date: Thu, 15 Sep 2022 16:45:44 +0900
Subject: [PATCH v5 3/4] Make omit_detached logic independent of ActiveSnapshot

In find_inheritance_children_extended() and elsewhere, we use
ActiveSnapshot to determine if a detach-pending partition should
be considered detached or not based on checking if the xmin of
such a partition's pg_inherits row appears committed to that
snapshot or not.

This logic really came in to make the RI queries over partitioned
PK tables running under REPEATABLE READ isolation level work
correctly by appropriately omitting or including the detach-pending
partition from the plan, based on the visibility of the pg_inherits
row of that partition to the latest snapshot.  To that end,
RI_FKey_check()  was made to force-push the latest snapshot to get
that desired behavior.  However, pushing a snapshot this way makes
the results of other scans that use ActiveSnapshot violate the
isolation of the parent transaction; 00cb86e75d added a test that
demonstrates this bug.

So, this commit changes the PartitionDesc interface to allow the
desired snapshot to be passed explicitly as a parameter, rather than
having to scribble on ActiveSnapshot to pass it.  A later commit will
change ExecGetLeafPartitionForKey() used by RI PK row lookups to use
this new interface.

Note that the default behavior in the absence of any explicitly
specified snapshot is still to use the ActiveSnapshot, so there is
no behavior change from this to non-RI queries and sites that call
find_inheritance_children() for purposes other than querying a
partitioned table.
---
 src/backend/catalog/pg_inherits.c    |  31 +++++----
 src/backend/executor/execPartition.c |   7 +-
 src/backend/optimizer/util/inherit.c |   2 +-
 src/backend/optimizer/util/plancat.c |   2 +-
 src/backend/partitioning/partdesc.c  | 100 +++++++++++++++++++--------
 src/include/catalog/pg_inherits.h    |   5 +-
 src/include/partitioning/partdesc.h  |   4 +-
 7 files changed, 100 insertions(+), 51 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 92afbc2f25..f810e5de0d 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,14 +52,18 @@ typedef struct SeenRelsEntry
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
  *
- * Partitions marked as being detached are omitted; see
+ * A partition marked as being detached is omitted from the result if the
+ * pg_inherits row showing the partition as being detached is visible to
+ * ActiveSnapshot, doing so only when one has been pushed; see
  * find_inheritance_children_extended for details.
  */
 List *
 find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 {
-	return find_inheritance_children_extended(parentrelId, true, lockmode,
-											  NULL, NULL);
+	return find_inheritance_children_extended(parentrelId, true,
+											  ActiveSnapshotSet() ?
+											  GetActiveSnapshot() : NULL,
+											  lockmode, NULL, NULL);
 }
 
 /*
@@ -71,16 +75,17 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * If a partition's pg_inherits row is marked "detach pending",
  * *detached_exist (if not null) is set true.
  *
- * If omit_detached is true and there is an active snapshot (not the same as
- * the catalog snapshot used to scan pg_inherits!) and a pg_inherits tuple
- * marked "detach pending" is visible to that snapshot, then that partition is
- * omitted from the output list.  This makes partitions invisible depending on
- * whether the transaction that marked those partitions as detached appears
- * committed to the active snapshot.  In addition, *detached_xmin (if not null)
- * is set to the xmin of the row of the detached partition.
+ * If omit_detached is true and the caller passed 'omit_detached_snapshot',
+ * the partition whose pg_inherits tuple marks it as "detach pending" is
+ * omitted from the output list if the tuple is visible to that snapshot.
+ * That is, such a partition is omitted from the output list depending on
+ * whether the transaction that marked that partition as detached appears
+ * committed to omit_detached_snapshot.  If omitted, *detached_xmin (if non
+ * NULL) is set to the xmin of that pg_inherits tuple.
  */
 List *
 find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
+								   Snapshot omit_detached_snapshot,
 								   LOCKMODE lockmode, bool *detached_exist,
 								   TransactionId *detached_xmin)
 {
@@ -141,15 +146,13 @@ find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
 			if (detached_exist)
 				*detached_exist = true;
 
-			if (omit_detached && ActiveSnapshotSet())
+			if (omit_detached && omit_detached_snapshot)
 			{
 				TransactionId xmin;
-				Snapshot	snap;
 
 				xmin = HeapTupleHeaderGetXmin(inheritsTuple->t_data);
-				snap = GetActiveSnapshot();
 
-				if (!XidInMVCCSnapshot(xmin, snap))
+				if (!XidInMVCCSnapshot(xmin, omit_detached_snapshot))
 				{
 					if (detached_xmin)
 					{
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 764f2b9f8a..c90f07c433 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1121,7 +1121,8 @@ ExecInitPartitionDispatchInfo(EState *estate,
 		rel = table_open(partoid, RowExclusiveLock);
 	else
 		rel = proute->partition_root;
-	partdesc = PartitionDirectoryLookup(estate->es_partition_directory, rel);
+	partdesc = PartitionDirectoryLookup(estate->es_partition_directory, rel,
+										NULL);
 
 	pd = (PartitionDispatch) palloc(offsetof(PartitionDispatchData, indexes) +
 									partdesc->nparts * sizeof(int));
@@ -1708,7 +1709,7 @@ ExecGetLeafPartitionForKey(Relation root_rel, int key_natts,
 
 		/* Get the PartitionDesc using the partition directory machinery.  */
 		partdir = CreatePartitionDirectory(CurrentMemoryContext, true);
-		partdesc = PartitionDirectoryLookup(partdir, rel);
+		partdesc = PartitionDirectoryLookup(partdir, rel, NULL);
 
 		/* Find the partition for the key. */
 		partidx = get_partition_for_tuple(partkey, partdesc, partkey_vals,
@@ -2085,7 +2086,7 @@ CreatePartitionPruneState(PlanState *planstate, PartitionPruneInfo *pruneinfo)
 			partrel = ExecGetRangeTableRelation(estate, pinfo->rtindex);
 			partkey = RelationGetPartitionKey(partrel);
 			partdesc = PartitionDirectoryLookup(estate->es_partition_directory,
-												partrel);
+												partrel, NULL);
 
 			/*
 			 * Initialize the subplan_map and subpart_map.
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index cf7691a474..cc4d27ece8 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -317,7 +317,7 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 	Assert(parentrte->inh);
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
-										parentrel);
+										parentrel, NULL);
 
 	/* A partitioned table should always have a partition descriptor. */
 	Assert(partdesc);
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 6d5718ee4c..9c6bc5c4a5 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2221,7 +2221,7 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
-										relation);
+										relation, NULL);
 	rel->part_scheme = find_partition_scheme(root, relation);
 	Assert(partdesc != NULL && rel->part_scheme != NULL);
 	rel->boundinfo = partdesc->boundinfo;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 737f0edd89..863b04c17d 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -48,17 +48,24 @@ typedef struct PartitionDirectoryEntry
 } PartitionDirectoryEntry;
 
 static PartitionDesc RelationBuildPartitionDesc(Relation rel,
-												bool omit_detached);
+												bool omit_detached,
+												Snapshot omit_detached_snapshot);
 
 
 /*
- * RelationGetPartitionDesc -- get partition descriptor, if relation is partitioned
+ * RelationGetPartitionDescExt
+ * 		Get partition descriptor of a partitioned table, building one and
+ * 		caching it for later use if not already or if the cached one would
+ * 		not be suitable for a given request
  *
  * We keep two partdescs in relcache: rd_partdesc includes all partitions
- * (even those being concurrently marked detached), while rd_partdesc_nodetach
- * omits (some of) those.  We store the pg_inherits.xmin value for the latter,
- * to determine whether it can be validly reused in each case, since that
- * depends on the active snapshot.
+ * (even the one being concurrently marked detached), while
+ * rd_partdesc_nodetach omits the detach-pending partition.  If the latter one
+ * is present, rd_partdesc_nodetach_xmin would have been set to the xmin of
+ * the detach-pending partition's pg_inherits row, which is used to determine
+ * whether rd_partdesc_nodetach can be validly reused for a given request by
+ * checking if the xmin appears visible to 'omit_detached_snapshot' passed by
+ * the caller.
  *
  * Note: we arrange for partition descriptors to not get freed until the
  * relcache entry's refcount goes to zero (see hacks in RelationClose,
@@ -69,7 +76,8 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel, bool omit_detached)
+RelationGetPartitionDescExt(Relation rel, bool omit_detached,
+							Snapshot omit_detached_snapshot)
 {
 	Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
@@ -78,36 +86,52 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
 	 * do so when we are asked to include all partitions including detached;
 	 * and also when we know that there are no detached partitions.
 	 *
-	 * If there is no active snapshot, detached partitions aren't omitted
-	 * either, so we can use the cached descriptor too in that case.
+	 * omit_detached_snapshot being NULL means that the caller doesn't care
+	 * that the returned partition descriptor may contain detached partitions,
+	 * so we we can used the cached descriptor in that case too.
 	 */
 	if (likely(rel->rd_partdesc &&
 			   (!rel->rd_partdesc->detached_exist || !omit_detached ||
-				!ActiveSnapshotSet())))
+				omit_detached_snapshot == NULL)))
 		return rel->rd_partdesc;
 
 	/*
-	 * If we're asked to omit detached partitions, we may be able to use a
-	 * cached descriptor too.  We determine that based on the pg_inherits.xmin
-	 * that was saved alongside that descriptor: if the xmin that was not in
-	 * progress for that active snapshot is also not in progress for the
-	 * current active snapshot, then we can use it.  Otherwise build one from
-	 * scratch.
+	 * If we're asked to omit the detached partition, we may be able to use
+	 * the other cached descriptor, which has been made to omit the detached
+	 * partition.  Whether that descriptor can be reused in this case is
+	 * determined based on cross-checking the visibility of
+	 * rd_partdesc_nodetached_xmin, that is, the pg_inherits.xmin of the
+	 * pg_inherits row of the detached partition: if the xmin seems in-progress
+	 * to both the given omit_detached_snapshot and to the snapshot that would
+	 * have been passed when rd_partdesc_nodetached was built, then we can
+	 * reuse it.  Otherwise we must build one from scratch.
 	 */
 	if (omit_detached &&
 		rel->rd_partdesc_nodetached &&
-		ActiveSnapshotSet())
+		omit_detached_snapshot)
 	{
-		Snapshot	activesnap;
-
 		Assert(TransactionIdIsValid(rel->rd_partdesc_nodetached_xmin));
-		activesnap = GetActiveSnapshot();
 
-		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin, activesnap))
+		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin,
+							   omit_detached_snapshot))
 			return rel->rd_partdesc_nodetached;
 	}
 
-	return RelationBuildPartitionDesc(rel, omit_detached);
+	return RelationBuildPartitionDesc(rel, omit_detached,
+									  omit_detached_snapshot);
+}
+
+/*
+ * RelationGetPartitionDesc
+ *		Like RelationGetPartitionDescExt() but for callers that are fine with
+ *		ActiveSnapshot being used as omit_detached_snapshot
+ */
+PartitionDesc
+RelationGetPartitionDesc(Relation rel, bool omit_detached)
+{
+	return RelationGetPartitionDescExt(rel, omit_detached,
+									   ActiveSnapshotSet() ?
+									   GetActiveSnapshot() : NULL);
 }
 
 /*
@@ -132,7 +156,8 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
  * for them.
  */
 static PartitionDesc
-RelationBuildPartitionDesc(Relation rel, bool omit_detached)
+RelationBuildPartitionDesc(Relation rel, bool omit_detached,
+						   Snapshot omit_detached_snapshot)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -160,7 +185,9 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	detached_exist = false;
 	detached_xmin = InvalidTransactionId;
 	inhoids = find_inheritance_children_extended(RelationGetRelid(rel),
-												 omit_detached, NoLock,
+												 omit_detached,
+												 omit_detached_snapshot,
+												 NoLock,
 												 &detached_exist,
 												 &detached_xmin);
 
@@ -322,11 +349,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	 *
 	 * Note that if a partition was found by the catalog's scan to have been
 	 * detached, but the pg_inherit tuple saying so was not visible to the
-	 * active snapshot (find_inheritance_children_extended will not have set
-	 * detached_xmin in that case), we consider there to be no "omittable"
-	 * detached partitions.
+	 * omit_detached_snapshot (find_inheritance_children_extended() will not
+	 * have set detached_xmin in that case), we consider there to be no
+	 * "omittable" detached partitions.
 	 */
-	is_omit = omit_detached && detached_exist && ActiveSnapshotSet() &&
+	is_omit = omit_detached && detached_exist && omit_detached_snapshot &&
 		TransactionIdIsValid(detached_xmin);
 
 	/*
@@ -411,9 +438,18 @@ CreatePartitionDirectory(MemoryContext mcxt, bool omit_detached)
  * different views of the catalog state, but any single particular OID
  * will always get the same PartitionDesc for as long as the same
  * PartitionDirectory is used.
+ *
+ * Callers can specify a snapshot to cross-check the visibility of the
+ * pg_inherits row marking a given partition being detached.  Depending on the
+ * result of that visibility check, such a partition is either included in
+ * the returned PartitionDesc, considering it not yet detached, or omitted
+ * from it, considering it detached.
+ * XXX - currently unused, because we don't have any callers of this that
+ * would like to pass a snapshot that is not ActiveSnapshot.
  */
 PartitionDesc
-PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
+PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel,
+						 Snapshot omit_detached_snapshot)
 {
 	PartitionDirectoryEntry *pde;
 	Oid			relid = RelationGetRelid(rel);
@@ -428,7 +464,11 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel, pdir->omit_detached);
+		Assert(omit_detached_snapshot == NULL);
+		if (pdir->omit_detached && ActiveSnapshotSet())
+			omit_detached_snapshot = GetActiveSnapshot();
+		pde->pd = RelationGetPartitionDescExt(rel, pdir->omit_detached,
+											  omit_detached_snapshot);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 9221c2ea57..67f148f2bf 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -23,6 +23,7 @@
 
 #include "nodes/pg_list.h"
 #include "storage/lock.h"
+#include "utils/snapshot.h"
 
 /* ----------------
  *		pg_inherits definition.  cpp turns this into
@@ -50,7 +51,9 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, InheritsParentIndexId, on pg_inher
 
 extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
 extern List *find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
-												LOCKMODE lockmode, bool *detached_exist, TransactionId *detached_xmin);
+												Snapshot omit_detached_snapshot,
+												LOCKMODE lockmode, bool *detached_exist,
+												TransactionId *detached_xmin);
 
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **numparents);
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 7e979433b6..f42d137fc1 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -65,9 +65,11 @@ typedef struct PartitionDescData
 
 
 extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool omit_detached);
+extern PartitionDesc RelationGetPartitionDescExt(Relation rel, bool omit_detached,
+												 Snapshot omit_detached_snapshot);
 
 extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool omit_detached);
-extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
+extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation, Snapshot);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
 extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
-- 
2.35.3

