ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Started by Alvaro Herreraover 5 years ago90 messages
#1Alvaro Herrera
alvherre@2ndquadrant.com
2 attachment(s)

I've been working on the ability to detach a partition from a
partitioned table, without causing blockages to concurrent activity.
I think this operation is critical for some use cases.

There was a lot of great discussion which ended up in Robert completing
a much sought implementation of non-blocking ATTACH. DETACH was
discussed too because it was a goal initially, but eventually dropped
from that patch altogether. Nonetheless, that thread provided a lot of
useful input to this implementation. Important ones:

[1]: /messages/by-id/CA+TgmoYg4x7AH=_QSptvuBKf+3hUdiCa4frPkt+RvXZyjX1n=w@mail.gmail.com
[2]: /messages/by-id/CA+TgmoaAjkTibkEr=xJg3ndbRsHHSiYi2SJgX69MVosj=LJmug@mail.gmail.com
[3]: /messages/by-id/CA+TgmoY13KQZF-=HNTrt9UYWYx3_oYOQpu9ioNT49jGgiDpUEA@mail.gmail.com

Attached is a patch that implements
ALTER TABLE ... DETACH PARTITION .. CONCURRENTLY.

In the previous thread we were able to implement the concurrent model
without the extra keyword. For this one I think that won't work; my
implementation works in two transactions so there's a restriction that
you can't run it in a transaction block. Also, there's a wait phase
that makes it slower than the non-concurrent one. Those two drawbacks
make me think that it's better to keep both modes available, just like
we offer both CREATE INDEX and CREATE INDEX CONCURRENTLY.

Why two transactions? The reason is that in order for this to work, we
make a catalog change (mark it detached), and commit so that all
concurrent transactions can see the change. A second transaction waits
for anybody who holds any lock on the partitioned table and grabs Access
Exclusive on the partition (which now no one cares about, if they're
looking at the partitioned table), where the DDL action on the partition
can be completed.

ALTER TABLE is normally unable to run in two transactions. I hacked it
(0001) so that the relation can be closed and reopened in the Exec phase
(by having the rel as part of AlteredTableInfo: when ATRewriteCatalogs
returns, it uses that pointer to close the rel). It turns out that this
is sufficient to make that work. This means that ALTER TABLE DETACH
CONCURRENTLY cannot work as part of a multi-command ALTER TABLE, but
that's alreay enforced by the grammar anyway.

DETACH CONCURRENTLY doesn't work if a default partition exists. It's
just too problematic a case; you would still need to have AEL on the
default partition.

I haven't yet experimented with queries running in a standby in tandem
with a detach.

--
�lvaro Herrera

Attachments:

v1-0001-Let-ALTER-TABLE-exec-routines-deal-with-the-relat.patchtext/x-diff; charset=us-asciiDownload
From 2f9202bf6f4c86d607032d7f04d3b2cee74a9617 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 13 Jul 2020 20:15:30 -0400
Subject: [PATCH v1 1/2] Let ALTER TABLE exec routines deal with the relation

This means that ATExecCmd relies on AlteredRelationInfo->rel instead of
keeping the relation as a local variable; this is useful if the
subcommand needs to modify the relation internally.  For example, a
subcommand that internally commits its transaction needs this.
---
 src/backend/commands/tablecmds.c | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index ac53f79ada..07f4f562ee 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -157,6 +157,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -354,7 +356,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -4326,7 +4328,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4335,10 +4336,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4350,7 +4351,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4376,11 +4381,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -5455,6 +5461,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
-- 
2.20.1

v1-0002-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From 392f1c5e411dc1472b431f6732291d8458f85ca2 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 3 Aug 2020 19:21:09 -0400
Subject: [PATCH v1 2/2] ALTER TABLE ... DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  25 +-
 src/backend/catalog/heap.c                    |  13 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/pg_inherits.c             |  51 +-
 src/backend/commands/indexcmds.c              |   5 +-
 src/backend/commands/tablecmds.c              | 491 ++++++++++++++----
 src/backend/commands/trigger.c                |   7 +-
 src/backend/executor/execPartition.c          |  16 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/plancat.c          |   2 +-
 src/backend/parser/gram.y                     |  22 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/bin/psql/describe.c                       |  40 +-
 src/include/catalog/pg_inherits.h             |   7 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 .../detach-partition-concurrently-1.out       | 182 +++++++
 .../detach-partition-concurrently-2.out       |  66 +++
 src/test/isolation/isolation_schedule         |   2 +
 .../detach-partition-concurrently-1.spec      |  71 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 26 files changed, 958 insertions(+), 153 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 26fda20d19..0f8abdc2d0 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4471,6 +4471,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetached</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index b2eb7097a9..2e67468e46 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction,
+      <literal>SHARE UPDATE EXCLUSIVE</literal> is taken in both parent table and
+      partition, and the partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, complete actions of a
+      previous <literal>DETACH CONCURRENTLY</literal> invocation that
+      was cancelled or crashed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index f2ca686397..7cc5cb4d51 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2552,10 +2552,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3787,7 +3789,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 1be27eec52..ad545a7866 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1655,7 +1655,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2241,7 +2241,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 17f37eb39f..f143db0ad4 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -200,7 +210,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -429,6 +440,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -448,10 +460,21 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * 'detached' is the expected state of the inhdetached flag.  If the catalog
+ * row does not match that state, an error is raised.  When used on
+ * partitions, the partition name must be passed, for possible error messages.
+ *
+ * If allow_detached is passed false, then an error is raised if the child is
+ * already marked detached.  A name must be supplied in that case, for the
+ * error message.  This should only be used with table partitions.
+ * XXX the name bit is pretty weird and problematic for non-partition cases;
+ * what if the flag check fails?
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -478,6 +501,28 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	inhdetached;
+
+			inhdetached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (inhdetached && !detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no partial concurrent detach in progress.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 2baca12c5f..d032fe4af3 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -1092,7 +1092,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1149,7 +1149,7 @@ DefineIndex(Oid relationId,
 		 */
 		if (!stmt->relation || stmt->relation->inh)
 		{
-			PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+			PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 			int			nparts = partdesc->nparts;
 			Oid		   *part_oids = palloc(sizeof(Oid) * nparts);
 			bool		invalidate_parent = false;
@@ -3550,6 +3550,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 07f4f562ee..682aa6a015 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -541,7 +541,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd);
 static void AttachPartitionEnsureIndexes(Relation rel, Relation attachrel);
@@ -550,7 +551,12 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -984,7 +990,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3099,7 +3106,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3298,7 +3305,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -3916,7 +3923,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4288,6 +4302,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4663,7 +4682,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
@@ -6081,7 +6105,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6314,7 +6338,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6468,7 +6493,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -7680,7 +7705,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8144,7 +8170,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -8759,7 +8786,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -8893,7 +8920,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10674,7 +10701,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11058,7 +11086,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -13953,7 +13982,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -13964,12 +13993,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetached = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -13983,7 +14070,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -13999,7 +14086,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16107,7 +16196,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16163,7 +16252,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16751,105 +16840,226 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with
+	 * ShareUpdateExclusiveLock in the first stage.
+	 */
+	partRel = table_openrv(name, concurrent ?
+						   ShareUpdateExclusiveLock : AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps lock
+	 * on the referencing tables, which prevent concurrent transactions from
+	 * adding rows that we wouldn't see.  For this to work in concurrent mode,
+	 * it is critical that the partition appears as no longer attached for the
+	 * RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint.  The reason for this is
+	 * that the planner may have made optimizations that depend on the
+	 * constraint.  XXX Isn't it sufficient to invalidate the partition's
+	 * relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all process to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		Constraint *n;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint on partition, equivalent to the partition constraint */
+		n = makeNode(Constraint);
+		n->contype = CONSTR_CHECK;
+		n->conname = NULL;
+		n->location = -1;
+		n->is_no_inherit = false;
+		n->raw_expr = NULL;
+		n->cooked_expr = nodeToString(make_ands_explicit(RelationGetPartitionQual(partRel)));
+		n->initially_valid = true;
+		n->skip_validation = true;
+		/* It's a re-add, since it nominally already exists */
+		ATAddCheckConstraint(wqueue, tab, partRel, n,
+							 true, false, true, ShareUpdateExclusiveLock);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -16922,17 +17132,98 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
@@ -17133,7 +17424,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17287,7 +17578,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 672fccff5b..ce65abd57e 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1054,7 +1054,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1076,7 +1076,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
@@ -1537,7 +1538,7 @@ EnableDisableTrigger(Relation rel, const char *tgname,
 			if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE &&
 				(TRIGGER_FOR_ROW(oldtrig->tgtype)))
 			{
-				PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+				PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 				int			i;
 
 				for (i = 0; i < partdesc->nparts; i++)
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index fb6ce49056..ef4fd7bf90 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -507,6 +507,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	rootrel = rootResultRelInfo->ri_RelationDesc,
 				partrel;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -517,7 +518,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -997,7 +998,7 @@ ExecInitPartitionDispatchInfo(EState *estate,
 
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1559,7 +1560,7 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1625,9 +1626,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 89c409de66..f48350bc2f 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4670,6 +4670,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index e3f33c40be..01ff684f90 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2932,6 +2932,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 25545029d7..75fe783b41 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2129,7 +2129,7 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 	/* Create the PartitionDirectory infrastructure if we didn't already */
 	if (root->glob->partition_directory == NULL)
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext, false);
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index dbb47d4982..da0b233695 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -649,7 +649,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2057,12 +2057,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2070,8 +2071,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2086,6 +2100,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15110,6 +15125,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 7553d55987..966d4bbe7f 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2810,7 +2810,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	ParseState *pstate = make_parsestate(NULL);
 	int			with = -1;
@@ -3984,7 +3984,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4184,7 +4184,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 0f124a5254..3c21f42ea9 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -38,6 +38,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -47,7 +48,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -62,13 +63,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -89,7 +91,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -111,7 +113,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock,
+										include_detached);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -239,6 +242,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -281,7 +285,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -296,6 +300,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 	pdir->pdir_mcxt = mcxt;
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -328,7 +333,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 9b0c376c8c..57ce760edc 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1240,6 +1240,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 57266f4fc3..329b650e12 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2109,7 +2109,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetached" :
+						  "false as inhdetached");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2127,17 +2132,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3179,9 +3186,18 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetached,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3190,14 +3206,14 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3247,11 +3263,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACHED)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 0432c3ab88..fcdea9f26f 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid;
 	Oid			inhparent;
 	int32		inhseqno;
+	bool		inhdetached;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -44,7 +45,8 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 typedef FormData_pg_inherits *Form_pg_inherits;
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -52,6 +54,7 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 151bcdb7ef..d291b4d7df 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -859,6 +859,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1845,6 +1846,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity				/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 08f22ce211..27414b0912 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -163,6 +163,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 70df764981..b1ee9fd5e3 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..04a0446ccb
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,182 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s1s s1c s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2d: <... completed>
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2d s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2d s1ins s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1s s2d s1ins s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1s s2d s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1exec: EXECUTE f(1); DEALLOCATE f;
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec2 s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1exec2: EXECUTE f(2); DEALLOCATE f;
+step s2d: <... completed>
+error in steps s1exec2 s2d: ERROR:  no partition of relation "d_listp" found for row
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s2d s3drop1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop1: DROP TABLE d_listp; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3drop1: <... completed>
+error in steps s1c s2d s3drop1: ERROR:  partitioned table "d_listp" was removed concurrently
+
+starting permutation: s1b s1s s2d s3drop2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop2: DROP TABLE d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3drop2: <... completed>
+error in steps s1c s2d s3drop2: ERROR:  partition "d_listp2" was removed concurrently
+
+starting permutation: s1b s1s s2d s3detach s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3detach: ALTER TABLE d_listp DETACH PARTITION d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3detach: <... completed>
+error in steps s1c s2d s3detach: ERROR:  cannot detach partition "d_listp2"
+
+starting permutation: s1b s1s s2d s3rename s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3rename: ALTER TABLE d_listp2 RENAME TO foobar; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3rename: <... completed>
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index 218c87b24b..a9c1abc244 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,8 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..ca92530824
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,71 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1exec"	{ EXECUTE f(1); DEALLOCATE f; }
+step "s1exec2"	{ EXECUTE f(2); DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3drop1"	{ DROP TABLE d_listp; }
+step "s3drop2"	{ DROP TABLE d_listp2; }
+step "s3detach"	{ ALTER TABLE d_listp DETACH PARTITION d_listp2; }
+step "s3rename"	{ ALTER TABLE d_listp2 RENAME TO foobar; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s1s" "s1c" "s2drop" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2d" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# Session s1 no longer sees the partition ...
+permutation "s1b" "s1s" "s2d" "s1ins" "s1c"
+# ... regardless of the isolation level.
+permutation "s1brr" "s1s" "s2d" "s1ins" "s1c"
+# However, DETACH CONCURRENTLY is not isolation-level-correct
+permutation "s1brr" "s1s" "s2d" "s1s" "s1c"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec" "s1c"
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec2" "s1c"
+
+
+# Note: the following tests are not very interesting in isolationtester because
+# the way this tool handles multiple sessions: it'll only run the drop/detach/
+# rename commands after s2 is blocked waiting.  It would be more useful to
+# run the DDL when s2 commits its first transaction instead.  Maybe these should
+# become TAP tests someday.
+
+# What happens if the partition or the parent table is dropped while waiting?
+permutation "s1b" "s1s" "s2d" "s3drop1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3drop2" "s1c"
+# Also try a non-concurrent detach while the other one is waiting
+permutation "s1b" "s1s" "s2d" "s3detach" "s1c"
+# and some other DDL
+permutation "s1b" "s1s" "s2d" "s3rename" "s1c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
-- 
2.20.1

#2Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Alvaro Herrera (#1)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Aug-03, Alvaro Herrera wrote:

There was a lot of great discussion which ended up in Robert completing
a much sought implementation of non-blocking ATTACH. DETACH was
discussed too because it was a goal initially, but eventually dropped
from that patch altogether. Nonetheless, that thread provided a lot of
useful input to this implementation. Important ones:

[1] /messages/by-id/CA+TgmoYg4x7AH=_QSptvuBKf+3hUdiCa4frPkt+RvXZyjX1n=w@mail.gmail.com
[2] /messages/by-id/CA+TgmoaAjkTibkEr=xJg3ndbRsHHSiYi2SJgX69MVosj=LJmug@mail.gmail.com
[3] /messages/by-id/CA+TgmoY13KQZF-=HNTrt9UYWYx3_oYOQpu9ioNT49jGgiDpUEA@mail.gmail.com

There was some discussion about having a version number in the partition
descriptor somewhere as a means to implement this. I couldn't figure
out how that would work, or what the version number would be attached
to. Surely the idea wasn't to increment the version number to every
partition other than the one being detached?

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#3Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Alvaro Herrera (#1)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Aug-03, Alvaro Herrera wrote:

Why two transactions? The reason is that in order for this to work, we
make a catalog change (mark it detached), and commit so that all
concurrent transactions can see the change. A second transaction waits
for anybody who holds any lock on the partitioned table and grabs Access
Exclusive on the partition (which now no one cares about, if they're
looking at the partitioned table), where the DDL action on the partition
can be completed.

I forgot to mention. If for whatever reason the second transaction
fails, (say the user aborts it or there is a crash), then the partition
is still marked as detached, so no queries would see it; but all the
ancillary catalog data remains. Just like when CREATE INDEX
CONCURRENTLY fails, you keep an invalid index that must be dropped; in
this case, the changes to do are much more extensive, so manual action
is out of the question. So there's another DDL command to be invoked,

ALTER TABLE parent DETACH PARTITION part FINALIZE;

which will complete the detach action.

If we had UNDO then perhaps it would be possible to register an action
so that the detach is completed automatically. But for now this seems
sufficient.

Another aspect worth mentioning is constraints. In the patch, I create
a CHECK constraint to stand for the partition constraint that's going to
logically disappear. This was mentioned as a potential problem in one
of Robert's emails (I didn't actually verify that this is a problem).
However, a funny thing is that if a constraint already exists, you get a
dupe, so after a few rounds of attach/detach you can see them pile up.
I'll have to fix this at some point. But also, I need to think about
whether foreign keys have similar problems, since they are also used by
the optimizer.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#4Robert Haas
robertmhaas@gmail.com
In reply to: Alvaro Herrera (#1)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Mon, Aug 3, 2020 at 7:49 PM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

Why two transactions? The reason is that in order for this to work, we
make a catalog change (mark it detached), and commit so that all
concurrent transactions can see the change. A second transaction waits
for anybody who holds any lock on the partitioned table and grabs Access
Exclusive on the partition (which now no one cares about, if they're
looking at the partitioned table), where the DDL action on the partition
can be completed.

Is there a more detailed theory of operation of this patch somewhere?
What exactly do you mean by marking it detached? Committing the change
makes it possible for all concurrent transactions to see the change,
but does not guarantee that they will. If you can't guarantee that,
then I'm not sure how you can guarantee that they will behave sanely.
Even if you can, it's not clear what the sane behavior is: what
happens when a tuple gets routed to an ex-partition? What happens when
an ex-partition needs to be scanned? What prevents problems if a
partition is detached, possibly modified, and then reattached,
possibly with different partition bounds?

I think the two-transaction approach is interesting and I can imagine
that it possibly solves some problems, but it's not clear to me
exactly which problems it solves or how it does so.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#5Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Robert Haas (#4)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Aug-04, Robert Haas wrote:

On Mon, Aug 3, 2020 at 7:49 PM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

Why two transactions? The reason is that in order for this to work, we
make a catalog change (mark it detached), and commit so that all
concurrent transactions can see the change. A second transaction waits
for anybody who holds any lock on the partitioned table and grabs Access
Exclusive on the partition (which now no one cares about, if they're
looking at the partitioned table), where the DDL action on the partition
can be completed.

Is there a more detailed theory of operation of this patch somewhere?
What exactly do you mean by marking it detached? Committing the change
makes it possible for all concurrent transactions to see the change,
but does not guarantee that they will. If you can't guarantee that,
then I'm not sure how you can guarantee that they will behave sanely.

Sorry for the long delay. I haven't written up the theory of operation.
I suppose it is complicated enough that it should be documented
somewhere.

To mark it detached means to set pg_inherits.inhdetached=true. That
column name is a bit of a misnomer, since that column really means "is
in the process of being detached"; the pg_inherits row goes away
entirely once the detach process completes. This mark guarantees that
everyone will see that row because the detaching session waits long
enough after committing the first transaction and before deleting the
pg_inherits row, until everyone else has moved on.

The other point is that the partition directory code can be asked to
include detached partitions, or not to. The executor does the former,
and the planner does the latter. This allows a transient period during
which the partition descriptor returned to planner and executor is
different; this makes the situation equivalent to what would have
happened if the partition was attached during the operation: in executor
we would detect that there is an additional partition that was not seen
by the planner, and we already know how to deal with that situation by
your handling of the ATTACH code.

Even if you can, it's not clear what the sane behavior is: what
happens when a tuple gets routed to an ex-partition? What happens when
an ex-partition needs to be scanned?

During the transient period, any transaction that obtained a partition
descriptor before the inhdetached mark is committed should be able to
get the tuple routing done successfully, but transactions using later
snapshots should get their insertions rejected. Reads should behave in
the same way.

What prevents problems if a partition is detached, possibly modified,
and then reattached, possibly with different partition bounds?

This should not be a problem, because the partition needs to be fully
detached before it can be attached again. And if the partition bounds
are different, that won't be a problem, because the previous partition
bounds won't be present in the pg_class row. Of course, the new
partition bounds will be checked to the existing contents.

There is one fly in the ointment though, which is that if you cancel the
wait and then immediately do the ALTER TABLE DETACH FINALIZE without
waiting for as long as the original execution would have waited, you
might end up killing the partition ahead of time. One solution to this
would be to cause the FINALIZE action to wait again at start. This
would cause it to take even longer, but it would be safer. (If you
don't want it to take longer, you can just not cancel it in the first
place.) This is not a problem if the server crashes in between (which
is the scenario I had in mind when doing the FINALIZE thing), because of
course no transaction can continue to run across a crash.

I'm going to see if I can get the new delay_execution module to help
test this, to verify whether my claims are true.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#6Robert Haas
robertmhaas@gmail.com
In reply to: Alvaro Herrera (#5)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Wed, Aug 26, 2020 at 7:40 PM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

To mark it detached means to set pg_inherits.inhdetached=true. That
column name is a bit of a misnomer, since that column really means "is
in the process of being detached"; the pg_inherits row goes away
entirely once the detach process completes. This mark guarantees that
everyone will see that row because the detaching session waits long
enough after committing the first transaction and before deleting the
pg_inherits row, until everyone else has moved on.

OK. Do you just wait until the XID of the transaction that set
inhdetached is all-visible, or how do you do it?

The other point is that the partition directory code can be asked to
include detached partitions, or not to. The executor does the former,
and the planner does the latter. This allows a transient period during
which the partition descriptor returned to planner and executor is
different; this makes the situation equivalent to what would have
happened if the partition was attached during the operation: in executor
we would detect that there is an additional partition that was not seen
by the planner, and we already know how to deal with that situation by
your handling of the ATTACH code.

Ah ha! That is quite clever and I don't think that I would have
thought of it. So all the plans that were created before you set
inhdetached=true have to be guaranteed to be invaliated or gone
altogether before you can actually delete the pg_inherits row. It
seems like it ought to be possible to ensure that, though I am not
surely of the details exactly. Is it sufficient to wait for all
transactions that have locked the table to go away? I'm not sure
exactly how this stuff interacts with the plan cache.

There is one fly in the ointment though, which is that if you cancel the
wait and then immediately do the ALTER TABLE DETACH FINALIZE without
waiting for as long as the original execution would have waited, you
might end up killing the partition ahead of time. One solution to this
would be to cause the FINALIZE action to wait again at start. This
would cause it to take even longer, but it would be safer. (If you
don't want it to take longer, you can just not cancel it in the first
place.) This is not a problem if the server crashes in between (which
is the scenario I had in mind when doing the FINALIZE thing), because of
course no transaction can continue to run across a crash.

Yeah, it sounds like this will require some solution, but I agree that
just waiting "longer" seems acceptable.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#7Hao Wu
hawu@vmware.com
In reply to: Robert Haas (#6)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi hacker,

I tested the patch provided by Alvaro. It seems not well in REPEATABLE READ.

gpadmin=*# \d+ tpart
Partitioned table "public.tpart"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
i | integer | | | | plain | |
j | integer | | | | plain | |
Partition key: RANGE (i)
Partitions: tpart_1 FOR VALUES FROM (0) TO (100),
tpart_2 FOR VALUES FROM (100) TO (200)

begin isolation level repeatable read;
BEGIN
gpadmin=*# select * from tpart;
i | j
-----+-----
10 | 10
50 | 50
110 | 110
120 | 120
150 | 150
(5 rows)
-- Here, run `ALTER TABLE tpart DROP PARTITION tpart_2 CONCURRENTLY`
-- but only complete the first transaction.

-- the tuples from tpart_2 are gone.
gpadmin=*# select * from tpart;
i | j
----+----
10 | 10
50 | 50
(2 rows)

gpadmin=*# \d+ tpart_2
Table "public.tpart_2"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
i | integer | | | | plain | |
j | integer | | | | plain | |
Partition of: tpart FOR VALUES FROM (100) TO (200)
Partition constraint: ((i IS NOT NULL) AND (i >= 100) AND (i < 200))
Access method: heap

-- the part tpart_2 is not showed as DETACHED
gpadmin=*# \d+ tpart
Partitioned table "public.tpart"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
i | integer | | | | plain | |
j | integer | | | | plain | |
Partition key: RANGE (i)
Partitions: tpart_1 FOR VALUES FROM (0) TO (100),
tpart_2 FOR VALUES FROM (100) TO (200)

-- next, the insert failed. It's OK.
gpadmin=*# insert into tpart values(130,130);
ERROR: no partition of relation "tpart" found for row
DETAIL: Partition key of the failing row contains (i) = (130).

Is this an expected behavior?

Regards,
Hao Wu

________________________________
From: Robert Haas <robertmhaas@gmail.com>
Sent: Thursday, August 27, 2020 11:46 PM
To: Alvaro Herrera <alvherre@2ndquadrant.com>
Cc: Pg Hackers <pgsql-hackers@lists.postgresql.org>
Subject: Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Wed, Aug 26, 2020 at 7:40 PM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

To mark it detached means to set pg_inherits.inhdetached=true. That
column name is a bit of a misnomer, since that column really means "is
in the process of being detached"; the pg_inherits row goes away
entirely once the detach process completes. This mark guarantees that
everyone will see that row because the detaching session waits long
enough after committing the first transaction and before deleting the
pg_inherits row, until everyone else has moved on.

OK. Do you just wait until the XID of the transaction that set
inhdetached is all-visible, or how do you do it?

The other point is that the partition directory code can be asked to
include detached partitions, or not to. The executor does the former,
and the planner does the latter. This allows a transient period during
which the partition descriptor returned to planner and executor is
different; this makes the situation equivalent to what would have
happened if the partition was attached during the operation: in executor
we would detect that there is an additional partition that was not seen
by the planner, and we already know how to deal with that situation by
your handling of the ATTACH code.

Ah ha! That is quite clever and I don't think that I would have
thought of it. So all the plans that were created before you set
inhdetached=true have to be guaranteed to be invaliated or gone
altogether before you can actually delete the pg_inherits row. It
seems like it ought to be possible to ensure that, though I am not
surely of the details exactly. Is it sufficient to wait for all
transactions that have locked the table to go away? I'm not sure
exactly how this stuff interacts with the plan cache.

There is one fly in the ointment though, which is that if you cancel the
wait and then immediately do the ALTER TABLE DETACH FINALIZE without
waiting for as long as the original execution would have waited, you
might end up killing the partition ahead of time. One solution to this
would be to cause the FINALIZE action to wait again at start. This
would cause it to take even longer, but it would be safer. (If you
don't want it to take longer, you can just not cancel it in the first
place.) This is not a problem if the server crashes in between (which
is the scenario I had in mind when doing the FINALIZE thing), because of
course no transaction can continue to run across a crash.

Yeah, it sounds like this will require some solution, but I agree that
just waiting "longer" seems acceptable.

--
Robert Haas
EnterpriseDB: https://urldefense.proofpoint.com/v2/url?u=http-3A__www.enterprisedb.com&amp;d=DwIBaQ&amp;c=lnl9vOaLMzsy2niBC8-h_K-7QJuNJEsFrzdndhuJ3Sw&amp;r=tqYUKh-fXcYPWSaF4E-D6A&amp;m=SEDl-6dEISo7BA0qWuv1-idQUVtO0M6qz7hcfwlrF3I&amp;s=pZ7Dx6xrJOYkKKMlXR4wpJNZv-W10wQkMfXdEjtIXJY&amp;e=
The Enterprise PostgreSQL Company

#8Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Robert Haas (#6)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Aug-27, Robert Haas wrote:

On Wed, Aug 26, 2020 at 7:40 PM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

To mark it detached means to set pg_inherits.inhdetached=true. That
column name is a bit of a misnomer, since that column really means "is
in the process of being detached"; the pg_inherits row goes away
entirely once the detach process completes. This mark guarantees that
everyone will see that row because the detaching session waits long
enough after committing the first transaction and before deleting the
pg_inherits row, until everyone else has moved on.

OK. Do you just wait until the XID of the transaction that set
inhdetached is all-visible, or how do you do it?

I'm just doing WaitForLockers( ... AccessExclusiveLock ...) on the
partitioned table at the start of the second transaction. That will
wait until all lockers that have obtained a partition descriptor with
the old definition are gone. Note we don't actually lock the
partitioned table with that lock level.

In the second transaction we additionally obtain AccessExclusiveLock on
the partition itself, but that's after nobody sees it as a partition
anymore. That lock level is needed for some of the internal DDL
changes, and should not cause problems.

I thought about using WaitForOlderSnapshots() instead of waiting for
lockers, but it didn't seem to solve any actual problem.

Note that on closing the first transaction, the locks on both tables are
released. This avoids the deadlock hazards because of the lock upgrades
that would otherwise occur. This means that the tables could be dropped
or changed in the meantime. The case where either relation is dropped
is handled by using try_relation_open() in the second transaction; if
either table is gone, then we can just mark the operation as completed.
This part is a bit fuzzy. One thing that should probably be done is
have a few operations (such as other ALTER TABLE) raise an error when
run on a table with inhdetached=true, because that might get things out
of step and potentially cause other problems. I've not done that yet.

So all the plans that were created before you set
inhdetached=true have to be guaranteed to be invaliated or gone
altogether before you can actually delete the pg_inherits row. It
seems like it ought to be possible to ensure that, though I am not
surely of the details exactly. Is it sufficient to wait for all
transactions that have locked the table to go away? I'm not sure
exactly how this stuff interacts with the plan cache.

Hmm, any cached plan should be released with relcache inval events, per
PlanCacheRelCallback(). There are some comments in plancache.h about
"unsaved" cached plans that I don't really understand :-(

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#9Hao Wu
hawu@vmware.com
In reply to: Hao Wu (#7)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Not related to DETACH PARTITION, but I found a bug in ATTACH PARTITION.
Here are the steps to reproduce the issue:

create table tpart(i int, j int) partition by range(i);
create table tpart_1(like tpart);
create table tpart_2(like tpart);
create table tpart_default(like tpart);alter table tpart attach partition tpart_1 for values from(0) to (100);
alter table tpart attach partition tpart_default default;insert into tpart_2 values(110,110),(120,120),(150,150);1: begin;
1: alter table tpart attach partition tpart_2 for values from(100) to (200);
2: begin;
-- insert will be blocked by ALTER TABLE ATTACH PARTITION
2: insert into tpart values (110,110),(120,120),(150,150);
1: end;
2: select * from tpart_default;
2: end;

After that the partition tpart_default contains (110,110),(120,120),(150,150)
inserted in session 2, which obviously violates the partition constraint.

Regards,
Hao Wu

#10Amit Langote
amitlangote09@gmail.com
In reply to: Hao Wu (#9)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi Hao,

On Wed, Sep 2, 2020 at 5:25 PM Hao Wu <hawu@vmware.com> wrote:

Not related to DETACH PARTITION, but I found a bug in ATTACH PARTITION.
Here are the steps to reproduce the issue:

create table tpart(i int, j int) partition by range(i);
create table tpart_1(like tpart);
create table tpart_2(like tpart);
create table tpart_default(like tpart);alter table tpart attach partition tpart_1 for values from(0) to (100);
alter table tpart attach partition tpart_default default;insert into tpart_2 values(110,110),(120,120),(150,150);1: begin;
1: alter table tpart attach partition tpart_2 for values from(100) to (200);
2: begin;
-- insert will be blocked by ALTER TABLE ATTACH PARTITION
2: insert into tpart values (110,110),(120,120),(150,150);
1: end;
2: select * from tpart_default;
2: end;

After that the partition tpart_default contains (110,110),(120,120),(150,150)
inserted in session 2, which obviously violates the partition constraint.

Thanks for the report. That looks like a bug.

I have started another thread to discuss this bug and a patch to fix
it to keep the discussion here focused on the new feature.

Subject: default partition and concurrent attach partition
/messages/by-id/CA+HiwqFqBmcSSap4sFnCBUEL_VfOMmEKaQ3gwUhyfa4c7J_-nA@mail.gmail.com

--
Amit Langote
EnterpriseDB: http://www.enterprisedb.com

#11Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Hao Wu (#7)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Aug-31, Hao Wu wrote:

I tested the patch provided by Alvaro. It seems not well in REPEATABLE READ.

-- the tuples from tpart_2 are gone.
gpadmin=*# select * from tpart;
i | j
----+----
10 | 10
50 | 50
(2 rows)

Interesting example, thanks. It seems this can be fixed without
breaking anything else by changing the planner so that it includes
detached partitions when we are in a snapshot-isolation transaction.
Indeed, the results from the detach-partition-concurrently-1.spec
isolation test are more satisfying with this change.

The attached v2, on current master, includes that change, as well as
fixes a couple of silly bugs in the previous one.

(Because of experimenting with git workflow I did not keep the 0001
part split in this one, but that part is unchanged from v1.)

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

v2-0001-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From 3ae63eba543990d78c233ebeffa299e25c446f42 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Thu, 10 Sep 2020 17:50:41 -0300
Subject: [PATCH v2] ALTER TABLE ...  DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  25 +-
 src/backend/catalog/heap.c                    |  13 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/pg_inherits.c             |  51 +-
 src/backend/commands/indexcmds.c              |   5 +-
 src/backend/commands/tablecmds.c              | 510 ++++++++++++++----
 src/backend/commands/trigger.c                |   7 +-
 src/backend/executor/execPartition.c          |  16 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/plancat.c          |  11 +-
 src/backend/parser/gram.y                     |  22 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/bin/psql/describe.c                       |  40 +-
 src/include/catalog/pg_inherits.h             |   7 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 .../detach-partition-concurrently-1.out       | 233 ++++++++
 .../detach-partition-concurrently-2.out       |  66 +++
 src/test/isolation/isolation_schedule         |   2 +
 .../detach-partition-concurrently-1.spec      |  76 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 26 files changed, 1035 insertions(+), 160 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 508bea3bc6..abef70ad46 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4481,6 +4481,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetached</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index e9c6a8a6c1..c35e83f229 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction,
+      <literal>SHARE UPDATE EXCLUSIVE</literal> is taken in both parent table and
+      partition, and the partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, complete actions of a
+      previous <literal>DETACH CONCURRENTLY</literal> invocation that
+      was cancelled or crashed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 67144aa3c9..e4a4064511 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2544,10 +2544,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3774,7 +3776,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 117e3fdef7..ffd2c2770f 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1669,7 +1669,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2255,7 +2255,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 17f37eb39f..f143db0ad4 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -200,7 +210,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -429,6 +440,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -448,10 +460,21 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * 'detached' is the expected state of the inhdetached flag.  If the catalog
+ * row does not match that state, an error is raised.  When used on
+ * partitions, the partition name must be passed, for possible error messages.
+ *
+ * If allow_detached is passed false, then an error is raised if the child is
+ * already marked detached.  A name must be supplied in that case, for the
+ * error message.  This should only be used with table partitions.
+ * XXX the name bit is pretty weird and problematic for non-partition cases;
+ * what if the flag check fails?
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -478,6 +501,28 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	inhdetached;
+
+			inhdetached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (inhdetached && !detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no partial concurrent detach in progress.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index f1b5f87e6a..424eb6a19b 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -1105,7 +1105,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1162,7 +1162,7 @@ DefineIndex(Oid relationId,
 		 */
 		if (!stmt->relation || stmt->relation->inh)
 		{
-			PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+			PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 			int			nparts = partdesc->nparts;
 			Oid		   *part_oids = palloc(sizeof(Oid) * nparts);
 			bool		invalidate_parent = false;
@@ -3751,6 +3751,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3e57c7f9e1..46a84fddfb 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -157,6 +157,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -354,7 +356,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -541,7 +543,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd);
 static void AttachPartitionEnsureIndexes(Relation rel, Relation attachrel);
@@ -550,7 +553,12 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -984,7 +992,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3152,7 +3161,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3351,7 +3360,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -3970,7 +3979,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4349,6 +4365,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4389,7 +4410,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4398,10 +4418,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4413,7 +4433,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4439,11 +4463,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -4729,7 +4754,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
@@ -5540,6 +5570,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
@@ -6159,7 +6190,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6392,7 +6423,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6546,7 +6578,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -7787,7 +7819,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8251,7 +8284,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -8866,7 +8900,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9000,7 +9034,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10781,7 +10815,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11165,7 +11200,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14060,7 +14096,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14071,12 +14107,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetached = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14090,7 +14184,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14106,7 +14200,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16214,7 +16310,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16270,7 +16366,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16858,105 +16954,226 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with
+	 * ShareUpdateExclusiveLock in the first stage.
+	 */
+	partRel = table_openrv(name, concurrent ?
+						   ShareUpdateExclusiveLock : AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps lock
+	 * on the referencing tables, which prevent concurrent transactions from
+	 * adding rows that we wouldn't see.  For this to work in concurrent mode,
+	 * it is critical that the partition appears as no longer attached for the
+	 * RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint.  The reason for this is
+	 * that the planner may have made optimizations that depend on the
+	 * constraint.  XXX Isn't it sufficient to invalidate the partition's
+	 * relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all process to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		Constraint *n;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint on partition, equivalent to the partition constraint */
+		n = makeNode(Constraint);
+		n->contype = CONSTR_CHECK;
+		n->conname = NULL;
+		n->location = -1;
+		n->is_no_inherit = false;
+		n->raw_expr = NULL;
+		n->cooked_expr = nodeToString(make_ands_explicit(RelationGetPartitionQual(partRel)));
+		n->initially_valid = true;
+		n->skip_validation = true;
+		/* It's a re-add, since it nominally already exists */
+		ATAddCheckConstraint(wqueue, tab, partRel, n,
+							 true, false, true, ShareUpdateExclusiveLock);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17029,17 +17246,98 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
@@ -17240,7 +17538,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17394,7 +17692,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 672fccff5b..ce65abd57e 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1054,7 +1054,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1076,7 +1076,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
@@ -1537,7 +1538,7 @@ EnableDisableTrigger(Relation rel, const char *tgname,
 			if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE &&
 				(TRIGGER_FOR_ROW(oldtrig->tgtype)))
 			{
-				PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+				PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 				int			i;
 
 				for (i = 0; i < partdesc->nparts; i++)
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index bd2ea25804..e4779c5433 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -564,6 +564,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	rootrel = rootResultRelInfo->ri_RelationDesc,
 				partrel;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -574,7 +575,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1054,7 +1055,7 @@ ExecInitPartitionDispatchInfo(EState *estate,
 
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1636,7 +1637,7 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1702,9 +1703,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 0409a40b82..044488f14c 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4669,6 +4669,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index e2d1b987bf..8fb1d082e5 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2931,6 +2931,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index f9d0d67aa7..0fe053be96 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2123,10 +2123,17 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 * In snapshot-isolation transactions, we always include detached
+	 * partitions, because otherwise they become invisible too soon.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index c5154b818c..3bfbfdd8a5 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -649,7 +649,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2057,12 +2057,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2070,8 +2071,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2086,6 +2100,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15114,6 +15129,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 58f9b46289..0e872b9283 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2808,7 +2808,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	ParseState *pstate = make_parsestate(NULL);
 	int			with = -1;
@@ -3982,7 +3982,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4182,7 +4182,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 0f124a5254..301d1c6844 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -38,6 +38,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -47,7 +48,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -62,13 +63,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -89,7 +91,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -111,7 +113,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -239,6 +242,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -281,7 +285,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -296,6 +300,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 	pdir->pdir_mcxt = mcxt;
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -328,7 +333,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 9713a7ac41..81d3c15de8 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1264,6 +1264,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 0861d74a6f..86c23ea66a 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2109,7 +2109,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetached" :
+						  "false as inhdetached");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2127,17 +2132,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3179,9 +3186,18 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetached,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3190,14 +3206,14 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3247,11 +3263,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACHED)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 0432c3ab88..fcdea9f26f 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid;
 	Oid			inhparent;
 	int32		inhseqno;
+	bool		inhdetached;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -44,7 +45,8 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 typedef FormData_pg_inherits *Form_pg_inherits;
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -52,6 +54,7 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index e83329fd6d..939ccb4d4a 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -859,6 +859,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1846,6 +1847,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity				/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 08f22ce211..27414b0912 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -163,6 +163,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 70df764981..b1ee9fd5e3 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..ce493e3b93
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,233 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s1s s1c s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2d: <... completed>
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2d s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2d s1ins s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1s s2d s1ins s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1s s2d s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec: EXECUTE f(1);
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec2 s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2d: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2d s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2d: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2d s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2d: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1b s1s s2d s3drop1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop1: DROP TABLE d_listp; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3drop1: <... completed>
+error in steps s1c s2d s3drop1: ERROR:  partitioned table "d_listp" was removed concurrently
+
+starting permutation: s1b s1s s2d s3drop2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop2: DROP TABLE d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3drop2: <... completed>
+error in steps s1c s2d s3drop2: ERROR:  partition "d_listp2" was removed concurrently
+
+starting permutation: s1b s1s s2d s3detach s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3detach: ALTER TABLE d_listp DETACH PARTITION d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3detach: <... completed>
+error in steps s1c s2d s3detach: ERROR:  cannot detach partition "d_listp2"
+
+starting permutation: s1b s1s s2d s3rename s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3rename: ALTER TABLE d_listp2 RENAME TO d_listp_foobar; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3rename: <... completed>
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index aa386ab1a2..c381fcd967 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,8 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..f035db8c5d
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,76 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1exec"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3drop1"	{ DROP TABLE d_listp; }
+step "s3drop2"	{ DROP TABLE d_listp2; }
+step "s3detach"	{ ALTER TABLE d_listp DETACH PARTITION d_listp2; }
+step "s3rename"	{ ALTER TABLE d_listp2 RENAME TO d_listp_foobar; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s1s" "s1c" "s2drop" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2d" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# Session s1 no longer sees the partition ...
+permutation "s1b" "s1s" "s2d" "s1ins" "s1c"
+# ... regardless of the isolation level.
+permutation "s1brr" "s1s" "s2d" "s1ins" "s1c"
+# However, DETACH CONCURRENTLY is not isolation-level-correct
+permutation "s1brr" "s1s" "s2d" "s1s" "s1c"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec2" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2d" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2d" "s1s" "s1exec2" "s1c" "s1dealloc"
+
+
+# Note: the following tests are not very interesting in isolationtester because
+# the way this tool handles multiple sessions: it'll only run the drop/detach/
+# rename commands after s2 is blocked waiting.  It would be more useful to
+# run the DDL when s2 commits its first transaction instead.  Maybe these should
+# become TAP tests someday.
+
+# What happens if the partition or the parent table is dropped while waiting?
+permutation "s1b" "s1s" "s2d" "s3drop1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3drop2" "s1c"
+# Also try a non-concurrent detach while the other one is waiting
+permutation "s1b" "s1s" "s2d" "s3detach" "s1c"
+# and some other DDL
+permutation "s1b" "s1s" "s2d" "s3rename" "s1c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"

base-commit: 58c6feccfae1321ea4ff16c535c6f3fb90a07d69
-- 
2.20.1

#12Robert Haas
robertmhaas@gmail.com
In reply to: Alvaro Herrera (#11)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Thu, Sep 10, 2020 at 4:54 PM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

Interesting example, thanks. It seems this can be fixed without
breaking anything else by changing the planner so that it includes
detached partitions when we are in a snapshot-isolation transaction.
Indeed, the results from the detach-partition-concurrently-1.spec
isolation test are more satisfying with this change.

Hmm, so I think the idea here is that since we're out-waiting plans
with the old partition descriptor by waiting for lock release, it's OK
for anyone who has a lock to keep using the old partition descriptor
as long as they continuously hold the lock. Is that right? I can't
think of a hole in that logic, but it's probably worth noting in the
comments, in case someone is tempted to change the way that we
out-wait plans with the old partition descriptor to some other
mechanism.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#13Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#1)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi Alvaro,

Studying the patch to understand how it works.

On Tue, Aug 4, 2020 at 8:49 AM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

Why two transactions? The reason is that in order for this to work, we
make a catalog change (mark it detached), and commit so that all
concurrent transactions can see the change. A second transaction waits
for anybody who holds any lock on the partitioned table and grabs Access
Exclusive on the partition (which now no one cares about, if they're
looking at the partitioned table), where the DDL action on the partition
can be completed.

+   /*
+    * Concurrent mode has to work harder; first we add a new constraint to the
+    * partition that matches the partition constraint.  The reason for this is
+    * that the planner may have made optimizations that depend on the
+    * constraint.  XXX Isn't it sufficient to invalidate the partition's
+    * relcache entry?
...
+       /* Add constraint on partition, equivalent to the partition
constraint */
+       n = makeNode(Constraint);
+       n->contype = CONSTR_CHECK;
+       n->conname = NULL;
+       n->location = -1;
+       n->is_no_inherit = false;
+       n->raw_expr = NULL;
+       n->cooked_expr =
nodeToString(make_ands_explicit(RelationGetPartitionQual(partRel)));
+       n->initially_valid = true;
+       n->skip_validation = true;
+       /* It's a re-add, since it nominally already exists */
+       ATAddCheckConstraint(wqueue, tab, partRel, n,
+                            true, false, true, ShareUpdateExclusiveLock);

I suspect that we don't really need this defensive constraint. I mean
even after committing the 1st transaction, the partition being
detached still has relispartition set to true and
RelationGetPartitionQual() still returns the partition constraint, so
it seems the constraint being added above is duplicative. Moreover,
the constraint is not removed as part of any cleaning up after the
DETACH process, so repeated attach/detach of the same partition
results in the constraints piling up:

create table foo (a int, b int) partition by range (a);
create table foo1 partition of foo for values from (1) to (2);
create table foo2 partition of foo for values from (2) to (3);
alter table foo detach partition foo2 concurrently;
alter table foo attach partition foo2 for values from (2) to (3);
alter table foo detach partition foo2 concurrently;
alter table foo attach partition foo2 for values from (2) to (3);
\d foo2
Table "public.foo2"
Column | Type | Collation | Nullable | Default
--------+---------+-----------+----------+---------
a | integer | | |
b | integer | | |
Partition of: foo FOR VALUES FROM (2) TO (3)
Check constraints:
"foo2_a_check" CHECK (a IS NOT NULL AND a >= 2 AND a < 3)
"foo2_a_check1" CHECK (a IS NOT NULL AND a >= 2 AND a < 3)

Noticed a bug/typo in the patched RelationBuildPartitionDesc():

-   inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+   inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock,
+                                       include_detached);

You're passing NoLock for include_detached which means you never
actually end up including detached partitions from here. I think it
is due to this bug that partition-concurrent-attach test fails in my
run. Also, the error seen in the following hunk of
detach-partition-concurrently-1 test is not intentional:

+starting permutation: s1brr s1prep s1s s2d s1s s1exec2 s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a
+
+1
+2
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY;
<waiting ...>
+step s1s: SELECT * FROM d_listp;
+a
+
+1
+step s1exec2: EXECUTE f(2); DEALLOCATE f;
+step s2d: <... completed>
+error in steps s1exec2 s2d: ERROR:  no partition of relation
"d_listp" found for row
+step s1c: COMMIT;

As you're intending to make the executor always *include* detached
partitions, the insert should be able route (2) to d_listp2, the
detached partition. It's the bug mentioned above that causes the
executor's partition descriptor build to falsely fail to include the
detached partition.

--
Amit Langote
EnterpriseDB: http://www.enterprisedb.com

#14Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Amit Langote (#13)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Sep-23, Amit Langote wrote:

Hi Amit, thanks for reviewing this patch!

On Tue, Aug 4, 2020 at 8:49 AM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

I suspect that we don't really need this defensive constraint. I mean
even after committing the 1st transaction, the partition being
detached still has relispartition set to true and
RelationGetPartitionQual() still returns the partition constraint, so
it seems the constraint being added above is duplicative.

Ah, thanks for thinking through that. I had vague thoughts about this
being unnecessary in the current mechanics, but hadn't fully
materialized the thought. (The patch was completely different a few
unposted iterations ago).

Moreover, the constraint is not removed as part of any cleaning up
after the DETACH process, so repeated attach/detach of the same
partition results in the constraints piling up:

Yeah, I knew about this issue (mentioned in my self-reply on Aug 4) and
didn't worry too much about it because I was thinking I'd rather get rid
of the constraint addition in the first place.

Noticed a bug/typo in the patched RelationBuildPartitionDesc():

-   inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+   inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock,
+                                       include_detached);

You're passing NoLock for include_detached which means you never
actually end up including detached partitions from here.

I fixed this in the version I posted on Sept 10. I think you were
reading the version posted at the start of this thread.

Thanks,

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#15Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#14)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi Alvaro,

Sorry I totally failed to see the v2 you had posted and a couple of
other emails where you mentioned the issues I brought up.

On Thu, Sep 24, 2020 at 12:23 AM Alvaro Herrera
<alvherre@2ndquadrant.com> wrote:

On 2020-Sep-23, Amit Langote wrote:
I suspect that we don't really need this defensive constraint. I mean

even after committing the 1st transaction, the partition being
detached still has relispartition set to true and
RelationGetPartitionQual() still returns the partition constraint, so
it seems the constraint being added above is duplicative.

Ah, thanks for thinking through that. I had vague thoughts about this
being unnecessary in the current mechanics, but hadn't fully
materialized the thought. (The patch was completely different a few
unposted iterations ago).

Moreover, the constraint is not removed as part of any cleaning up
after the DETACH process, so repeated attach/detach of the same
partition results in the constraints piling up:

Yeah, I knew about this issue (mentioned in my self-reply on Aug 4) and
didn't worry too much about it because I was thinking I'd rather get rid
of the constraint addition in the first place.

Okay, gotcha.

Noticed a bug/typo in the patched RelationBuildPartitionDesc():

-   inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+   inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock,
+                                       include_detached);

You're passing NoLock for include_detached which means you never
actually end up including detached partitions from here.

I fixed this in the version I posted on Sept 10. I think you were
reading the version posted at the start of this thread.

I am trying the v2 now and I can confirm that those problems are now fixed.

However, I am a bit curious about including detached partitions in
some cases while not in other, which can result in a (to me)
surprising behavior as follows:

Session 1:

create table foo (a int, b int) partition by range (a);
create table foo1 partition of foo for values from (1) to (2);
create table foo2 partition of foo for values from (2) to (3);

...attach GDB and set breakpoint so as to block right after finishing
the 1st transaction of DETACH PARTITION CONCURRENTLY...
alter table foo detach partition foo2 concurrently;
<hits breakpoint, wait...>

Session 2:

begin;
insert into foo values (2); -- ok
select * from foo;
select * from foo; -- ?!
a | b
---+---
(0 rows)

Maybe, it's fine to just always exclude detached partitions, although
perhaps I am missing some corner cases that you have thought of?

Also, I noticed that looking up a parent's partitions via
RelationBuildPartitionDesc or directly will inspect inhdetached to
include or exclude partitions, but checking if a child table is a
partition of a given parent table via get_partition_parent doesn't.
Now if you fix get_partition_parent() to also take into account
inhdetached, for example, to return InvalidOid if true, then the
callers would need to not consider the child table a valid partition.
So, RelationGetPartitionQual() on a detached partition should actually
return NIL, making my earlier claim about not needing the defensive
CHECK constraint invalid. But maybe that's fine if all places agree
on a detached partition not being a valid partition anymore?

--
Amit Langote
EnterpriseDB: http://www.enterprisedb.com

#16Michael Paquier
michael@paquier.xyz
In reply to: Amit Langote (#15)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Thu, Sep 24, 2020 at 12:51:52PM +0900, Amit Langote wrote:

Also, I noticed that looking up a parent's partitions via
RelationBuildPartitionDesc or directly will inspect inhdetached to
include or exclude partitions, but checking if a child table is a
partition of a given parent table via get_partition_parent doesn't.
Now if you fix get_partition_parent() to also take into account
inhdetached, for example, to return InvalidOid if true, then the
callers would need to not consider the child table a valid partition.
So, RelationGetPartitionQual() on a detached partition should actually
return NIL, making my earlier claim about not needing the defensive
CHECK constraint invalid. But maybe that's fine if all places agree
on a detached partition not being a valid partition anymore?

It would be good to get that answered, and while on it please note
that the patch needs a rebase.
--
Michael

#17Andy Fan
zhihui.fan1213@gmail.com
In reply to: Alvaro Herrera (#1)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi Alvaro:

On Tue, Aug 4, 2020 at 7:49 AM Alvaro Herrera <alvherre@2ndquadrant.com>
wrote:

I've been working on the ability to detach a partition from a
partitioned table, without causing blockages to concurrent activity.
I think this operation is critical for some use cases.

I think if it is possible to implement the detech with a NoWait option .

ALTER TABLE ... DETACH PARTITION .. [NoWait].

if it can't get the lock, raise "Resource is Busy" immediately, without
blocking others.
this should be a default behavior. If people do want to keep trying, it
can set
a ddl_lock_timeout to 'some-interval', in this case, it will still block
others(so it
can't be as good as what you are doing, but very simple), however the user
would know what would happen exactly and can coordinate with their
application accordingly. I'm sorry about this since it is a bit of
off-topics
or it has been discussed already.

--
Best Regards
Andy Fan

#18Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Andy Fan (#17)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Oct-15, Andy Fan wrote:

I think if it is possible to implement the detech with a NoWait option .

ALTER TABLE ... DETACH PARTITION .. [NoWait].

if it can't get the lock, raise "Resource is Busy" immediately,
without blocking others. this should be a default behavior. If
people do want to keep trying, it can set a ddl_lock_timeout to
'some-interval', in this case, it will still block others(so it can't
be as good as what you are doing, but very simple), however the user
would know what would happen exactly and can coordinate with their
application accordingly. I'm sorry about this since it is a bit of
off-topics or it has been discussed already.

Hi. I don't think this has been discussed, but it doesn't really solve
the use case I want to -- in many cases where the tables are
continuously busy, this would lead to starvation. I think the proposal
to make the algorithm work with reduced lock level is much more useful.

I think you can already do NOWAIT behavior, with LOCK TABLE .. NOWAIT
followed by DETACH PARTITION, perhaps with a nonzero statement timeout.

#19David Rowley
dgrowleyml@gmail.com
In reply to: Andy Fan (#17)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Thu, 15 Oct 2020 at 14:04, Andy Fan <zhihui.fan1213@gmail.com> wrote:

I think if it is possible to implement the detech with a NoWait option .

ALTER TABLE ... DETACH PARTITION .. [NoWait].

if it can't get the lock, raise "Resource is Busy" immediately, without blocking others.
this should be a default behavior. If people do want to keep trying, it can set
a ddl_lock_timeout to 'some-interval', in this case, it will still block others(so it
can't be as good as what you are doing, but very simple), however the user
would know what would happen exactly and can coordinate with their
application accordingly. I'm sorry about this since it is a bit of off-topics
or it has been discussed already.

How would that differ from setting a low lock_timeout and running the DDL?

I think what Alvaro wants to avoid is taking the AEL in the first
place. When you have multiple long overlapping queries to the
partitioned table, then there be no point in time where there are zero
locks on the table. It does not sound like your idea would help with
that.

David

#20Andy Fan
zhihui.fan1213@gmail.com
In reply to: David Rowley (#19)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi David/Alvaro:

On Thu, Oct 15, 2020 at 9:09 AM David Rowley <dgrowleyml@gmail.com> wrote:

On Thu, 15 Oct 2020 at 14:04, Andy Fan <zhihui.fan1213@gmail.com> wrote:

I think if it is possible to implement the detech with a NoWait option .

ALTER TABLE ... DETACH PARTITION .. [NoWait].

if it can't get the lock, raise "Resource is Busy" immediately, without

blocking others.

this should be a default behavior. If people do want to keep trying,

it can set

a ddl_lock_timeout to 'some-interval', in this case, it will still

block others(so it

can't be as good as what you are doing, but very simple), however the

user

would know what would happen exactly and can coordinate with their
application accordingly. I'm sorry about this since it is a bit of

off-topics

or it has been discussed already.

How would that differ from setting a low lock_timeout and running the DDL?

They are exactly the same (I didn't realize this parameter when I sent the
email).

I think what Alvaro wants to avoid is taking the AEL in the first
place.

I'm agreed with this, that's why I said "so it can't be as good as what
you are doing"

When you have multiple long overlapping queries to the
partitioned table, then there be no point in time where there are zero
locks on the table. It does not sound like your idea would help with that.

Based on my current knowledge, "detach" will hold an exclusive lock
and it will have higher priority than other waiters. so it has to wait for
the lock
holder before it (named as sess 1). and at the same time, block all the
other
waiters which are requiring a lock even the lock mode is compatible with
session 1.
So "deteach" can probably get its lock in a short time (unless some long
transaction
before it). I'm not sure if I have some misunderstanding here.

Overall I'd be +1 for this patch.

--
Best Regards
Andy Fan

#21Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#15)
2 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Sep-24, Amit Langote wrote:

Hello Amit,

Sorry I totally failed to see the v2 you had posted and a couple of
other emails where you mentioned the issues I brought up.

No worries, I appreciate you reviewing this.

However, I am a bit curious about including detached partitions in
some cases while not in other, which can result in a (to me)
surprising behavior as follows:

[ snip ]

begin;
insert into foo values (2); -- ok
select * from foo;
select * from foo; -- ?!
a | b
---+---
(0 rows)

Maybe, it's fine to just always exclude detached partitions, although
perhaps I am missing some corner cases that you have thought of?

Well, this particular case can be fixed by changing
ExecInitPartitionDispatchInfo specifically, from including detached
partitions to excluding them, as in the attached version. Given your
example I think the case is fairly good that they should be excluded
there. I can't think of a case that this change break.

However I'm not sure that excluding them everywhere is sensible. There
are currently two cases where they are included (search for calls to
CreatePartitionDirectory if you're curious). One is snapshot-isolation
transactions (repeatable read and serializable) in
set_relation_partition_info, per the example from Hao Wu. If we simply
exclude detached transaction there, repeatable read no longer works
properly; rows will just go missing for no apparent reason. I don't
think this is acceptable.

The other case is ExecCreatePartitionPruneState(). The whole point of
including detached partitions here is to make them available for queries
that were planned before the detach and executed after the detach. My
fear is that the pruning plan will contain references (from planner) to
partitions that the executor doesn't know about. If there are extra
partitions at the executor side, it shouldn't harm anything (and it
shouldn't change query results); but I'm not sure that things will work
OK if partitions seen by the planner disappear from under the executor.

I'm posting this version just as a fresh rebase -- it is not yet
intended for commit. I haven't touched the constraint stuff. I still
think that that can be removed before commit, which is a simple change;
but even if not, the problem with the duplicated constraints should be
easy to fix.

Again, my thanks for reviewing.

Attachments:

v3-0001-Let-ALTER-TABLE-exec-routines-deal-with-the-relat.patchtext/x-diff; charset=us-asciiDownload
From 2cf81404fb4fa4e85665d6695361830d015acac0 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 13 Jul 2020 20:15:30 -0400
Subject: [PATCH v3 1/2] Let ALTER TABLE exec routines deal with the relation

This means that ATExecCmd relies on AlteredRelationInfo->rel instead of
keeping the relation as a local variable; this is useful if the
subcommand needs to modify the relation internally.  For example, a
subcommand that internally commits its transaction needs this.
---
 src/backend/commands/tablecmds.c | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 511f015a86..2784fb11f9 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -157,6 +157,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -354,7 +356,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -4393,7 +4395,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4402,10 +4403,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4417,7 +4418,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4443,11 +4448,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -5545,6 +5551,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
-- 
2.20.1

v3-0002-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From e992924eaf6516db4553cd44428a1dceb10aca64 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 3 Aug 2020 19:21:09 -0400
Subject: [PATCH v3 2/2] ALTER TABLE ... DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  25 +-
 src/backend/catalog/heap.c                    |  13 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/partition.c               |  28 +-
 src/backend/catalog/pg_inherits.c             |  51 +-
 src/backend/commands/indexcmds.c              |   5 +-
 src/backend/commands/tablecmds.c              | 514 ++++++++++++++----
 src/backend/commands/trigger.c                |   7 +-
 src/backend/executor/execPartition.c          |  16 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/plancat.c          |  11 +-
 src/backend/parser/gram.y                     |  23 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/backend/utils/cache/partcache.c           |  15 +-
 src/bin/psql/describe.c                       |  40 +-
 src/include/catalog/pg_inherits.h             |   7 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 .../detach-partition-concurrently-1.out       | 233 ++++++++
 .../detach-partition-concurrently-2.out       |  66 +++
 src/test/isolation/isolation_schedule         |   2 +
 .../detach-partition-concurrently-1.spec      |  76 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 28 files changed, 1072 insertions(+), 171 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 3927b1030d..6731c28993 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4481,6 +4481,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetached</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..1ff6214fa7 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction,
+      <literal>SHARE UPDATE EXCLUSIVE</literal> is taken in both parent table and
+      partition, and the partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, complete actions of a
+      previous <literal>DETACH CONCURRENTLY</literal> invocation that
+      was cancelled or crashed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 67144aa3c9..e4a4064511 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2544,10 +2544,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3774,7 +3776,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 0974f3e23a..6a3d033780 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1669,7 +1669,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2255,7 +2255,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 239ac017fa..1e651ca763 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -32,7 +32,7 @@
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
+static Oid	get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 										   List **ancestors);
 
@@ -42,6 +42,9 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
+ * If the partition is in the process of being detached, InvalidOid is
+ * returned and no error is thrown.
+ *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
@@ -51,12 +54,15 @@ get_partition_parent(Oid relid)
 {
 	Relation	catalogRelation;
 	Oid			result;
+	bool		detached;
 
 	catalogRelation = table_open(InheritsRelationId, AccessShareLock);
 
-	result = get_partition_parent_worker(catalogRelation, relid);
+	result = get_partition_parent_worker(catalogRelation, relid, &detached);
 
-	if (!OidIsValid(result))
+	if (detached)
+		result = InvalidOid;
+	else if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
 	table_close(catalogRelation, AccessShareLock);
@@ -70,13 +76,15 @@ get_partition_parent(Oid relid)
  *		given relation
  */
 static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
+get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
 {
 	SysScanDesc scan;
 	ScanKeyData key[2];
 	Oid			result = InvalidOid;
 	HeapTuple	tuple;
 
+	*detached = false;
+
 	ScanKeyInit(&key[0],
 				Anum_pg_inherits_inhrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
@@ -93,7 +101,14 @@ get_partition_parent_worker(Relation inhRel, Oid relid)
 	{
 		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
 
-		result = form->inhparent;
+		/* A partition being detached has no parent */
+		if (form->inhdetached)
+		{
+			*detached = true;
+			result = InvalidOid;
+		}
+		else
+			result = form->inhparent;
 	}
 
 	systable_endscan(scan);
@@ -134,9 +149,10 @@ static void
 get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 {
 	Oid			parentOid;
+	bool		detached;
 
 	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
+	parentOid = get_partition_parent_worker(inhRel, relid, &detached);
 	if (parentOid == InvalidOid)
 		return;
 
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 17f37eb39f..f143db0ad4 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -200,7 +210,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -429,6 +440,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -448,10 +460,21 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * 'detached' is the expected state of the inhdetached flag.  If the catalog
+ * row does not match that state, an error is raised.  When used on
+ * partitions, the partition name must be passed, for possible error messages.
+ *
+ * If allow_detached is passed false, then an error is raised if the child is
+ * already marked detached.  A name must be supplied in that case, for the
+ * error message.  This should only be used with table partitions.
+ * XXX the name bit is pretty weird and problematic for non-partition cases;
+ * what if the flag check fails?
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -478,6 +501,28 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	inhdetached;
+
+			inhdetached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (inhdetached && !detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no partial concurrent detach in progress.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 75552c64ed..0aae01ad9f 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -1104,7 +1104,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1161,7 +1161,7 @@ DefineIndex(Oid relationId,
 		 */
 		if (!stmt->relation || stmt->relation->inh)
 		{
-			PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+			PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 			int			nparts = partdesc->nparts;
 			Oid		   *part_oids = palloc(sizeof(Oid) * nparts);
 			bool		invalidate_parent = false;
@@ -3788,6 +3788,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 2784fb11f9..29b12bb8a2 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -543,7 +543,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd,
 										   AlterTableUtilityContext *context);
@@ -553,7 +554,12 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -987,7 +993,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3158,7 +3165,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3357,7 +3364,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -3976,7 +3983,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4355,6 +4369,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4740,7 +4759,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
@@ -6167,7 +6191,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6400,7 +6424,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6554,7 +6579,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -6654,17 +6679,22 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	if (rel->rd_rel->relispartition)
 	{
 		Oid			parentId = get_partition_parent(RelationGetRelid(rel));
-		Relation	parent = table_open(parentId, AccessShareLock);
-		TupleDesc	tupDesc = RelationGetDescr(parent);
+		Relation	parent;
+		TupleDesc	tupDesc;
 		AttrNumber	parent_attnum;
 
-		parent_attnum = get_attnum(parentId, colName);
-		if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
-					 errmsg("column \"%s\" is marked NOT NULL in parent table",
-							colName)));
-		table_close(parent, AccessShareLock);
+		if (parentId != InvalidOid)
+		{
+			parent = table_open(parentId, AccessShareLock);
+			tupDesc = RelationGetDescr(parent);
+			parent_attnum = get_attnum(parentId, colName);
+			if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
+				ereport(ERROR,
+						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+						 errmsg("column \"%s\" is marked NOT NULL in parent table",
+								colName)));
+			table_close(parent, AccessShareLock);
+		}
 	}
 
 	/*
@@ -7830,7 +7860,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8294,7 +8325,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -8909,7 +8941,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9043,7 +9075,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10824,7 +10856,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11208,7 +11241,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14103,7 +14137,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14114,12 +14148,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetached = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14133,7 +14225,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14149,7 +14241,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16257,7 +16351,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16317,7 +16411,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16905,105 +16999,226 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with
+	 * ShareUpdateExclusiveLock in the first stage.
+	 */
+	partRel = table_openrv(name, concurrent ?
+						   ShareUpdateExclusiveLock : AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps lock
+	 * on the referencing tables, which prevent concurrent transactions from
+	 * adding rows that we wouldn't see.  For this to work in concurrent mode,
+	 * it is critical that the partition appears as no longer attached for the
+	 * RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint.  The reason for this is
+	 * that the planner may have made optimizations that depend on the
+	 * constraint.  XXX Isn't it sufficient to invalidate the partition's
+	 * relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all process to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		Constraint *n;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint on partition, equivalent to the partition constraint */
+		n = makeNode(Constraint);
+		n->contype = CONSTR_CHECK;
+		n->conname = NULL;
+		n->location = -1;
+		n->is_no_inherit = false;
+		n->raw_expr = NULL;
+		n->cooked_expr = nodeToString(make_ands_explicit(RelationGetPartitionQual(partRel)));
+		n->initially_valid = true;
+		n->skip_validation = true;
+		/* It's a re-add, since it nominally already exists */
+		ATAddCheckConstraint(wqueue, tab, partRel, n,
+							 true, false, true, ShareUpdateExclusiveLock);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17076,17 +17291,98 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
@@ -17287,7 +17583,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17441,7 +17737,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 3b4fbdadf4..ee3fc5e39f 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1054,7 +1054,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1076,7 +1076,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
@@ -1537,7 +1538,7 @@ EnableDisableTrigger(Relation rel, const char *tgname,
 			if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE &&
 				(TRIGGER_FOR_ROW(oldtrig->tgtype)))
 			{
-				PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+				PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 				int			i;
 
 				for (i = 0; i < partdesc->nparts; i++)
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 33d2c6f63d..f7ef63a1ba 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -564,6 +564,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	rootrel = rootResultRelInfo->ri_RelationDesc,
 				partrel;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -574,7 +575,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1054,7 +1055,7 @@ ExecInitPartitionDispatchInfo(EState *estate,
 
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, false);
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1636,7 +1637,7 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1702,9 +1703,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 2b4d7654cc..94dab5f5c6 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4667,6 +4667,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index e2d1b987bf..8fb1d082e5 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2931,6 +2931,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index f9d0d67aa7..0fe053be96 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2123,10 +2123,17 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 * In snapshot-isolation transactions, we always include detached
+	 * partitions, because otherwise they become invisible too soon.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 480d168346..45be2677dc 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -651,7 +651,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2054,12 +2054,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2067,8 +2068,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2083,6 +2097,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15148,6 +15163,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
@@ -15686,6 +15702,7 @@ bare_label_keyword:
 			| EXTRACT
 			| FALSE_P
 			| FAMILY
+			| FINALIZE
 			| FIRST_P
 			| FLOAT_P
 			| FOLLOWING
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 66c42b5898..4a7445e8f5 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2808,7 +2808,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec, ParseState *pstate)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	int			with = -1;
 	bool		overlap = false;
@@ -4017,7 +4017,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4217,7 +4217,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 0f124a5254..301d1c6844 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -38,6 +38,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -47,7 +48,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -62,13 +63,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -89,7 +91,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -111,7 +113,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -239,6 +242,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -281,7 +285,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -296,6 +300,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 	pdir->pdir_mcxt = mcxt;
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -328,7 +333,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 9a35147b26..9c6a1c13c5 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1275,6 +1275,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index acf8a44f30..348b4f7dd2 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -341,6 +341,7 @@ generate_partition_qual(Relation rel)
 	bool		isnull;
 	List	   *my_qual = NIL,
 			   *result = NIL;
+	Oid			parentrelid;
 	Relation	parent;
 
 	/* Guard against stack overflow due to overly deep partition tree */
@@ -350,9 +351,19 @@ generate_partition_qual(Relation rel)
 	if (rel->rd_partcheckvalid)
 		return copyObject(rel->rd_partcheck);
 
+	/*
+	 * Obtain parent relid; if it's invalid, then the partition is being
+	 * detached
+	 */
+	parentrelid = get_partition_parent(RelationGetRelid(rel));
+	if (parentrelid == InvalidOid)
+	{
+		rel->rd_partcheckvalid = true;
+		return NIL;
+	}
+
 	/* Grab at least an AccessShareLock on the parent table */
-	parent = relation_open(get_partition_parent(RelationGetRelid(rel)),
-						   AccessShareLock);
+	parent = relation_open(parentrelid, AccessShareLock);
 
 	/* Get pg_class.relpartbound */
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 6bb0316bd9..9bcfb1d5fd 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2113,7 +2113,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetached" :
+						  "false as inhdetached");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2131,17 +2136,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3193,9 +3200,18 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetached,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3204,14 +3220,14 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3261,11 +3277,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACHED)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 0432c3ab88..fcdea9f26f 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid;
 	Oid			inhparent;
 	int32		inhseqno;
+	bool		inhdetached;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -44,7 +45,8 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 typedef FormData_pg_inherits *Form_pg_inherits;
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -52,6 +54,7 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 60c2f45466..95cee42c5f 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -860,6 +860,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1847,6 +1848,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity				/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 71dcdf2889..2d523568c1 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -163,6 +163,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD, AS_LABEL)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD, AS_LABEL)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD, BARE_LABEL)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD, BARE_LABEL)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 70df764981..b1ee9fd5e3 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..ce493e3b93
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,233 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s1s s1c s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2d: <... completed>
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2d s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2d s1ins s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1s s2d s1ins s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1s s2d s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec: EXECUTE f(1);
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec2 s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1brr s1prep s1s s2d s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2d: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2d s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2d: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2d s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2d: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1b s1s s2d s3drop1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop1: DROP TABLE d_listp; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3drop1: <... completed>
+error in steps s1c s2d s3drop1: ERROR:  partitioned table "d_listp" was removed concurrently
+
+starting permutation: s1b s1s s2d s3drop2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop2: DROP TABLE d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3drop2: <... completed>
+error in steps s1c s2d s3drop2: ERROR:  partition "d_listp2" was removed concurrently
+
+starting permutation: s1b s1s s2d s3detach s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3detach: ALTER TABLE d_listp DETACH PARTITION d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3detach: <... completed>
+error in steps s1c s2d s3detach: ERROR:  cannot detach partition "d_listp2"
+
+starting permutation: s1b s1s s2d s3rename s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3rename: ALTER TABLE d_listp2 RENAME TO d_listp_foobar; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3rename: <... completed>
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index aa386ab1a2..c381fcd967 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,8 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..f035db8c5d
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,76 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1exec"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3drop1"	{ DROP TABLE d_listp; }
+step "s3drop2"	{ DROP TABLE d_listp2; }
+step "s3detach"	{ ALTER TABLE d_listp DETACH PARTITION d_listp2; }
+step "s3rename"	{ ALTER TABLE d_listp2 RENAME TO d_listp_foobar; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s1s" "s1c" "s2drop" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2d" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# Session s1 no longer sees the partition ...
+permutation "s1b" "s1s" "s2d" "s1ins" "s1c"
+# ... regardless of the isolation level.
+permutation "s1brr" "s1s" "s2d" "s1ins" "s1c"
+# However, DETACH CONCURRENTLY is not isolation-level-correct
+permutation "s1brr" "s1s" "s2d" "s1s" "s1c"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec2" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1s" "s2d" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2d" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2d" "s1s" "s1exec2" "s1c" "s1dealloc"
+
+
+# Note: the following tests are not very interesting in isolationtester because
+# the way this tool handles multiple sessions: it'll only run the drop/detach/
+# rename commands after s2 is blocked waiting.  It would be more useful to
+# run the DDL when s2 commits its first transaction instead.  Maybe these should
+# become TAP tests someday.
+
+# What happens if the partition or the parent table is dropped while waiting?
+permutation "s1b" "s1s" "s2d" "s3drop1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3drop2" "s1c"
+# Also try a non-concurrent detach while the other one is waiting
+permutation "s1b" "s1s" "s2d" "s3detach" "s1c"
+# and some other DDL
+permutation "s1b" "s1s" "s2d" "s3rename" "s1c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
-- 
2.20.1

#22Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#21)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Here's an updated version of this patch.

Apart from rebasing to current master, I made the following changes:

* On the first transaction (the one that marks the partition as
detached), the partition is locked with ShareLock rather than
ShareUpdateExclusiveLock. This means that DML is not allowed anymore.
This seems a reasonable restriction to me; surely by the time you're
detaching the partition you're not inserting data into it anymore.

* In ExecInitPartitionDispatchInfo, the previous version always
excluded detached partitions. Now it does include them in isolation
level repeatable read and higher. Considering the point above, this
sounds a bit contradictory: you shouldn't be inserting new tuples in
partitions being detached. But if you do, it makes more sense: in RR
two queries that insert tuples in the same partition would not fail
mid-transaction. (In a read-committed transaction, the second query
does fail, but to me that does not sound surprising.)

* ALTER TABLE .. DETACH PARTITION FINALIZE now waits for concurrent old
snapshots, as previously discussed. This should ensure that the user
doesn't just cancel the wait during the second transaction by Ctrl-C and
run FINALIZE immediately afterwards, which I claimed would bring
inconsistency.

* Avoid creating the CHECK constraint if an identical one already
exists.

(Note I do not remove the constraint on ATTACH. That seems pointless.)

Still to do: test this using the new hook added by 6f0b632f083b.

Attachments:

v4-0001-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From 8135f82493892ad755176c8c4c8251355dae2927 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 3 Aug 2020 19:21:09 -0400
Subject: [PATCH v4] ALTER TABLE ... DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  25 +-
 src/backend/catalog/heap.c                    |  13 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/partition.c               |  28 +-
 src/backend/catalog/pg_inherits.c             |  51 +-
 src/backend/commands/indexcmds.c              |   7 +-
 src/backend/commands/tablecmds.c              | 587 ++++++++++++++----
 src/backend/commands/trigger.c                |   5 +-
 src/backend/executor/execPartition.c          |  29 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/plancat.c          |  11 +-
 src/backend/parser/gram.y                     |  23 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/backend/utils/cache/partcache.c           |  15 +-
 src/bin/psql/describe.c                       |  40 +-
 src/include/catalog/pg_inherits.h             |   7 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 src/include/utils/snapmgr.h                   |   1 +
 .../detach-partition-concurrently-1.out       | 301 +++++++++
 .../detach-partition-concurrently-2.out       |  66 ++
 src/test/isolation/isolation_schedule         |   2 +
 .../detach-partition-concurrently-1.spec      |  90 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 29 files changed, 1234 insertions(+), 178 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 5fb9dca425..b628ffaa4b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4482,6 +4482,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetached</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..1ff6214fa7 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction,
+      <literal>SHARE UPDATE EXCLUSIVE</literal> is taken in both parent table and
+      partition, and the partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, complete actions of a
+      previous <literal>DETACH CONCURRENTLY</literal> invocation that
+      was cancelled or crashed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 4cd7d76938..38728f268e 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2544,10 +2544,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3816,7 +3818,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index b88b4a1f12..95d1370cfe 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1831,7 +1831,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2424,7 +2424,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 239ac017fa..1e651ca763 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -32,7 +32,7 @@
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
+static Oid	get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 										   List **ancestors);
 
@@ -42,6 +42,9 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
+ * If the partition is in the process of being detached, InvalidOid is
+ * returned and no error is thrown.
+ *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
@@ -51,12 +54,15 @@ get_partition_parent(Oid relid)
 {
 	Relation	catalogRelation;
 	Oid			result;
+	bool		detached;
 
 	catalogRelation = table_open(InheritsRelationId, AccessShareLock);
 
-	result = get_partition_parent_worker(catalogRelation, relid);
+	result = get_partition_parent_worker(catalogRelation, relid, &detached);
 
-	if (!OidIsValid(result))
+	if (detached)
+		result = InvalidOid;
+	else if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
 	table_close(catalogRelation, AccessShareLock);
@@ -70,13 +76,15 @@ get_partition_parent(Oid relid)
  *		given relation
  */
 static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
+get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
 {
 	SysScanDesc scan;
 	ScanKeyData key[2];
 	Oid			result = InvalidOid;
 	HeapTuple	tuple;
 
+	*detached = false;
+
 	ScanKeyInit(&key[0],
 				Anum_pg_inherits_inhrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
@@ -93,7 +101,14 @@ get_partition_parent_worker(Relation inhRel, Oid relid)
 	{
 		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
 
-		result = form->inhparent;
+		/* A partition being detached has no parent */
+		if (form->inhdetached)
+		{
+			*detached = true;
+			result = InvalidOid;
+		}
+		else
+			result = form->inhparent;
 	}
 
 	systable_endscan(scan);
@@ -134,9 +149,10 @@ static void
 get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 {
 	Oid			parentOid;
+	bool		detached;
 
 	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
+	parentOid = get_partition_parent_worker(inhRel, relid, &detached);
 	if (parentOid == InvalidOid)
 		return;
 
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 17f37eb39f..f143db0ad4 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -200,7 +210,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -429,6 +440,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -448,10 +460,21 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * 'detached' is the expected state of the inhdetached flag.  If the catalog
+ * row does not match that state, an error is raised.  When used on
+ * partitions, the partition name must be passed, for possible error messages.
+ *
+ * If allow_detached is passed false, then an error is raised if the child is
+ * already marked detached.  A name must be supplied in that case, for the
+ * error message.  This should only be used with table partitions.
+ * XXX the name bit is pretty weird and problematic for non-partition cases;
+ * what if the flag check fails?
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -478,6 +501,28 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	inhdetached;
+
+			inhdetached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (inhdetached && !detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no partial concurrent detach in progress.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 75552c64ed..798e9c76d0 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -398,7 +398,7 @@ CompareOpclassOptions(Datum *opts1, Datum *opts2, int natts)
  * GetCurrentVirtualXIDs.  If, during any iteration, a particular vxid
  * doesn't show up in the output, we know we can forget about it.
  */
-static void
+void
 WaitForOlderSnapshots(TransactionId limitXmin, bool progress)
 {
 	int			n_old_snapshots;
@@ -1104,7 +1104,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1161,7 +1161,7 @@ DefineIndex(Oid relationId,
 		 */
 		if (!stmt->relation || stmt->relation->inh)
 		{
-			PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+			PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 			int			nparts = partdesc->nparts;
 			Oid		   *part_oids = palloc(sizeof(Oid) * nparts);
 			bool		invalidate_parent = false;
@@ -3788,6 +3788,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 4bb33ee03c..7d6aa91f78 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -158,6 +158,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -355,7 +357,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -542,7 +544,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd,
 										   AlterTableUtilityContext *context);
@@ -551,8 +554,14 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   List *partConstraint,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
+static void DetachAddConstraintIfNeeded(List **wqueue, Relation partRel);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid, bool wait);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -987,7 +996,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3158,7 +3168,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3357,7 +3367,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -3976,7 +3986,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4367,6 +4384,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4407,7 +4429,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4416,10 +4437,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4431,7 +4452,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4457,11 +4482,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -4748,7 +4774,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		case AT_AlterCollationRefreshVersion:
 			/* ATPrepCmd ensured it must be an index */
@@ -5564,6 +5595,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
@@ -6179,7 +6211,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6412,7 +6444,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6566,7 +6599,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -6666,17 +6699,22 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	if (rel->rd_rel->relispartition)
 	{
 		Oid			parentId = get_partition_parent(RelationGetRelid(rel));
-		Relation	parent = table_open(parentId, AccessShareLock);
-		TupleDesc	tupDesc = RelationGetDescr(parent);
+		Relation	parent;
+		TupleDesc	tupDesc;
 		AttrNumber	parent_attnum;
 
-		parent_attnum = get_attnum(parentId, colName);
-		if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
-					 errmsg("column \"%s\" is marked NOT NULL in parent table",
-							colName)));
-		table_close(parent, AccessShareLock);
+		if (parentId != InvalidOid)
+		{
+			parent = table_open(parentId, AccessShareLock);
+			tupDesc = RelationGetDescr(parent);
+			parent_attnum = get_attnum(parentId, colName);
+			if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
+				ereport(ERROR,
+						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+						 errmsg("column \"%s\" is marked NOT NULL in parent table",
+								colName)));
+			table_close(parent, AccessShareLock);
+		}
 	}
 
 	/*
@@ -7275,7 +7313,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), lockmode))
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -7858,7 +7896,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8322,7 +8361,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -8937,7 +8977,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9071,7 +9111,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10852,7 +10892,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11236,7 +11277,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14131,7 +14173,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14142,12 +14184,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetached = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14161,7 +14261,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14177,7 +14277,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16285,7 +16387,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16345,7 +16447,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16933,105 +17035,230 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with share-update-exclusive
+	 * in the first transaction.  This allows concurrent transactions to be
+	 * doing DML to the partition.
+	 */
+	partRel = table_openrv(name, concurrent ? ShareLock :
+						   AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps lock
+	 * on the referencing tables, which prevent concurrent transactions from
+	 * adding rows that we wouldn't see.  For this to work in concurrent mode,
+	 * it is critical that the partition appears as no longer attached for the
+	 * RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint, if there isn't a matching
+	 * one already.  The reason for this is that the planner may have made
+	 * optimizations that depend on the constraint.  XXX Isn't it sufficient to
+	 * invalidate the partition's relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all process to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint, if needed. XXX hopefully we can just remove this */
+		DetachAddConstraintIfNeeded(wqueue, partRel);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid, false);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid, bool wait)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	/*
+	 * If asked to, wait until existing snapshots are gone.  This is important
+	 * in the second transaction of DETACH PARTITION CONCURRENTLY is canceled:
+	 * the user could immediately run DETACH FINALIZE without actually waiting
+	 * for existing transactions.  We must not complete the detach action until
+	 * all such queries are complete (otherwise we would present them with an
+	 * inconsistent view of catalogs).
+	 */
+	if (wait)
+	{
+		Snapshot	snap = GetActiveSnapshot();
+
+		WaitForOlderSnapshots(snap->xmin, false);
+	}
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17104,22 +17331,150 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid, true);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
 }
 
+/*
+ * DetachAddConstraintIfNeeded
+ *		Subroutine for ATExecDetachPartition.  Create a constraint that
+ *		takes the place of the partition constraint, but avoid creating
+ *		a dupe if an equivalent constraint already exists.
+ */
+static void
+DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
+{
+	AlteredTableInfo *tab;
+	Expr	   *constraintExpr;
+	TupleDesc	td = RelationGetDescr(partRel);
+	Constraint *n;
+
+	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+
+	/* If an identical constraint exists, we don't need to create one */
+	if (td->constr && td->constr->num_check > 0)
+	{
+		for (int i = 0; i < td->constr->num_check; i++)
+		{
+			Node	*thisconstr;
+
+			thisconstr = stringToNode(td->constr->check[i].ccbin);
+
+			if (equal(constraintExpr, thisconstr))
+				return;
+		}
+	}
+
+	tab = ATGetQueueEntry(wqueue, partRel);
+
+	/* Add constraint on partition, equivalent to the partition constraint */
+	n = makeNode(Constraint);
+	n->contype = CONSTR_CHECK;
+	n->conname = NULL;
+	n->location = -1;
+	n->is_no_inherit = false;
+	n->raw_expr = NULL;
+	n->cooked_expr = nodeToString(constraintExpr);
+	n->initially_valid = true;
+	n->skip_validation = true;
+	/* It's a re-add, since it nominally already exists */
+	ATAddCheckConstraint(wqueue, tab, partRel, n,
+						 true, false, true, ShareUpdateExclusiveLock);
+}
+
 /*
  * DropClonedTriggersFromPartition
  *		subroutine for ATExecDetachPartition to remove any triggers that were
@@ -17315,7 +17670,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17469,7 +17824,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 092ac1646d..e461187b78 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1055,7 +1055,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1077,7 +1077,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 86594bd056..cc57dbf5a9 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -568,6 +568,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	rootrel = rootResultRelInfo->ri_RelationDesc,
 				partrel;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -578,7 +579,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1047,9 +1048,21 @@ ExecInitPartitionDispatchInfo(EState *estate,
 	int			dispatchidx;
 	MemoryContext oldcxt;
 
+	/*
+	 * For data modification, it is better that executor does not include
+	 * partitions being detached, except in snapshot-isolation mode.  This
+	 * means that a read-committed transaction immediately gets a "no
+	 * partition for tuple" error when a tuple is inserted into a partition
+	 * that's being detached concurrently, but a transaction in repeatable-
+	 * read mode can still use the partition.  Note that because partition
+	 * detach uses ShareLock on the partition (which conflicts with DML),
+	 * we're certain that the detach won't be able to complete until any
+	 * inserting transaction is done.
+	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt,
+									 IsolationUsesXactSnapshot());
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1629,9 +1642,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
+	/* Executor must always include detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1697,9 +1711,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 530aac68a7..c5d436dcfa 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4658,6 +4658,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 0cf90ef33c..53287fe9a5 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2923,6 +2923,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 52c01eb86b..e56a853d87 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2134,10 +2134,17 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 * In snapshot-isolation transactions, we always include detached
+	 * partitions, because otherwise they become invisible too soon.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 357ab93fb6..a8e034df10 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -651,7 +651,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2053,12 +2053,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2066,8 +2067,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2082,6 +2096,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15140,6 +15155,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
@@ -15678,6 +15694,7 @@ bare_label_keyword:
 			| EXTRACT
 			| FALSE_P
 			| FAMILY
+			| FINALIZE
 			| FIRST_P
 			| FLOAT_P
 			| FOLLOWING
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index ac0c495972..efb4a5e142 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2808,7 +2808,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec, ParseState *pstate)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	int			with = -1;
 	bool		overlap = false;
@@ -4017,7 +4017,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4217,7 +4217,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 0f124a5254..301d1c6844 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -38,6 +38,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -47,7 +48,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -62,13 +63,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -89,7 +91,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -111,7 +113,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -239,6 +242,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -281,7 +285,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -296,6 +300,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 	pdir->pdir_mcxt = mcxt;
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -328,7 +333,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index f398027fa6..6c3caaa1ba 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1275,6 +1275,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index acf8a44f30..348b4f7dd2 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -341,6 +341,7 @@ generate_partition_qual(Relation rel)
 	bool		isnull;
 	List	   *my_qual = NIL,
 			   *result = NIL;
+	Oid			parentrelid;
 	Relation	parent;
 
 	/* Guard against stack overflow due to overly deep partition tree */
@@ -350,9 +351,19 @@ generate_partition_qual(Relation rel)
 	if (rel->rd_partcheckvalid)
 		return copyObject(rel->rd_partcheck);
 
+	/*
+	 * Obtain parent relid; if it's invalid, then the partition is being
+	 * detached
+	 */
+	parentrelid = get_partition_parent(RelationGetRelid(rel));
+	if (parentrelid == InvalidOid)
+	{
+		rel->rd_partcheckvalid = true;
+		return NIL;
+	}
+
 	/* Grab at least an AccessShareLock on the parent table */
-	parent = relation_open(get_partition_parent(RelationGetRelid(rel)),
-						   AccessShareLock);
+	parent = relation_open(parentrelid, AccessShareLock);
 
 	/* Get pg_class.relpartbound */
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 07d640021c..9d2f1b562d 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2113,7 +2113,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetached" :
+						  "false as inhdetached");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2131,17 +2136,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3193,9 +3200,18 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetached,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3204,14 +3220,14 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3261,11 +3277,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACHED)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 0432c3ab88..fcdea9f26f 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid;
 	Oid			inhparent;
 	int32		inhseqno;
+	bool		inhdetached;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -44,7 +45,8 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 typedef FormData_pg_inherits *Form_pg_inherits;
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -52,6 +54,7 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index e1aeea2560..86d9839b1b 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -860,6 +860,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1851,6 +1852,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity,			/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 71dcdf2889..2d523568c1 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -163,6 +163,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD, AS_LABEL)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD, AS_LABEL)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD, BARE_LABEL)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD, BARE_LABEL)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 70df764981..b1ee9fd5e3 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index b6b403e293..df40a3b4a4 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -133,6 +133,7 @@ extern void AtEOXact_Snapshot(bool isCommit, bool resetXmin);
 extern void ImportSnapshot(const char *idstr);
 extern bool XactHasExportedSnapshots(void);
 extern void DeleteAllExportedSnapshotFiles(void);
+extern void WaitForOlderSnapshots(TransactionId limitXmin, bool progress);
 extern bool ThereAreNoPriorRegisteredSnapshots(void);
 extern bool TransactionIdLimitedForOldSnapshots(TransactionId recentXmin,
 												Relation relation,
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..8d15beac60
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,301 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2detach s1s s1c s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1s s1ins2 s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1ins s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1ins2 s2detach s3ins2 s1c s3debugtest
+step s1b: BEGIN;
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3ins2: INSERT INTO d_listp VALUES (2); <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3ins2: <... completed>
+step s3debugtest: SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2;
+tableoid       
+
+d_listp1       
+a              
+
+2              
+2              
+2              
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec1 s3s s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec1: EXECUTE f(1);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1prep s1exec2 s2detach s1s s1exec2 s3s s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1exec2: EXECUTE f(2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+2              
+step s1exec2: EXECUTE f(2);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep2 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep2: PREPARE f(int) AS INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1b s1s s2detach s3drop1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop1: DROP TABLE d_listp; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop1: <... completed>
+error in steps s1c s2detach s3drop1: ERROR:  partitioned table "d_listp" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3drop2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop2: DROP TABLE d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop2: <... completed>
+error in steps s1c s2detach s3drop2: ERROR:  partition "d_listp2" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3detach s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3detach: ALTER TABLE d_listp DETACH PARTITION d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3detach: <... completed>
+error in steps s1c s2detach s3detach: ERROR:  cannot detach partition "d_listp2"
+
+starting permutation: s1b s1s s2detach s3rename s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3rename: ALTER TABLE d_listp2 RENAME TO d_listp_foobar; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3rename: <... completed>
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index f2e752c445..3c8fa4e07d 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,8 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..1e9bea8198
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,90 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1ins2"	{ INSERT INTO d_listp VALUES (2); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1prep2"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (2); }
+step "s1exec1"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2detach"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3ins2"		{ INSERT INTO d_listp VALUES (2); }
+step "s3drop1"	{ DROP TABLE d_listp; }
+step "s3drop2"	{ DROP TABLE d_listp2; }
+step "s3detach"	{ ALTER TABLE d_listp DETACH PARTITION d_listp2; }
+step "s3rename"	{ ALTER TABLE d_listp2 RENAME TO d_listp_foobar; }
+# XXX remove this step
+step "s3debugtest" { SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate, as does anybody else.
+permutation "s1b" "s1s" "s2detach" "s1s" "s1c" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2detach" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# In read-committed mode, the partition disappears from view of concurrent
+# transactions immediately.  But if a write lock is held, then the detach
+# has to wait.
+permutation "s1b"   "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1b"   "s1s" "s1ins2" "s2detach" "s1ins" "s1s" "s1c"
+
+# In repeatable-read mode, the partition remains visible until commit even
+# if the to-be-detached partition is not locked for write.
+permutation "s1brr" "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1brr" "s1s"          "s2detach"         "s1s" "s1c"
+
+# Another process trying to acquire a write lock will be blocked behind the
+# detacher
+permutation "s1b" "s1ins2" "s2detach" "s3ins2" "s1c" "s3debugtest"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec1" "s3s" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1exec2" "s2detach" "s1s" "s1exec2" "s3s" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep2" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+
+
+# Note: the following tests are not very interesting in isolationtester because
+# the way this tool handles multiple sessions: it'll only run the drop/detach/
+# rename commands after s2 is blocked waiting.  It would be more useful to
+# run the DDL when s2 commits its first transaction instead.  Maybe these should
+# become TAP tests someday.
+
+# What happens if the partition or the parent table is dropped while waiting?
+permutation "s1b" "s1s" "s2detach" "s3drop1" "s1c"
+permutation "s1b" "s1s" "s2detach" "s3drop2" "s1c"
+# Also try a non-concurrent detach while the other one is waiting
+permutation "s1b" "s1s" "s2detach" "s3detach" "s1c"
+# and some other DDL
+permutation "s1b" "s1s" "s2detach" "s3rename" "s1c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
-- 
2.20.1

#23Anastasia Lubennikova
a.lubennikova@postgrespro.ru
In reply to: Alvaro Herrera (#22)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 04.11.2020 02:56, Alvaro Herrera wrote:

Here's an updated version of this patch.

Apart from rebasing to current master, I made the following changes:

* On the first transaction (the one that marks the partition as
detached), the partition is locked with ShareLock rather than
ShareUpdateExclusiveLock. This means that DML is not allowed anymore.
This seems a reasonable restriction to me; surely by the time you're
detaching the partition you're not inserting data into it anymore.

* In ExecInitPartitionDispatchInfo, the previous version always
excluded detached partitions. Now it does include them in isolation
level repeatable read and higher. Considering the point above, this
sounds a bit contradictory: you shouldn't be inserting new tuples in
partitions being detached. But if you do, it makes more sense: in RR
two queries that insert tuples in the same partition would not fail
mid-transaction. (In a read-committed transaction, the second query
does fail, but to me that does not sound surprising.)

* ALTER TABLE .. DETACH PARTITION FINALIZE now waits for concurrent old
snapshots, as previously discussed. This should ensure that the user
doesn't just cancel the wait during the second transaction by Ctrl-C and
run FINALIZE immediately afterwards, which I claimed would bring
inconsistency.

* Avoid creating the CHECK constraint if an identical one already
exists.

(Note I do not remove the constraint on ATTACH. That seems pointless.)

Still to do: test this using the new hook added by 6f0b632f083b.

Status update for a commitfest entry.

The commitfest is nearing the end and this thread is "Waiting on Author".
As far as I see the last message contains a patch. Is there anything
left to work on or it needs review now? Alvaro, are you planning to
continue working on it?

--
Anastasia Lubennikova
Postgres Professional: http://www.postgrespro.com
The Russian Postgres Company

#24Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Anastasia Lubennikova (#23)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Nov-30, Anastasia Lubennikova wrote:

The commitfest is nearing the end and this thread is "Waiting on Author".
As far as I see the last message contains a patch. Is there anything left to
work on or it needs review now? Alvaro, are you planning to continue working
on it?

Thanks Anastasia. I marked it as needs review.

#25Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#22)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Tue, Nov 03, 2020 at 08:56:06PM -0300, Alvaro Herrera wrote:

Here's an updated version of this patch.

Apart from rebasing to current master, I made the following changes:

* On the first transaction (the one that marks the partition as
detached), the partition is locked with ShareLock rather than
ShareUpdateExclusiveLock. This means that DML is not allowed anymore.
This seems a reasonable restriction to me; surely by the time you're
detaching the partition you're not inserting data into it anymore.

I don't think it's an issue with your patch, but FYI that sounds like something
I had to do recently: detach *all* partitions of various tabls to promote their
partition key column from timestamp to timestamptz. And we insert directly
into child tables, not routed via parent.

I don't your patch is still useful, but not to us. So the documentation should
be clear about that.

* ALTER TABLE .. DETACH PARTITION FINALIZE now waits for concurrent old
snapshots, as previously discussed. This should ensure that the user
doesn't just cancel the wait during the second transaction by Ctrl-C and
run FINALIZE immediately afterwards, which I claimed would bring
inconsistency.

* Avoid creating the CHECK constraint if an identical one already
exists.

(Note I do not remove the constraint on ATTACH. That seems pointless.)

Still to do: test this using the new hook added by 6f0b632f083b.

tab complete?

+        * Ensure that foreign keys still hold after this detach.  This keeps lock
+        * on the referencing tables, which prevent concurrent transactions from

keeps locks or
which prevents

+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
attached to the target table's indexes are detached.  Any triggers that
were created as clones of those in the target table are removed.
</para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction,
+      <literal>SHARE UPDATE EXCLUSIVE</literal> is taken in both parent table and

missing "lock"
taken *on* ?

+ partition, and the partition is marked detached; at that point, the transaction

probably "its partition,"

+      If <literal>FINALIZE</literal> is specified, complete actions of a
+      previous <literal>DETACH CONCURRENTLY</literal> invocation that
+      was cancelled or crashed.

say "actions are completed" or:

If FINALIZE is specified, a previous DETACH that was cancelled or interrupted
is completed.

+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no partial concurrent detach in progress.")));

maybe say "partially-complete" or remove "partial"

+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but

after *that* ?

+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint, if there isn't a matching
+	 * one already.  The reason for this is that the planner may have made
+	 * optimizations that depend on the constraint.  XXX Isn't it sufficient to
+	 * invalidate the partition's relcache entry?

Ha. I suggested this years ago.
/messages/by-id/20180601221428.GU5164@telsasoft.com
|. The docs say: if detaching/re-attach a partition, should first ADD CHECK to
| avoid a slow ATTACH operation. Perhaps DETACHing a partition could
| implicitly CREATE a constraint which is usable when reATTACHing?

+	 * Then we close our existing transaction, and in a new one wait for
+	 * all process to catch up on the catalog updates we've done so far; at

processes

+		 * We don't need to concern ourselves with waiting for a lock the
+		 * partition itself, since we will acquire AccessExclusiveLock below.

lock *on* ?

+	 * If asked to, wait until existing snapshots are gone.  This is important
+	 * in the second transaction of DETACH PARTITION CONCURRENTLY is canceled:

s/in/if/

+++ b/src/bin/psql/describe.c
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");

The attname "detached" is a stretch of what's intuitive (it's more like
"detachING" or half-detached). But I think psql should for sure show something
more obvious to users. Esp. seeing as psql output isn't documented. Let's
figure out what to show to users and then maybe rename the column that, too.

+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)

Instead of finalize .. deferred ? Or ??

ATExecDetachPartition:
Doesn't this need to lock the table before testing for default partition ?

I ended up with apparently broken constraint when running multiple loops around
a concurrent detach / attach:

while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&
while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&

"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

#26Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#25)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi Justin,

Thanks for all the comments. I'll incorporate everything and submit an
updated version later.

On 2020-Nov-30, Justin Pryzby wrote:

On Tue, Nov 03, 2020 at 08:56:06PM -0300, Alvaro Herrera wrote:

+++ b/src/bin/psql/describe.c
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");

The attname "detached" is a stretch of what's intuitive (it's more like
"detachING" or half-detached). But I think psql should for sure show something
more obvious to users. Esp. seeing as psql output isn't documented. Let's
figure out what to show to users and then maybe rename the column that, too.

OK. I agree that "being detached" is the state we want users to see, or
maybe "detach pending", or "unfinisheddetach" (ugh). I'm not sure that
pg_inherits.inhbeingdetached" is a great column name. Opinions welcome.

+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)

Instead of finalize .. deferred ? Or ??

Well, I'm thinking that this has to be a verb in the imperative mood.
The user is commanding the server to "finalize this detach operation".
I'm not sure that DEFERRED fits that grammatical role. If there are
other ideas, let's discuss them.

ALTER TABLE tst DETACH PARTITION tst_1 FINALIZE <-- decent
ALTER TABLE tst DETACH PARTITION tst_1 COMPLETE <-- I don't like it
ALTER TABLE tst DETACH PARTITION tst_1 DEFERRED <-- grammatically faulty?

ATExecDetachPartition:
Doesn't this need to lock the table before testing for default partition ?

Correct, it does.

I ended up with apparently broken constraint when running multiple loops around
a concurrent detach / attach:

while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&
while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&

"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

Not good.

#27Andy Fan
zhihui.fan1213@gmail.com
In reply to: Alvaro Herrera (#1)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Tue, Aug 4, 2020 at 7:49 AM Alvaro Herrera <alvherre@2ndquadrant.com>
wrote:

I've been working on the ability to detach a partition from a
partitioned table, without causing blockages to concurrent activity.
I think this operation is critical for some use cases.

This would be a very great feature. When we can't handle thousands of
partitions
very well, and user agree to detach some old partitions automatically, the
blocking
issue here would be a big blocker for this solution. Thanks for working on
this!

There was a lot of great discussion which ended up in Robert completing
a much sought implementation of non-blocking ATTACH. DETACH was
discussed too because it was a goal initially, but eventually dropped
from that patch altogether. Nonetheless, that thread provided a lot of
useful input to this implementation. Important ones:

[1]
/messages/by-id/CA+TgmoYg4x7AH=_QSptvuBKf+3hUdiCa4frPkt+RvXZyjX1n=w@mail.gmail.com
[2]
/messages/by-id/CA+TgmoaAjkTibkEr=xJg3ndbRsHHSiYi2SJgX69MVosj=LJmug@mail.gmail.com
[3]
/messages/by-id/CA+TgmoY13KQZF-=HNTrt9UYWYx3_oYOQpu9ioNT49jGgiDpUEA@mail.gmail.com

Attached is a patch that implements
ALTER TABLE ... DETACH PARTITION .. CONCURRENTLY.

In the previous thread we were able to implement the concurrent model
without the extra keyword. For this one I think that won't work; my
implementation works in two transactions so there's a restriction that
you can't run it in a transaction block. Also, there's a wait phase
that makes it slower than the non-concurrent one. Those two drawbacks
make me think that it's better to keep both modes available, just like
we offer both CREATE INDEX and CREATE INDEX CONCURRENTLY.

Why two transactions? The reason is that in order for this to work, we
make a catalog change (mark it detached), and commit so that all
concurrent transactions can see the change. A second transaction waits
for anybody who holds any lock on the partitioned table and grabs Access
Exclusive on the partition (which now no one cares about, if they're
looking at the partitioned table), where the DDL action on the partition
can be completed.

ALTER TABLE is normally unable to run in two transactions. I hacked it
(0001) so that the relation can be closed and reopened in the Exec phase
(by having the rel as part of AlteredTableInfo: when ATRewriteCatalogs
returns, it uses that pointer to close the rel). It turns out that this
is sufficient to make that work. This means that ALTER TABLE DETACH
CONCURRENTLY cannot work as part of a multi-command ALTER TABLE, but
that's alreay enforced by the grammar anyway.

DETACH CONCURRENTLY doesn't work if a default partition exists. It's
just too problematic a case; you would still need to have AEL on the
default partition.

I haven't yet experimented with queries running in a standby in tandem
with a detach.

--
Álvaro Herrera

--
Best Regards
Andy Fan

#28Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#26)
2 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Dec-01, Alvaro Herrera wrote:

On 2020-Nov-30, Justin Pryzby wrote:

Thanks for all the comments. I'll incorporate everything and submit an
updated version later.

Here's a rebased version 5, with the typos fixed. More comments below.

The attname "detached" is a stretch of what's intuitive (it's more like
"detachING" or half-detached). But I think psql should for sure show something
more obvious to users. Esp. seeing as psql output isn't documented. Let's
figure out what to show to users and then maybe rename the column that, too.

OK. I agree that "being detached" is the state we want users to see, or
maybe "detach pending", or "unfinisheddetach" (ugh). I'm not sure that
pg_inherits.inhbeingdetached" is a great column name. Opinions welcome.

I haven't changed this yet; I can't make up my mind about what I like
best.

Partition of: parent FOR VALUES IN (1) UNFINISHED DETACH
Partition of: parent FOR VALUES IN (1) UNDER DETACH
Partition of: parent FOR VALUES IN (1) BEING DETACHED

ATExecDetachPartition:
Doesn't this need to lock the table before testing for default partition ?

Correct, it does.

I failed to point out that by the time ATExecDetachPartition is called,
the relation has already been locked by the invoking ALTER TABLE support
code.

I ended up with apparently broken constraint when running multiple loops around
a concurrent detach / attach:

while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&
while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&

"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

Not good.

Haven't had time to investigate this problem yet.

--
�lvaro Herrera

Attachments:

v5-0001-Let-ALTER-TABLE-exec-routines-deal-with-the-relat.patchtext/x-diff; charset=us-asciiDownload
From d21acb0e99ed3d296db70fed2d5d036d721d611b Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 13 Jul 2020 20:15:30 -0400
Subject: [PATCH v5 1/2] Let ALTER TABLE exec routines deal with the relation

This means that ATExecCmd relies on AlteredRelationInfo->rel instead of
keeping the relation as a local variable; this is useful if the
subcommand needs to modify the relation internally.  For example, a
subcommand that internally commits its transaction needs this.
---
 src/backend/commands/tablecmds.c | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 993da56d43..a8528a3423 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -156,6 +156,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -353,7 +355,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -4405,7 +4407,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4414,10 +4415,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4429,7 +4430,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4455,11 +4460,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -5562,6 +5568,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
-- 
2.20.1

v5-0002-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From 210d9a50c28b7ec85f9b08687a38b78bcce56f71 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 3 Aug 2020 19:21:09 -0400
Subject: [PATCH v5 2/2] ALTER TABLE ... DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  25 +-
 src/backend/catalog/heap.c                    |  13 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/partition.c               |  28 +-
 src/backend/catalog/pg_inherits.c             |  51 +-
 src/backend/commands/indexcmds.c              |   7 +-
 src/backend/commands/tablecmds.c              | 568 ++++++++++++++----
 src/backend/commands/trigger.c                |   5 +-
 src/backend/executor/execPartition.c          |  29 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/plancat.c          |  11 +-
 src/backend/parser/gram.y                     |  23 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/backend/utils/cache/partcache.c           |  15 +-
 src/bin/psql/describe.c                       |  40 +-
 src/include/catalog/pg_inherits.h             |   7 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 src/include/utils/snapmgr.h                   |   1 +
 .../detach-partition-concurrently-1.out       | 301 ++++++++++
 .../detach-partition-concurrently-2.out       |  66 ++
 src/test/isolation/isolation_schedule         |   2 +
 .../detach-partition-concurrently-1.spec      |  90 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 src/test/modules/delay_execution/Makefile     |   3 +-
 .../expected/partition-removal-1.out          |  79 +++
 .../specs/partition-removal-1.spec            |  45 ++
 32 files changed, 1347 insertions(+), 173 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec
 create mode 100644 src/test/modules/delay_execution/expected/partition-removal-1.out
 create mode 100644 src/test/modules/delay_execution/specs/partition-removal-1.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 3a2266526c..759dd8da5b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4482,6 +4482,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetached</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..09673dcf1c 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction, a
+      <literal>SHARE UPDATE EXCLUSIVE</literal> lock is taken on both parent table and
+      partition, and its partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, a previous
+      <literal>DETACH CONCURRENTLY</literal> invocation that was cancelled or
+      interrupted is completed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 21f2240ade..73e26fa400 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2546,10 +2546,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3818,7 +3820,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index cffbc0ac38..0dc53b25b8 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1831,7 +1831,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2425,7 +2425,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index af7754d6ab..4327a0b150 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -32,7 +32,7 @@
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
+static Oid	get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 										   List **ancestors);
 
@@ -42,6 +42,9 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
+ * If the partition is in the process of being detached, InvalidOid is
+ * returned and no error is thrown.
+ *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
@@ -51,12 +54,15 @@ get_partition_parent(Oid relid)
 {
 	Relation	catalogRelation;
 	Oid			result;
+	bool		detached;
 
 	catalogRelation = table_open(InheritsRelationId, AccessShareLock);
 
-	result = get_partition_parent_worker(catalogRelation, relid);
+	result = get_partition_parent_worker(catalogRelation, relid, &detached);
 
-	if (!OidIsValid(result))
+	if (detached)
+		result = InvalidOid;
+	else if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
 	table_close(catalogRelation, AccessShareLock);
@@ -70,13 +76,15 @@ get_partition_parent(Oid relid)
  *		given relation
  */
 static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
+get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
 {
 	SysScanDesc scan;
 	ScanKeyData key[2];
 	Oid			result = InvalidOid;
 	HeapTuple	tuple;
 
+	*detached = false;
+
 	ScanKeyInit(&key[0],
 				Anum_pg_inherits_inhrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
@@ -93,7 +101,14 @@ get_partition_parent_worker(Relation inhRel, Oid relid)
 	{
 		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
 
-		result = form->inhparent;
+		/* A partition being detached has no parent */
+		if (form->inhdetached)
+		{
+			*detached = true;
+			result = InvalidOid;
+		}
+		else
+			result = form->inhparent;
 	}
 
 	systable_endscan(scan);
@@ -134,9 +149,10 @@ static void
 get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 {
 	Oid			parentOid;
+	bool		detached;
 
 	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
+	parentOid = get_partition_parent_worker(inhRel, relid, &detached);
 	if (parentOid == InvalidOid)
 		return;
 
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index f3783961b7..3aec885288 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -199,7 +209,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -428,6 +439,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -447,10 +459,21 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * 'detached' is the expected state of the inhdetached flag.  If the catalog
+ * row does not match that state, an error is raised.  When used on
+ * partitions, the partition name must be passed, for possible error messages.
+ *
+ * If allow_detached is passed false, then an error is raised if the child is
+ * already marked detached.  A name must be supplied in that case, for the
+ * error message.  This should only be used with table partitions.
+ * XXX the name bit is pretty weird and problematic for non-partition cases;
+ * what if the flag check fails?
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -477,6 +500,28 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	inhdetached;
+
+			inhdetached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (inhdetached && !detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no pending concurrent detach.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 992f4813b4..ee4af9ea32 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -401,7 +401,7 @@ CompareOpclassOptions(Datum *opts1, Datum *opts2, int natts)
  * GetCurrentVirtualXIDs.  If, during any iteration, a particular vxid
  * doesn't show up in the output, we know we can forget about it.
  */
-static void
+void
 WaitForOlderSnapshots(TransactionId limitXmin, bool progress)
 {
 	int			n_old_snapshots;
@@ -1114,7 +1114,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1171,7 +1171,7 @@ DefineIndex(Oid relationId,
 		 *
 		 * If we're called internally (no stmt->relation), recurse always.
 		 */
-		partdesc = RelationGetPartitionDesc(rel);
+		partdesc = RelationGetPartitionDesc(rel, false);
 		if ((!stmt->relation || stmt->relation->inh) && partdesc->nparts > 0)
 		{
 			int			nparts = partdesc->nparts;
@@ -3849,6 +3849,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index a8528a3423..d7b9c63e5f 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -542,7 +542,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd,
 										   AlterTableUtilityContext *context);
@@ -551,8 +552,14 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   List *partConstraint,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
+static void DetachAddConstraintIfNeeded(List **wqueue, Relation partRel);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid, bool wait);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -987,7 +994,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3158,7 +3166,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3357,7 +3365,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -3976,7 +3984,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4367,6 +4382,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4752,7 +4772,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		case AT_AlterCollationRefreshVersion:
 			/* ATPrepCmd ensured it must be an index */
@@ -6184,7 +6209,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6417,7 +6442,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6571,7 +6597,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -6671,17 +6697,22 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	if (rel->rd_rel->relispartition)
 	{
 		Oid			parentId = get_partition_parent(RelationGetRelid(rel));
-		Relation	parent = table_open(parentId, AccessShareLock);
-		TupleDesc	tupDesc = RelationGetDescr(parent);
+		Relation	parent;
+		TupleDesc	tupDesc;
 		AttrNumber	parent_attnum;
 
-		parent_attnum = get_attnum(parentId, colName);
-		if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
-					 errmsg("column \"%s\" is marked NOT NULL in parent table",
-							colName)));
-		table_close(parent, AccessShareLock);
+		if (parentId != InvalidOid)
+		{
+			parent = table_open(parentId, AccessShareLock);
+			tupDesc = RelationGetDescr(parent);
+			parent_attnum = get_attnum(parentId, colName);
+			if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
+				ereport(ERROR,
+						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+						 errmsg("column \"%s\" is marked NOT NULL in parent table",
+								colName)));
+			table_close(parent, AccessShareLock);
+		}
 	}
 
 	/*
@@ -7280,7 +7311,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), lockmode))
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -7863,7 +7894,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8327,7 +8359,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -8942,7 +8975,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9076,7 +9109,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10860,7 +10893,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11244,7 +11278,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14139,7 +14174,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14150,12 +14185,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetached = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14169,7 +14262,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14185,7 +14278,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16293,7 +16388,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16353,7 +16448,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16942,105 +17037,230 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with share-update-exclusive
+	 * in the first transaction.  This allows concurrent transactions to be
+	 * doing DML to the partition.
+	 */
+	partRel = table_openrv(name, concurrent ? ShareLock :
+						   AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps
+	 * locks on the referencing tables, which prevents concurrent transactions
+	 * from adding rows that we wouldn't see.  For this to work in concurrent
+	 * mode, it is critical that the partition appears as no longer attached
+	 * for the RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint, if there isn't a matching
+	 * one already.  The reason for this is that the planner may have made
+	 * optimizations that depend on the constraint.  XXX Isn't it sufficient to
+	 * invalidate the partition's relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all processes to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint, if needed. XXX hopefully we can just remove this */
+		DetachAddConstraintIfNeeded(wqueue, partRel);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock on the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid, false);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid, bool wait)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	/*
+	 * If asked to, wait until existing snapshots are gone.  This is important
+	 * if the second transaction of DETACH PARTITION CONCURRENTLY is canceled:
+	 * the user could immediately run DETACH FINALIZE without actually waiting
+	 * for existing transactions.  We must not complete the detach action until
+	 * all such queries are complete (otherwise we would present them with an
+	 * inconsistent view of catalogs).
+	 */
+	if (wait)
+	{
+		Snapshot	snap = GetActiveSnapshot();
+
+		WaitForOlderSnapshots(snap->xmin, false);
+	}
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17113,22 +17333,150 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid, true);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
 }
 
+/*
+ * DetachAddConstraintIfNeeded
+ *		Subroutine for ATExecDetachPartition.  Create a constraint that
+ *		takes the place of the partition constraint, but avoid creating
+ *		a dupe if an equivalent constraint already exists.
+ */
+static void
+DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
+{
+	AlteredTableInfo *tab;
+	Expr	   *constraintExpr;
+	TupleDesc	td = RelationGetDescr(partRel);
+	Constraint *n;
+
+	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+
+	/* If an identical constraint exists, we don't need to create one */
+	if (td->constr && td->constr->num_check > 0)
+	{
+		for (int i = 0; i < td->constr->num_check; i++)
+		{
+			Node	*thisconstr;
+
+			thisconstr = stringToNode(td->constr->check[i].ccbin);
+
+			if (equal(constraintExpr, thisconstr))
+				return;
+		}
+	}
+
+	tab = ATGetQueueEntry(wqueue, partRel);
+
+	/* Add constraint on partition, equivalent to the partition constraint */
+	n = makeNode(Constraint);
+	n->contype = CONSTR_CHECK;
+	n->conname = NULL;
+	n->location = -1;
+	n->is_no_inherit = false;
+	n->raw_expr = NULL;
+	n->cooked_expr = nodeToString(constraintExpr);
+	n->initially_valid = true;
+	n->skip_validation = true;
+	/* It's a re-add, since it nominally already exists */
+	ATAddCheckConstraint(wqueue, tab, partRel, n,
+						 true, false, true, ShareUpdateExclusiveLock);
+}
+
 /*
  * DropClonedTriggersFromPartition
  *		subroutine for ATExecDetachPartition to remove any triggers that were
@@ -17324,7 +17672,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17478,7 +17826,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 12229364f1..d422d332d5 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1129,7 +1129,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1151,7 +1151,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 941731a0a9..87336d81e0 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -567,6 +567,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	rootrel = rootResultRelInfo->ri_RelationDesc,
 				partrel;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -577,7 +578,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1046,9 +1047,21 @@ ExecInitPartitionDispatchInfo(EState *estate,
 	int			dispatchidx;
 	MemoryContext oldcxt;
 
+	/*
+	 * For data modification, it is better that executor does not include
+	 * partitions being detached, except in snapshot-isolation mode.  This
+	 * means that a read-committed transaction immediately gets a "no
+	 * partition for tuple" error when a tuple is inserted into a partition
+	 * that's being detached concurrently, but a transaction in repeatable-
+	 * read mode can still use the partition.  Note that because partition
+	 * detach uses ShareLock on the partition (which conflicts with DML),
+	 * we're certain that the detach won't be able to complete until any
+	 * inserting transaction is done.
+	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt,
+									 IsolationUsesXactSnapshot());
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1628,9 +1641,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
+	/* Executor must always include detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1696,9 +1710,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ba3ccc712c..80685bdff9 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4678,6 +4678,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index a2ef853dc2..1e73308280 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2941,6 +2941,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index da322b453e..da5a8c5f5a 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2135,10 +2135,17 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 * In snapshot-isolation transactions, we always include detached
+	 * partitions, because otherwise they become invisible too soon.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 31c95443a5..4ae8caf79f 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -644,7 +644,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2094,12 +2094,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2107,8 +2108,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2123,6 +2137,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15276,6 +15291,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
@@ -15814,6 +15830,7 @@ bare_label_keyword:
 			| EXTRACT
 			| FALSE_P
 			| FAMILY
+			| FINALIZE
 			| FIRST_P
 			| FLOAT_P
 			| FOLLOWING
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index b9aeb77bc2..336cc1db9b 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2808,7 +2808,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec, ParseState *pstate)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	int			with = -1;
 	bool		overlap = false;
@@ -4017,7 +4017,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4217,7 +4217,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index f852b6e99d..58570fecfd 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -37,6 +37,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -46,7 +47,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -61,13 +62,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -88,7 +90,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -110,7 +112,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -238,6 +241,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -280,7 +284,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -295,6 +299,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -327,7 +332,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 53a511f1da..38ff1e0923 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1274,6 +1274,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index a6388d980e..74d85b89fb 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -341,6 +341,7 @@ generate_partition_qual(Relation rel)
 	bool		isnull;
 	List	   *my_qual = NIL,
 			   *result = NIL;
+	Oid			parentrelid;
 	Relation	parent;
 
 	/* Guard against stack overflow due to overly deep partition tree */
@@ -350,9 +351,19 @@ generate_partition_qual(Relation rel)
 	if (rel->rd_partcheckvalid)
 		return copyObject(rel->rd_partcheck);
 
+	/*
+	 * Obtain parent relid; if it's invalid, then the partition is being
+	 * detached
+	 */
+	parentrelid = get_partition_parent(RelationGetRelid(rel));
+	if (parentrelid == InvalidOid)
+	{
+		rel->rd_partcheckvalid = true;
+		return NIL;
+	}
+
 	/* Grab at least an AccessShareLock on the parent table */
-	parent = relation_open(get_partition_parent(RelationGetRelid(rel)),
-						   AccessShareLock);
+	parent = relation_open(parentrelid, AccessShareLock);
 
 	/* Get pg_class.relpartbound */
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index caf97563f4..883ebdc474 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2117,7 +2117,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetached" :
+						  "false as inhdetached");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2135,17 +2140,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3197,9 +3204,18 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetached,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3208,14 +3224,14 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3265,11 +3281,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACHED)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index a0f2ec4e0b..ba2a150128 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid;
 	Oid			inhparent;
 	int32		inhseqno;
+	bool		inhdetached;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -49,7 +50,8 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -57,6 +59,7 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index dc2bb40926..1a2f87f5f9 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -860,6 +860,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1870,6 +1871,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity,			/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 8c554e1f69..c79b76edca 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -163,6 +163,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD, AS_LABEL)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD, AS_LABEL)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD, BARE_LABEL)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD, BARE_LABEL)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index ff113199e5..7f03ff4271 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 579be352c5..21487b6d44 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -133,6 +133,7 @@ extern void AtEOXact_Snapshot(bool isCommit, bool resetXmin);
 extern void ImportSnapshot(const char *idstr);
 extern bool XactHasExportedSnapshots(void);
 extern void DeleteAllExportedSnapshotFiles(void);
+extern void WaitForOlderSnapshots(TransactionId limitXmin, bool progress);
 extern bool ThereAreNoPriorRegisteredSnapshots(void);
 extern bool TransactionIdLimitedForOldSnapshots(TransactionId recentXmin,
 												Relation relation,
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..8d15beac60
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,301 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2detach s1s s1c s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1s s1ins2 s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1ins s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1ins2 s2detach s3ins2 s1c s3debugtest
+step s1b: BEGIN;
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3ins2: INSERT INTO d_listp VALUES (2); <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3ins2: <... completed>
+step s3debugtest: SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2;
+tableoid       
+
+d_listp1       
+a              
+
+2              
+2              
+2              
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec1 s3s s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec1: EXECUTE f(1);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1prep s1exec2 s2detach s1s s1exec2 s3s s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1exec2: EXECUTE f(2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+2              
+step s1exec2: EXECUTE f(2);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep2 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep2: PREPARE f(int) AS INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1b s1s s2detach s3drop1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop1: DROP TABLE d_listp; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop1: <... completed>
+error in steps s1c s2detach s3drop1: ERROR:  partitioned table "d_listp" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3drop2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop2: DROP TABLE d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop2: <... completed>
+error in steps s1c s2detach s3drop2: ERROR:  partition "d_listp2" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3detach s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3detach: ALTER TABLE d_listp DETACH PARTITION d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3detach: <... completed>
+error in steps s1c s2detach s3detach: ERROR:  cannot detach partition "d_listp2"
+
+starting permutation: s1b s1s s2detach s3rename s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3rename: ALTER TABLE d_listp2 RENAME TO d_listp_foobar; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3rename: <... completed>
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index f2e752c445..3c8fa4e07d 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,8 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..1e9bea8198
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,90 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1ins2"	{ INSERT INTO d_listp VALUES (2); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1prep2"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (2); }
+step "s1exec1"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2detach"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3ins2"		{ INSERT INTO d_listp VALUES (2); }
+step "s3drop1"	{ DROP TABLE d_listp; }
+step "s3drop2"	{ DROP TABLE d_listp2; }
+step "s3detach"	{ ALTER TABLE d_listp DETACH PARTITION d_listp2; }
+step "s3rename"	{ ALTER TABLE d_listp2 RENAME TO d_listp_foobar; }
+# XXX remove this step
+step "s3debugtest" { SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate, as does anybody else.
+permutation "s1b" "s1s" "s2detach" "s1s" "s1c" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2detach" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# In read-committed mode, the partition disappears from view of concurrent
+# transactions immediately.  But if a write lock is held, then the detach
+# has to wait.
+permutation "s1b"   "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1b"   "s1s" "s1ins2" "s2detach" "s1ins" "s1s" "s1c"
+
+# In repeatable-read mode, the partition remains visible until commit even
+# if the to-be-detached partition is not locked for write.
+permutation "s1brr" "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1brr" "s1s"          "s2detach"         "s1s" "s1c"
+
+# Another process trying to acquire a write lock will be blocked behind the
+# detacher
+permutation "s1b" "s1ins2" "s2detach" "s3ins2" "s1c" "s3debugtest"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec1" "s3s" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1exec2" "s2detach" "s1s" "s1exec2" "s3s" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep2" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+
+
+# Note: the following tests are not very interesting in isolationtester because
+# the way this tool handles multiple sessions: it'll only run the drop/detach/
+# rename commands after s2 is blocked waiting.  It would be more useful to
+# run the DDL when s2 commits its first transaction instead.  Maybe these should
+# become TAP tests someday.
+
+# What happens if the partition or the parent table is dropped while waiting?
+permutation "s1b" "s1s" "s2detach" "s3drop1" "s1c"
+permutation "s1b" "s1s" "s2detach" "s3drop2" "s1c"
+# Also try a non-concurrent detach while the other one is waiting
+permutation "s1b" "s1s" "s2detach" "s3detach" "s1c"
+# and some other DDL
+permutation "s1b" "s1s" "s2detach" "s3rename" "s1c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
diff --git a/src/test/modules/delay_execution/Makefile b/src/test/modules/delay_execution/Makefile
index f270aebf3a..70f24e846d 100644
--- a/src/test/modules/delay_execution/Makefile
+++ b/src/test/modules/delay_execution/Makefile
@@ -7,7 +7,8 @@ OBJS = \
 	$(WIN32RES) \
 	delay_execution.o
 
-ISOLATION = partition-addition
+ISOLATION = partition-addition \
+	    partition-removal-1
 
 ifdef USE_PGXS
 PG_CONFIG = pg_config
diff --git a/src/test/modules/delay_execution/expected/partition-removal-1.out b/src/test/modules/delay_execution/expected/partition-removal-1.out
new file mode 100644
index 0000000000..00b93ecf65
--- /dev/null
+++ b/src/test/modules/delay_execution/expected/partition-removal-1.out
@@ -0,0 +1,79 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s3lock s1b s1exec s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1b s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
diff --git a/src/test/modules/delay_execution/specs/partition-removal-1.spec b/src/test/modules/delay_execution/specs/partition-removal-1.spec
new file mode 100644
index 0000000000..e8e99f15bf
--- /dev/null
+++ b/src/test/modules/delay_execution/specs/partition-removal-1.spec
@@ -0,0 +1,45 @@
+# Test removal of a partition with less-than-exclusive locking.
+
+setup
+{
+  CREATE TABLE partrem (a int, b text) PARTITION BY LIST(a);
+  CREATE TABLE partrem1 PARTITION OF partrem FOR VALUES IN (1);
+  CREATE TABLE partrem2 PARTITION OF partrem FOR VALUES IN (2);
+  CREATE TABLE partrem3 PARTITION OF partrem FOR VALUES IN (3);
+  INSERT INTO partrem VALUES (1, 'ABC');
+  INSERT INTO partrem VALUES (2, 'JKL');
+  INSERT INTO partrem VALUES (3, 'DEF');
+}
+
+teardown
+{
+  DROP TABLE IF EXISTS partrem, partrem2;
+}
+
+# The SELECT will be planned with all three partitions shown above,
+# of which we expect partrem1 to be pruned at planning and partrem3 at
+# execution. Then we'll block, and by the time the query is actually
+# executed, detach of partrem2 is already underway; however we expect
+# its rows to still appear in the result.
+
+session "s1"
+setup		{ LOAD 'delay_execution';
+		  SET delay_execution.post_planning_lock_id = 12543; }
+step "s1b"	{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1exec"	{ SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); }
+step "s1exec2"	{ SELECT * FROM partrem WHERE a <> (SELECT 2); }
+step "s1c"	{ COMMIT; }
+
+session "s2"
+step "s2remp"	{ ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; }
+
+session "s3"
+step "s3lock"	{ SELECT pg_advisory_lock(12543); }
+step "s3unlock"	{ SELECT pg_advisory_unlock(12543); }
+
+permutation "s3lock" "s1b" "s1exec" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec" "s2remp" "s3unlock" "s1c"
+
+permutation "s3lock" "s1b" "s1exec2" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec2" "s2remp" "s3unlock" "s1c"
-- 
2.20.1

#29Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#28)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Fri, Jan 08, 2021 at 04:14:33PM -0300, Alvaro Herrera wrote:

I ended up with apparently broken constraint when running multiple loops around
a concurrent detach / attach:

while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&
while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&

"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

Not good.

Haven't had time to investigate this problem yet.

I guess it's because you commited the txn and released lock in the middle of
the command.

--
Justin

Attachments:

0001-fix.notapxtchtext/x-diff; charset=us-asciiDownload
From e18c11fd5bcc4f5cd981a3219383265b55974f34 Mon Sep 17 00:00:00 2001
From: Justin Pryzby <pryzbyj@telsasoft.com>
Date: Sun, 10 Jan 2021 15:41:43 -0600
Subject: [PATCH] fix

---
 src/backend/commands/tablecmds.c | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index d7b9c63e5f..144c27c303 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -17131,6 +17131,7 @@ ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
 		Oid		partrelid,
 				parentrelid;
 		LOCKTAG		tag;
+		LockRelId	partlockrelid;
 		char   *parentrelname;
 		char   *partrelname;
 
@@ -17162,6 +17163,10 @@ ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
 		table_close(rel, NoLock);
 		tab->rel = NULL;
 
+		partlockrelid.relId = parentrelid;
+		partlockrelid.dbId = MyDatabaseId;
+		LockRelationIdForSession(&partlockrelid, ShareUpdateExclusiveLock);
+
 		/* Make updated catalog entry visible */
 		PopActiveSnapshot();
 		CommitTransactionCommand();
@@ -17204,7 +17209,7 @@ ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
 					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
 
 		tab->rel = rel;
-
+		UnlockRelationIdForSession(&partlockrelid, ShareUpdateExclusiveLock);
 	}
 
 	/* Do the final part of detaching */
@@ -17444,7 +17449,10 @@ DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
 	TupleDesc	td = RelationGetDescr(partRel);
 	Constraint *n;
 
-	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+	List *l = RelationGetPartitionQual(partRel);
+	Assert(partRel->rd_rel->relispartition);
+	Assert(l != NIL);
+	constraintExpr = make_ands_explicit(l);
 
 	/* If an identical constraint exists, we don't need to create one */
 	if (td->constr && td->constr->num_check > 0)
-- 
2.17.0

#30Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#29)
2 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Jan-10, Justin Pryzby wrote:

On Fri, Jan 08, 2021 at 04:14:33PM -0300, Alvaro Herrera wrote:

I ended up with apparently broken constraint when running multiple loops around
a concurrent detach / attach:

while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&
while psql -h /tmp postgres -c "ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2)" -c "ALTER TABLE p DETACH PARTITION p1 CONCURRENTLY"; do :; done&

"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

Not good.

Haven't had time to investigate this problem yet.

I guess it's because you commited the txn and released lock in the middle of
the command.

Hmm, but if we take this approach, then we're still vulnerable to the
problem that somebody can do DETACH CONCURRENTLY and cancel the wait (or
crash the server), then mess up the state before doing DETACH FINALIZE:
when they cancel the wait, the lock will be released.

I think the right fix is to disallow any action on a partition which is
pending detach other than DETACH FINALIZE. (Didn't do that here.)

Here's a rebase to current sources; there are no changes from v5.

--
�lvaro Herrera Valdivia, Chile
"No hay ausente sin culpa ni presente sin disculpa" (Prov. franc�s)

Attachments:

v6-0001-Let-ALTER-TABLE-exec-routines-deal-with-the-relat.patchtext/x-diff; charset=us-asciiDownload
From d042b99ad3368ba0b658ac9260450ed8e39accea Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 13 Jul 2020 20:15:30 -0400
Subject: [PATCH v6 1/2] Let ALTER TABLE exec routines deal with the relation

This means that ATExecCmd relies on AlteredRelationInfo->rel instead of
keeping the relation as a local variable; this is useful if the
subcommand needs to modify the relation internally.  For example, a
subcommand that internally commits its transaction needs this.
---
 src/backend/commands/tablecmds.c | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index b2457a6924..b990063d38 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -156,6 +156,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -353,7 +355,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -4513,7 +4515,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4522,10 +4523,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4537,7 +4538,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4563,11 +4568,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -5670,6 +5676,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
-- 
2.20.1

v6-0002-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From 7cc4cc8aa0f7897a51dac3bcc3fc2b6f13b0c3f9 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 3 Aug 2020 19:21:09 -0400
Subject: [PATCH v6 2/2] ALTER TABLE ... DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  25 +-
 src/backend/catalog/heap.c                    |  13 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/partition.c               |  28 +-
 src/backend/catalog/pg_inherits.c             |  51 +-
 src/backend/commands/indexcmds.c              |   7 +-
 src/backend/commands/tablecmds.c              | 568 ++++++++++++++----
 src/backend/commands/trigger.c                |   5 +-
 src/backend/executor/execPartition.c          |  29 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/plancat.c          |  11 +-
 src/backend/parser/gram.y                     |  23 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/backend/utils/cache/partcache.c           |  15 +-
 src/bin/psql/describe.c                       |  40 +-
 src/include/catalog/pg_inherits.h             |   7 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 src/include/utils/snapmgr.h                   |   1 +
 .../detach-partition-concurrently-1.out       | 301 ++++++++++
 .../detach-partition-concurrently-2.out       |  66 ++
 src/test/isolation/isolation_schedule         |   2 +
 .../detach-partition-concurrently-1.spec      |  90 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 src/test/modules/delay_execution/Makefile     |   3 +-
 .../expected/partition-removal-1.out          |  79 +++
 .../specs/partition-removal-1.spec            |  45 ++
 32 files changed, 1347 insertions(+), 173 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec
 create mode 100644 src/test/modules/delay_execution/expected/partition-removal-1.out
 create mode 100644 src/test/modules/delay_execution/specs/partition-removal-1.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index db29905e91..95293788d4 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4485,6 +4485,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetached</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..09673dcf1c 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction, a
+      <literal>SHARE UPDATE EXCLUSIVE</literal> lock is taken on both parent table and
+      partition, and its partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, a previous
+      <literal>DETACH CONCURRENTLY</literal> invocation that was cancelled or
+      interrupted is completed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 9abc4a1f55..1a05f2a2fe 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2555,10 +2555,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3827,7 +3829,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 4ef61b5efd..28f04a0700 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1836,7 +1836,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2486,7 +2486,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index af7754d6ab..4327a0b150 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -32,7 +32,7 @@
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
+static Oid	get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 										   List **ancestors);
 
@@ -42,6 +42,9 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
+ * If the partition is in the process of being detached, InvalidOid is
+ * returned and no error is thrown.
+ *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
@@ -51,12 +54,15 @@ get_partition_parent(Oid relid)
 {
 	Relation	catalogRelation;
 	Oid			result;
+	bool		detached;
 
 	catalogRelation = table_open(InheritsRelationId, AccessShareLock);
 
-	result = get_partition_parent_worker(catalogRelation, relid);
+	result = get_partition_parent_worker(catalogRelation, relid, &detached);
 
-	if (!OidIsValid(result))
+	if (detached)
+		result = InvalidOid;
+	else if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
 	table_close(catalogRelation, AccessShareLock);
@@ -70,13 +76,15 @@ get_partition_parent(Oid relid)
  *		given relation
  */
 static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
+get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
 {
 	SysScanDesc scan;
 	ScanKeyData key[2];
 	Oid			result = InvalidOid;
 	HeapTuple	tuple;
 
+	*detached = false;
+
 	ScanKeyInit(&key[0],
 				Anum_pg_inherits_inhrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
@@ -93,7 +101,14 @@ get_partition_parent_worker(Relation inhRel, Oid relid)
 	{
 		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
 
-		result = form->inhparent;
+		/* A partition being detached has no parent */
+		if (form->inhdetached)
+		{
+			*detached = true;
+			result = InvalidOid;
+		}
+		else
+			result = form->inhparent;
 	}
 
 	systable_endscan(scan);
@@ -134,9 +149,10 @@ static void
 get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 {
 	Oid			parentOid;
+	bool		detached;
 
 	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
+	parentOid = get_partition_parent_worker(inhRel, relid, &detached);
 	if (parentOid == InvalidOid)
 		return;
 
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 5ab7902827..3fd0880ca0 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -199,7 +209,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -430,6 +441,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -449,10 +461,21 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * 'detached' is the expected state of the inhdetached flag.  If the catalog
+ * row does not match that state, an error is raised.  When used on
+ * partitions, the partition name must be passed, for possible error messages.
+ *
+ * If allow_detached is passed false, then an error is raised if the child is
+ * already marked detached.  A name must be supplied in that case, for the
+ * error message.  This should only be used with table partitions.
+ * XXX the name bit is pretty weird and problematic for non-partition cases;
+ * what if the flag check fails?
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -479,6 +502,28 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	inhdetached;
+
+			inhdetached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (inhdetached && !detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no pending concurrent detach.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 8bc652ecd3..82670bac21 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -410,7 +410,7 @@ CompareOpclassOptions(Datum *opts1, Datum *opts2, int natts)
  * GetCurrentVirtualXIDs.  If, during any iteration, a particular vxid
  * doesn't show up in the output, we know we can forget about it.
  */
-static void
+void
 WaitForOlderSnapshots(TransactionId limitXmin, bool progress)
 {
 	int			n_old_snapshots;
@@ -1123,7 +1123,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1180,7 +1180,7 @@ DefineIndex(Oid relationId,
 		 *
 		 * If we're called internally (no stmt->relation), recurse always.
 		 */
-		partdesc = RelationGetPartitionDesc(rel);
+		partdesc = RelationGetPartitionDesc(rel, false);
 		if ((!stmt->relation || stmt->relation->inh) && partdesc->nparts > 0)
 		{
 			int			nparts = partdesc->nparts;
@@ -4073,6 +4073,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index b990063d38..91ef4760f6 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -542,7 +542,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd,
 										   AlterTableUtilityContext *context);
@@ -551,8 +552,14 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   List *partConstraint,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
+static void DetachAddConstraintIfNeeded(List **wqueue, Relation partRel);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid, bool wait);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -987,7 +994,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3266,7 +3274,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3465,7 +3473,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -4084,7 +4092,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4475,6 +4490,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4860,7 +4880,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		case AT_AlterCollationRefreshVersion:
 			/* ATPrepCmd ensured it must be an index */
@@ -6292,7 +6317,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6525,7 +6550,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6679,7 +6705,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -6779,17 +6805,22 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	if (rel->rd_rel->relispartition)
 	{
 		Oid			parentId = get_partition_parent(RelationGetRelid(rel));
-		Relation	parent = table_open(parentId, AccessShareLock);
-		TupleDesc	tupDesc = RelationGetDescr(parent);
+		Relation	parent;
+		TupleDesc	tupDesc;
 		AttrNumber	parent_attnum;
 
-		parent_attnum = get_attnum(parentId, colName);
-		if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
-					 errmsg("column \"%s\" is marked NOT NULL in parent table",
-							colName)));
-		table_close(parent, AccessShareLock);
+		if (parentId != InvalidOid)
+		{
+			parent = table_open(parentId, AccessShareLock);
+			tupDesc = RelationGetDescr(parent);
+			parent_attnum = get_attnum(parentId, colName);
+			if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
+				ereport(ERROR,
+						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+						 errmsg("column \"%s\" is marked NOT NULL in parent table",
+								colName)));
+			table_close(parent, AccessShareLock);
+		}
 	}
 
 	/*
@@ -7388,7 +7419,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), lockmode))
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -7971,7 +8002,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8435,7 +8467,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -9050,7 +9083,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9184,7 +9217,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10968,7 +11001,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11352,7 +11386,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14173,7 +14208,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14184,12 +14219,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetached = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14203,7 +14296,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14219,7 +14312,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16327,7 +16422,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16387,7 +16482,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16976,105 +17071,230 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with share-update-exclusive
+	 * in the first transaction.  This allows concurrent transactions to be
+	 * doing DML to the partition.
+	 */
+	partRel = table_openrv(name, concurrent ? ShareLock :
+						   AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps
+	 * locks on the referencing tables, which prevents concurrent transactions
+	 * from adding rows that we wouldn't see.  For this to work in concurrent
+	 * mode, it is critical that the partition appears as no longer attached
+	 * for the RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint, if there isn't a matching
+	 * one already.  The reason for this is that the planner may have made
+	 * optimizations that depend on the constraint.  XXX Isn't it sufficient to
+	 * invalidate the partition's relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all processes to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint, if needed. XXX hopefully we can just remove this */
+		DetachAddConstraintIfNeeded(wqueue, partRel);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock on the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid, false);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid, bool wait)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	/*
+	 * If asked to, wait until existing snapshots are gone.  This is important
+	 * if the second transaction of DETACH PARTITION CONCURRENTLY is canceled:
+	 * the user could immediately run DETACH FINALIZE without actually waiting
+	 * for existing transactions.  We must not complete the detach action until
+	 * all such queries are complete (otherwise we would present them with an
+	 * inconsistent view of catalogs).
+	 */
+	if (wait)
+	{
+		Snapshot	snap = GetActiveSnapshot();
+
+		WaitForOlderSnapshots(snap->xmin, false);
+	}
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17147,22 +17367,150 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid, true);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
 }
 
+/*
+ * DetachAddConstraintIfNeeded
+ *		Subroutine for ATExecDetachPartition.  Create a constraint that
+ *		takes the place of the partition constraint, but avoid creating
+ *		a dupe if an equivalent constraint already exists.
+ */
+static void
+DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
+{
+	AlteredTableInfo *tab;
+	Expr	   *constraintExpr;
+	TupleDesc	td = RelationGetDescr(partRel);
+	Constraint *n;
+
+	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+
+	/* If an identical constraint exists, we don't need to create one */
+	if (td->constr && td->constr->num_check > 0)
+	{
+		for (int i = 0; i < td->constr->num_check; i++)
+		{
+			Node	*thisconstr;
+
+			thisconstr = stringToNode(td->constr->check[i].ccbin);
+
+			if (equal(constraintExpr, thisconstr))
+				return;
+		}
+	}
+
+	tab = ATGetQueueEntry(wqueue, partRel);
+
+	/* Add constraint on partition, equivalent to the partition constraint */
+	n = makeNode(Constraint);
+	n->contype = CONSTR_CHECK;
+	n->conname = NULL;
+	n->location = -1;
+	n->is_no_inherit = false;
+	n->raw_expr = NULL;
+	n->cooked_expr = nodeToString(constraintExpr);
+	n->initially_valid = true;
+	n->skip_validation = true;
+	/* It's a re-add, since it nominally already exists */
+	ATAddCheckConstraint(wqueue, tab, partRel, n,
+						 true, false, true, ShareUpdateExclusiveLock);
+}
+
 /*
  * DropClonedTriggersFromPartition
  *		subroutine for ATExecDetachPartition to remove any triggers that were
@@ -17358,7 +17706,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17512,7 +17860,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 8908847c6c..fb2cdbad6d 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1119,7 +1119,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1141,7 +1141,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index b8da4c5967..619aaffae4 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -569,6 +569,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	partrel;
 	int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -579,7 +580,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1065,9 +1066,21 @@ ExecInitPartitionDispatchInfo(EState *estate,
 	int			dispatchidx;
 	MemoryContext oldcxt;
 
+	/*
+	 * For data modification, it is better that executor does not include
+	 * partitions being detached, except in snapshot-isolation mode.  This
+	 * means that a read-committed transaction immediately gets a "no
+	 * partition for tuple" error when a tuple is inserted into a partition
+	 * that's being detached concurrently, but a transaction in repeatable-
+	 * read mode can still use the partition.  Note that because partition
+	 * detach uses ShareLock on the partition (which conflicts with DML),
+	 * we're certain that the detach won't be able to complete until any
+	 * inserting transaction is done.
+	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt,
+									 IsolationUsesXactSnapshot());
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1645,9 +1658,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
+	/* Executor must always include detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1713,9 +1727,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 65bbc18ecb..1cd2931a74 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4713,6 +4713,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index c2d73626fc..250a23b347 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2972,6 +2972,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 177e6e336a..87b21dcbdc 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2136,10 +2136,17 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 * In snapshot-isolation transactions, we always include detached
+	 * partitions, because otherwise they become invisible too soon.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index dd72a9fc3c..6d2dc11b23 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -646,7 +646,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2096,12 +2096,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2109,8 +2110,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2125,6 +2139,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15337,6 +15352,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
@@ -15877,6 +15893,7 @@ bare_label_keyword:
 			| EXTRACT
 			| FALSE_P
 			| FAMILY
+			| FINALIZE
 			| FIRST_P
 			| FLOAT_P
 			| FOLLOWING
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index e5f3482d52..2b2b1dc1ad 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2798,7 +2798,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec, ParseState *pstate)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	int			with = -1;
 	bool		overlap = false;
@@ -3990,7 +3990,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4190,7 +4190,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index f852b6e99d..58570fecfd 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -37,6 +37,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -46,7 +47,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -61,13 +62,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -88,7 +90,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -110,7 +112,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -238,6 +241,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -280,7 +284,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -295,6 +299,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -327,7 +332,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 05bb698cf4..729274b330 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1236,6 +1236,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index a6388d980e..74d85b89fb 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -341,6 +341,7 @@ generate_partition_qual(Relation rel)
 	bool		isnull;
 	List	   *my_qual = NIL,
 			   *result = NIL;
+	Oid			parentrelid;
 	Relation	parent;
 
 	/* Guard against stack overflow due to overly deep partition tree */
@@ -350,9 +351,19 @@ generate_partition_qual(Relation rel)
 	if (rel->rd_partcheckvalid)
 		return copyObject(rel->rd_partcheck);
 
+	/*
+	 * Obtain parent relid; if it's invalid, then the partition is being
+	 * detached
+	 */
+	parentrelid = get_partition_parent(RelationGetRelid(rel));
+	if (parentrelid == InvalidOid)
+	{
+		rel->rd_partcheckvalid = true;
+		return NIL;
+	}
+
 	/* Grab at least an AccessShareLock on the parent table */
-	parent = relation_open(get_partition_parent(RelationGetRelid(rel)),
-						   AccessShareLock);
+	parent = relation_open(parentrelid, AccessShareLock);
 
 	/* Get pg_class.relpartbound */
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 20af5a92b4..a8d64014f0 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2117,7 +2117,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetached" :
+						  "false as inhdetached");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2135,17 +2140,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3197,9 +3204,18 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetached,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3208,14 +3224,14 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3265,11 +3281,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACHED)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 2b71cad9a2..155596907c 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid BKI_LOOKUP(pg_class);
 	Oid			inhparent BKI_LOOKUP(pg_class);
 	int32		inhseqno;
+	bool		inhdetached;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -49,7 +50,8 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -57,6 +59,7 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 236832a2ca..f97105ed3a 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -860,6 +860,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1896,6 +1897,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity,			/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 28083aaac9..7da248a385 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -165,6 +165,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD, AS_LABEL)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD, AS_LABEL)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD, BARE_LABEL)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD, BARE_LABEL)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index ff113199e5..7f03ff4271 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 2c8b881a09..901eccec25 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -133,6 +133,7 @@ extern void AtEOXact_Snapshot(bool isCommit, bool resetXmin);
 extern void ImportSnapshot(const char *idstr);
 extern bool XactHasExportedSnapshots(void);
 extern void DeleteAllExportedSnapshotFiles(void);
+extern void WaitForOlderSnapshots(TransactionId limitXmin, bool progress);
 extern bool ThereAreNoPriorRegisteredSnapshots(void);
 extern bool TransactionIdLimitedForOldSnapshots(TransactionId recentXmin,
 												Relation relation,
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..8d15beac60
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,301 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2detach s1s s1c s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1s s1ins2 s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1ins s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1ins2 s2detach s3ins2 s1c s3debugtest
+step s1b: BEGIN;
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3ins2: INSERT INTO d_listp VALUES (2); <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3ins2: <... completed>
+step s3debugtest: SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2;
+tableoid       
+
+d_listp1       
+a              
+
+2              
+2              
+2              
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec1 s3s s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec1: EXECUTE f(1);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1prep s1exec2 s2detach s1s s1exec2 s3s s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1exec2: EXECUTE f(2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+2              
+step s1exec2: EXECUTE f(2);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep2 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep2: PREPARE f(int) AS INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1b s1s s2detach s3drop1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop1: DROP TABLE d_listp; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop1: <... completed>
+error in steps s1c s2detach s3drop1: ERROR:  partitioned table "d_listp" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3drop2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop2: DROP TABLE d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop2: <... completed>
+error in steps s1c s2detach s3drop2: ERROR:  partition "d_listp2" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3detach s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3detach: ALTER TABLE d_listp DETACH PARTITION d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3detach: <... completed>
+error in steps s1c s2detach s3detach: ERROR:  cannot detach partition "d_listp2"
+
+starting permutation: s1b s1s s2detach s3rename s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3rename: ALTER TABLE d_listp2 RENAME TO d_listp_foobar; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3rename: <... completed>
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index 5d6b79e66e..69928dc540 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,8 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..1e9bea8198
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,90 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1ins2"	{ INSERT INTO d_listp VALUES (2); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1prep2"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (2); }
+step "s1exec1"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2detach"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3ins2"		{ INSERT INTO d_listp VALUES (2); }
+step "s3drop1"	{ DROP TABLE d_listp; }
+step "s3drop2"	{ DROP TABLE d_listp2; }
+step "s3detach"	{ ALTER TABLE d_listp DETACH PARTITION d_listp2; }
+step "s3rename"	{ ALTER TABLE d_listp2 RENAME TO d_listp_foobar; }
+# XXX remove this step
+step "s3debugtest" { SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate, as does anybody else.
+permutation "s1b" "s1s" "s2detach" "s1s" "s1c" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2detach" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# In read-committed mode, the partition disappears from view of concurrent
+# transactions immediately.  But if a write lock is held, then the detach
+# has to wait.
+permutation "s1b"   "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1b"   "s1s" "s1ins2" "s2detach" "s1ins" "s1s" "s1c"
+
+# In repeatable-read mode, the partition remains visible until commit even
+# if the to-be-detached partition is not locked for write.
+permutation "s1brr" "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1brr" "s1s"          "s2detach"         "s1s" "s1c"
+
+# Another process trying to acquire a write lock will be blocked behind the
+# detacher
+permutation "s1b" "s1ins2" "s2detach" "s3ins2" "s1c" "s3debugtest"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec1" "s3s" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1exec2" "s2detach" "s1s" "s1exec2" "s3s" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep2" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+
+
+# Note: the following tests are not very interesting in isolationtester because
+# the way this tool handles multiple sessions: it'll only run the drop/detach/
+# rename commands after s2 is blocked waiting.  It would be more useful to
+# run the DDL when s2 commits its first transaction instead.  Maybe these should
+# become TAP tests someday.
+
+# What happens if the partition or the parent table is dropped while waiting?
+permutation "s1b" "s1s" "s2detach" "s3drop1" "s1c"
+permutation "s1b" "s1s" "s2detach" "s3drop2" "s1c"
+# Also try a non-concurrent detach while the other one is waiting
+permutation "s1b" "s1s" "s2detach" "s3detach" "s1c"
+# and some other DDL
+permutation "s1b" "s1s" "s2detach" "s3rename" "s1c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
diff --git a/src/test/modules/delay_execution/Makefile b/src/test/modules/delay_execution/Makefile
index f270aebf3a..70f24e846d 100644
--- a/src/test/modules/delay_execution/Makefile
+++ b/src/test/modules/delay_execution/Makefile
@@ -7,7 +7,8 @@ OBJS = \
 	$(WIN32RES) \
 	delay_execution.o
 
-ISOLATION = partition-addition
+ISOLATION = partition-addition \
+	    partition-removal-1
 
 ifdef USE_PGXS
 PG_CONFIG = pg_config
diff --git a/src/test/modules/delay_execution/expected/partition-removal-1.out b/src/test/modules/delay_execution/expected/partition-removal-1.out
new file mode 100644
index 0000000000..00b93ecf65
--- /dev/null
+++ b/src/test/modules/delay_execution/expected/partition-removal-1.out
@@ -0,0 +1,79 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s3lock s1b s1exec s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1b s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
diff --git a/src/test/modules/delay_execution/specs/partition-removal-1.spec b/src/test/modules/delay_execution/specs/partition-removal-1.spec
new file mode 100644
index 0000000000..e8e99f15bf
--- /dev/null
+++ b/src/test/modules/delay_execution/specs/partition-removal-1.spec
@@ -0,0 +1,45 @@
+# Test removal of a partition with less-than-exclusive locking.
+
+setup
+{
+  CREATE TABLE partrem (a int, b text) PARTITION BY LIST(a);
+  CREATE TABLE partrem1 PARTITION OF partrem FOR VALUES IN (1);
+  CREATE TABLE partrem2 PARTITION OF partrem FOR VALUES IN (2);
+  CREATE TABLE partrem3 PARTITION OF partrem FOR VALUES IN (3);
+  INSERT INTO partrem VALUES (1, 'ABC');
+  INSERT INTO partrem VALUES (2, 'JKL');
+  INSERT INTO partrem VALUES (3, 'DEF');
+}
+
+teardown
+{
+  DROP TABLE IF EXISTS partrem, partrem2;
+}
+
+# The SELECT will be planned with all three partitions shown above,
+# of which we expect partrem1 to be pruned at planning and partrem3 at
+# execution. Then we'll block, and by the time the query is actually
+# executed, detach of partrem2 is already underway; however we expect
+# its rows to still appear in the result.
+
+session "s1"
+setup		{ LOAD 'delay_execution';
+		  SET delay_execution.post_planning_lock_id = 12543; }
+step "s1b"	{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1exec"	{ SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); }
+step "s1exec2"	{ SELECT * FROM partrem WHERE a <> (SELECT 2); }
+step "s1c"	{ COMMIT; }
+
+session "s2"
+step "s2remp"	{ ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; }
+
+session "s3"
+step "s3lock"	{ SELECT pg_advisory_lock(12543); }
+step "s3unlock"	{ SELECT pg_advisory_unlock(12543); }
+
+permutation "s3lock" "s1b" "s1exec" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec" "s2remp" "s3unlock" "s1c"
+
+permutation "s3lock" "s1b" "s1exec2" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec2" "s2remp" "s3unlock" "s1c"
-- 
2.20.1

#31Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#30)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Rebase to current sources, to appease CF bot; no other changes.

--
�lvaro Herrera 39�49'30"S 73�17'W

Attachments:

v7-detach-concurrently.patchtext/x-diff; charset=us-asciiDownload
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index b1de6d0674..ea3ae56991 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4485,6 +4485,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetached</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..09673dcf1c 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction, a
+      <literal>SHARE UPDATE EXCLUSIVE</literal> lock is taken on both parent table and
+      partition, and its partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, a previous
+      <literal>DETACH CONCURRENTLY</literal> invocation that was cancelled or
+      interrupted is completed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 9abc4a1f55..1a05f2a2fe 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2555,10 +2555,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3827,7 +3829,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 4ef61b5efd..28f04a0700 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1836,7 +1836,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2486,7 +2486,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index af7754d6ab..4327a0b150 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -32,7 +32,7 @@
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
+static Oid	get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 										   List **ancestors);
 
@@ -42,6 +42,9 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
+ * If the partition is in the process of being detached, InvalidOid is
+ * returned and no error is thrown.
+ *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
@@ -51,12 +54,15 @@ get_partition_parent(Oid relid)
 {
 	Relation	catalogRelation;
 	Oid			result;
+	bool		detached;
 
 	catalogRelation = table_open(InheritsRelationId, AccessShareLock);
 
-	result = get_partition_parent_worker(catalogRelation, relid);
+	result = get_partition_parent_worker(catalogRelation, relid, &detached);
 
-	if (!OidIsValid(result))
+	if (detached)
+		result = InvalidOid;
+	else if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
 	table_close(catalogRelation, AccessShareLock);
@@ -70,13 +76,15 @@ get_partition_parent(Oid relid)
  *		given relation
  */
 static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
+get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
 {
 	SysScanDesc scan;
 	ScanKeyData key[2];
 	Oid			result = InvalidOid;
 	HeapTuple	tuple;
 
+	*detached = false;
+
 	ScanKeyInit(&key[0],
 				Anum_pg_inherits_inhrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
@@ -93,7 +101,14 @@ get_partition_parent_worker(Relation inhRel, Oid relid)
 	{
 		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
 
-		result = form->inhparent;
+		/* A partition being detached has no parent */
+		if (form->inhdetached)
+		{
+			*detached = true;
+			result = InvalidOid;
+		}
+		else
+			result = form->inhparent;
 	}
 
 	systable_endscan(scan);
@@ -134,9 +149,10 @@ static void
 get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 {
 	Oid			parentOid;
+	bool		detached;
 
 	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
+	parentOid = get_partition_parent_worker(inhRel, relid, &detached);
 	if (parentOid == InvalidOid)
 		return;
 
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 5ab7902827..3fd0880ca0 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -199,7 +209,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -430,6 +441,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -449,10 +461,21 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * 'detached' is the expected state of the inhdetached flag.  If the catalog
+ * row does not match that state, an error is raised.  When used on
+ * partitions, the partition name must be passed, for possible error messages.
+ *
+ * If allow_detached is passed false, then an error is raised if the child is
+ * already marked detached.  A name must be supplied in that case, for the
+ * error message.  This should only be used with table partitions.
+ * XXX the name bit is pretty weird and problematic for non-partition cases;
+ * what if the flag check fails?
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -479,6 +502,28 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	inhdetached;
+
+			inhdetached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (inhdetached && !detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!inhdetached && detached)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname),
+						 errdetail("There's no pending concurrent detach.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 8bc652ecd3..82670bac21 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -410,7 +410,7 @@ CompareOpclassOptions(Datum *opts1, Datum *opts2, int natts)
  * GetCurrentVirtualXIDs.  If, during any iteration, a particular vxid
  * doesn't show up in the output, we know we can forget about it.
  */
-static void
+void
 WaitForOlderSnapshots(TransactionId limitXmin, bool progress)
 {
 	int			n_old_snapshots;
@@ -1123,7 +1123,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1180,7 +1180,7 @@ DefineIndex(Oid relationId,
 		 *
 		 * If we're called internally (no stmt->relation), recurse always.
 		 */
-		partdesc = RelationGetPartitionDesc(rel);
+		partdesc = RelationGetPartitionDesc(rel, false);
 		if ((!stmt->relation || stmt->relation->inh) && partdesc->nparts > 0)
 		{
 			int			nparts = partdesc->nparts;
@@ -4073,6 +4073,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetached - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 559fa1d2e5..5986b4dd56 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -156,6 +156,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -353,7 +355,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -539,7 +541,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd,
 										   AlterTableUtilityContext *context);
@@ -548,8 +551,14 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   List *partConstraint,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
+static void DetachAddConstraintIfNeeded(List **wqueue, Relation partRel);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid, bool wait);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -984,7 +993,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3263,7 +3273,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3462,7 +3472,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -4081,7 +4091,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4472,6 +4489,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4512,7 +4534,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4521,10 +4542,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4536,7 +4557,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4562,11 +4587,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -4853,7 +4879,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		case AT_AlterCollationRefreshVersion:
 			/* ATPrepCmd ensured it must be an index */
@@ -5669,6 +5700,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
@@ -6284,7 +6316,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6517,7 +6549,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6671,7 +6704,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -6771,17 +6804,22 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	if (rel->rd_rel->relispartition)
 	{
 		Oid			parentId = get_partition_parent(RelationGetRelid(rel));
-		Relation	parent = table_open(parentId, AccessShareLock);
-		TupleDesc	tupDesc = RelationGetDescr(parent);
+		Relation	parent;
+		TupleDesc	tupDesc;
 		AttrNumber	parent_attnum;
 
-		parent_attnum = get_attnum(parentId, colName);
-		if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
-					 errmsg("column \"%s\" is marked NOT NULL in parent table",
-							colName)));
-		table_close(parent, AccessShareLock);
+		if (parentId != InvalidOid)
+		{
+			parent = table_open(parentId, AccessShareLock);
+			tupDesc = RelationGetDescr(parent);
+			parent_attnum = get_attnum(parentId, colName);
+			if (TupleDescAttr(tupDesc, parent_attnum - 1)->attnotnull)
+				ereport(ERROR,
+						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+						 errmsg("column \"%s\" is marked NOT NULL in parent table",
+								colName)));
+			table_close(parent, AccessShareLock);
+		}
 	}
 
 	/*
@@ -7380,7 +7418,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), lockmode))
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -7963,7 +8001,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8427,7 +8466,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -9042,7 +9082,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9176,7 +9216,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10960,7 +11000,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11344,7 +11385,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14165,7 +14207,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14176,12 +14218,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetached = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14195,7 +14295,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14211,7 +14311,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16296,7 +16398,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16356,7 +16458,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16945,105 +17047,230 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with share-update-exclusive
+	 * in the first transaction.  This allows concurrent transactions to be
+	 * doing DML to the partition.
+	 */
+	partRel = table_openrv(name, concurrent ? ShareLock :
+						   AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps
+	 * locks on the referencing tables, which prevents concurrent transactions
+	 * from adding rows that we wouldn't see.  For this to work in concurrent
+	 * mode, it is critical that the partition appears as no longer attached
+	 * for the RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint, if there isn't a matching
+	 * one already.  The reason for this is that the planner may have made
+	 * optimizations that depend on the constraint.  XXX Isn't it sufficient to
+	 * invalidate the partition's relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all processes to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint, if needed. XXX hopefully we can just remove this */
+		DetachAddConstraintIfNeeded(wqueue, partRel);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock on the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid, false);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid, bool wait)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	/*
+	 * If asked to, wait until existing snapshots are gone.  This is important
+	 * if the second transaction of DETACH PARTITION CONCURRENTLY is canceled:
+	 * the user could immediately run DETACH FINALIZE without actually waiting
+	 * for existing transactions.  We must not complete the detach action until
+	 * all such queries are complete (otherwise we would present them with an
+	 * inconsistent view of catalogs).
+	 */
+	if (wait)
+	{
+		Snapshot	snap = GetActiveSnapshot();
+
+		WaitForOlderSnapshots(snap->xmin, false);
+	}
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17116,22 +17343,150 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid, true);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
 }
 
+/*
+ * DetachAddConstraintIfNeeded
+ *		Subroutine for ATExecDetachPartition.  Create a constraint that
+ *		takes the place of the partition constraint, but avoid creating
+ *		a dupe if an equivalent constraint already exists.
+ */
+static void
+DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
+{
+	AlteredTableInfo *tab;
+	Expr	   *constraintExpr;
+	TupleDesc	td = RelationGetDescr(partRel);
+	Constraint *n;
+
+	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+
+	/* If an identical constraint exists, we don't need to create one */
+	if (td->constr && td->constr->num_check > 0)
+	{
+		for (int i = 0; i < td->constr->num_check; i++)
+		{
+			Node	*thisconstr;
+
+			thisconstr = stringToNode(td->constr->check[i].ccbin);
+
+			if (equal(constraintExpr, thisconstr))
+				return;
+		}
+	}
+
+	tab = ATGetQueueEntry(wqueue, partRel);
+
+	/* Add constraint on partition, equivalent to the partition constraint */
+	n = makeNode(Constraint);
+	n->contype = CONSTR_CHECK;
+	n->conname = NULL;
+	n->location = -1;
+	n->is_no_inherit = false;
+	n->raw_expr = NULL;
+	n->cooked_expr = nodeToString(constraintExpr);
+	n->initially_valid = true;
+	n->skip_validation = true;
+	/* It's a re-add, since it nominally already exists */
+	ATAddCheckConstraint(wqueue, tab, partRel, n,
+						 true, false, true, ShareUpdateExclusiveLock);
+}
+
 /*
  * DropClonedTriggersFromPartition
  *		subroutine for ATExecDetachPartition to remove any triggers that were
@@ -17327,7 +17682,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17481,7 +17836,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 4e4e05844c..7383d5994e 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1119,7 +1119,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1141,7 +1141,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index b8da4c5967..619aaffae4 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -569,6 +569,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	partrel;
 	int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -579,7 +580,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1065,9 +1066,21 @@ ExecInitPartitionDispatchInfo(EState *estate,
 	int			dispatchidx;
 	MemoryContext oldcxt;
 
+	/*
+	 * For data modification, it is better that executor does not include
+	 * partitions being detached, except in snapshot-isolation mode.  This
+	 * means that a read-committed transaction immediately gets a "no
+	 * partition for tuple" error when a tuple is inserted into a partition
+	 * that's being detached concurrently, but a transaction in repeatable-
+	 * read mode can still use the partition.  Note that because partition
+	 * detach uses ShareLock on the partition (which conflicts with DML),
+	 * we're certain that the detach won't be able to complete until any
+	 * inserting transaction is done.
+	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt,
+									 IsolationUsesXactSnapshot());
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1645,9 +1658,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
+	/* Executor must always include detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1713,9 +1727,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index da91cbd2b1..850e50f1a2 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4735,6 +4735,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index c2d73626fc..250a23b347 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2972,6 +2972,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 7e25f94293..830341d08e 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1192,11 +1192,15 @@ target_rel_partitions_max_parallel_hazard(Relation rel,
 
 	/* Recursively check each partition ... */
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already. Must
+	 * match args used in set_relation_partition_info.
+	 */
 	glob = context->planner_global;
 	if (glob->partition_directory == NULL)
 		glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
 
 	pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
 
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index c5947fa418..9ec7d7f518 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2142,10 +2142,17 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 * In snapshot-isolation transactions, we always include detached
+	 * partitions, because otherwise they become invisible too soon.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 652be0b96d..b26d0a3d0b 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -646,7 +646,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2096,12 +2096,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2109,8 +2110,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2125,6 +2139,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15348,6 +15363,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
@@ -15888,6 +15904,7 @@ bare_label_keyword:
 			| EXTRACT
 			| FALSE_P
 			| FAMILY
+			| FINALIZE
 			| FIRST_P
 			| FLOAT_P
 			| FOLLOWING
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index e5f3482d52..2b2b1dc1ad 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2798,7 +2798,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec, ParseState *pstate)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	int			with = -1;
 	bool		overlap = false;
@@ -3990,7 +3990,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4190,7 +4190,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index f852b6e99d..58570fecfd 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -37,6 +37,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -46,7 +47,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -61,13 +62,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -88,7 +90,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -110,7 +112,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -238,6 +241,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -280,7 +284,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -295,6 +299,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -327,7 +332,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 05bb698cf4..729274b330 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1236,6 +1236,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index a6388d980e..74d85b89fb 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -341,6 +341,7 @@ generate_partition_qual(Relation rel)
 	bool		isnull;
 	List	   *my_qual = NIL,
 			   *result = NIL;
+	Oid			parentrelid;
 	Relation	parent;
 
 	/* Guard against stack overflow due to overly deep partition tree */
@@ -350,9 +351,19 @@ generate_partition_qual(Relation rel)
 	if (rel->rd_partcheckvalid)
 		return copyObject(rel->rd_partcheck);
 
+	/*
+	 * Obtain parent relid; if it's invalid, then the partition is being
+	 * detached
+	 */
+	parentrelid = get_partition_parent(RelationGetRelid(rel));
+	if (parentrelid == InvalidOid)
+	{
+		rel->rd_partcheckvalid = true;
+		return NIL;
+	}
+
 	/* Grab at least an AccessShareLock on the parent table */
-	parent = relation_open(get_partition_parent(RelationGetRelid(rel)),
-						   AccessShareLock);
+	parent = relation_open(parentrelid, AccessShareLock);
 
 	/* Get pg_class.relpartbound */
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 20af5a92b4..a8d64014f0 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2117,7 +2117,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetached" :
+						  "false as inhdetached");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2135,17 +2140,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACHED" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3197,9 +3204,18 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetached,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3208,14 +3224,14 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind, false AS inhdetached, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3265,11 +3281,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACHED)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 2b71cad9a2..155596907c 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid BKI_LOOKUP(pg_class);
 	Oid			inhparent BKI_LOOKUP(pg_class);
 	int32		inhseqno;
+	bool		inhdetached;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -49,7 +50,8 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -57,6 +59,7 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 236832a2ca..f97105ed3a 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -860,6 +860,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1896,6 +1897,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity,			/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 28083aaac9..7da248a385 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -165,6 +165,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD, AS_LABEL)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD, AS_LABEL)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD, BARE_LABEL)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD, BARE_LABEL)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index ff113199e5..7f03ff4271 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 2c8b881a09..901eccec25 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -133,6 +133,7 @@ extern void AtEOXact_Snapshot(bool isCommit, bool resetXmin);
 extern void ImportSnapshot(const char *idstr);
 extern bool XactHasExportedSnapshots(void);
 extern void DeleteAllExportedSnapshotFiles(void);
+extern void WaitForOlderSnapshots(TransactionId limitXmin, bool progress);
 extern bool ThereAreNoPriorRegisteredSnapshots(void);
 extern bool TransactionIdLimitedForOldSnapshots(TransactionId recentXmin,
 												Relation relation,
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..8d15beac60
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,301 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2detach s1s s1c s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1s s1ins2 s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1ins s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1ins2 s2detach s3ins2 s1c s3debugtest
+step s1b: BEGIN;
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3ins2: INSERT INTO d_listp VALUES (2); <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3ins2: <... completed>
+step s3debugtest: SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2;
+tableoid       
+
+d_listp1       
+a              
+
+2              
+2              
+2              
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec1 s3s s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec1: EXECUTE f(1);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1prep s1exec2 s2detach s1s s1exec2 s3s s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1exec2: EXECUTE f(2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+2              
+step s1exec2: EXECUTE f(2);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep2 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep2: PREPARE f(int) AS INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1b s1s s2detach s3drop1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop1: DROP TABLE d_listp; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop1: <... completed>
+error in steps s1c s2detach s3drop1: ERROR:  partitioned table "d_listp" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3drop2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3drop2: DROP TABLE d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3drop2: <... completed>
+error in steps s1c s2detach s3drop2: ERROR:  partition "d_listp2" was removed concurrently
+
+starting permutation: s1b s1s s2detach s3detach s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3detach: ALTER TABLE d_listp DETACH PARTITION d_listp2; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3detach: <... completed>
+error in steps s1c s2detach s3detach: ERROR:  cannot detach partition "d_listp2"
+
+starting permutation: s1b s1s s2detach s3rename s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3rename: ALTER TABLE d_listp2 RENAME TO d_listp_foobar; <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3rename: <... completed>
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index 5d6b79e66e..69928dc540 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,8 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..1e9bea8198
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,90 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1ins2"	{ INSERT INTO d_listp VALUES (2); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1prep2"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (2); }
+step "s1exec1"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2detach"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3ins2"		{ INSERT INTO d_listp VALUES (2); }
+step "s3drop1"	{ DROP TABLE d_listp; }
+step "s3drop2"	{ DROP TABLE d_listp2; }
+step "s3detach"	{ ALTER TABLE d_listp DETACH PARTITION d_listp2; }
+step "s3rename"	{ ALTER TABLE d_listp2 RENAME TO d_listp_foobar; }
+# XXX remove this step
+step "s3debugtest" { SELECT tableoid::regclass FROM d_listp; SELECT * from d_listp2; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate, as does anybody else.
+permutation "s1b" "s1s" "s2detach" "s1s" "s1c" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2detach" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# In read-committed mode, the partition disappears from view of concurrent
+# transactions immediately.  But if a write lock is held, then the detach
+# has to wait.
+permutation "s1b"   "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1b"   "s1s" "s1ins2" "s2detach" "s1ins" "s1s" "s1c"
+
+# In repeatable-read mode, the partition remains visible until commit even
+# if the to-be-detached partition is not locked for write.
+permutation "s1brr" "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1brr" "s1s"          "s2detach"         "s1s" "s1c"
+
+# Another process trying to acquire a write lock will be blocked behind the
+# detacher
+permutation "s1b" "s1ins2" "s2detach" "s3ins2" "s1c" "s3debugtest"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec1" "s3s" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1exec2" "s2detach" "s1s" "s1exec2" "s3s" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep2" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+
+
+# Note: the following tests are not very interesting in isolationtester because
+# the way this tool handles multiple sessions: it'll only run the drop/detach/
+# rename commands after s2 is blocked waiting.  It would be more useful to
+# run the DDL when s2 commits its first transaction instead.  Maybe these should
+# become TAP tests someday.
+
+# What happens if the partition or the parent table is dropped while waiting?
+permutation "s1b" "s1s" "s2detach" "s3drop1" "s1c"
+permutation "s1b" "s1s" "s2detach" "s3drop2" "s1c"
+# Also try a non-concurrent detach while the other one is waiting
+permutation "s1b" "s1s" "s2detach" "s3detach" "s1c"
+# and some other DDL
+permutation "s1b" "s1s" "s2detach" "s3rename" "s1c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
diff --git a/src/test/modules/delay_execution/Makefile b/src/test/modules/delay_execution/Makefile
index f270aebf3a..70f24e846d 100644
--- a/src/test/modules/delay_execution/Makefile
+++ b/src/test/modules/delay_execution/Makefile
@@ -7,7 +7,8 @@ OBJS = \
 	$(WIN32RES) \
 	delay_execution.o
 
-ISOLATION = partition-addition
+ISOLATION = partition-addition \
+	    partition-removal-1
 
 ifdef USE_PGXS
 PG_CONFIG = pg_config
diff --git a/src/test/modules/delay_execution/expected/partition-removal-1.out b/src/test/modules/delay_execution/expected/partition-removal-1.out
new file mode 100644
index 0000000000..00b93ecf65
--- /dev/null
+++ b/src/test/modules/delay_execution/expected/partition-removal-1.out
@@ -0,0 +1,79 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s3lock s1b s1exec s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1b s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
diff --git a/src/test/modules/delay_execution/specs/partition-removal-1.spec b/src/test/modules/delay_execution/specs/partition-removal-1.spec
new file mode 100644
index 0000000000..e8e99f15bf
--- /dev/null
+++ b/src/test/modules/delay_execution/specs/partition-removal-1.spec
@@ -0,0 +1,45 @@
+# Test removal of a partition with less-than-exclusive locking.
+
+setup
+{
+  CREATE TABLE partrem (a int, b text) PARTITION BY LIST(a);
+  CREATE TABLE partrem1 PARTITION OF partrem FOR VALUES IN (1);
+  CREATE TABLE partrem2 PARTITION OF partrem FOR VALUES IN (2);
+  CREATE TABLE partrem3 PARTITION OF partrem FOR VALUES IN (3);
+  INSERT INTO partrem VALUES (1, 'ABC');
+  INSERT INTO partrem VALUES (2, 'JKL');
+  INSERT INTO partrem VALUES (3, 'DEF');
+}
+
+teardown
+{
+  DROP TABLE IF EXISTS partrem, partrem2;
+}
+
+# The SELECT will be planned with all three partitions shown above,
+# of which we expect partrem1 to be pruned at planning and partrem3 at
+# execution. Then we'll block, and by the time the query is actually
+# executed, detach of partrem2 is already underway; however we expect
+# its rows to still appear in the result.
+
+session "s1"
+setup		{ LOAD 'delay_execution';
+		  SET delay_execution.post_planning_lock_id = 12543; }
+step "s1b"	{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1exec"	{ SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); }
+step "s1exec2"	{ SELECT * FROM partrem WHERE a <> (SELECT 2); }
+step "s1c"	{ COMMIT; }
+
+session "s2"
+step "s2remp"	{ ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; }
+
+session "s3"
+step "s3lock"	{ SELECT pg_advisory_lock(12543); }
+step "s3unlock"	{ SELECT pg_advisory_unlock(12543); }
+
+permutation "s3lock" "s1b" "s1exec" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec" "s2remp" "s3unlock" "s1c"
+
+permutation "s3lock" "s1b" "s1exec2" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec2" "s2remp" "s3unlock" "s1c"
#32Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#30)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Feb-26, Alvaro Herrera wrote:

Hmm, but if we take this approach, then we're still vulnerable to the
problem that somebody can do DETACH CONCURRENTLY and cancel the wait (or
crash the server), then mess up the state before doing DETACH FINALIZE:
when they cancel the wait, the lock will be released.

I think the right fix is to disallow any action on a partition which is
pending detach other than DETACH FINALIZE. (Didn't do that here.)

Here's a fixup patch to do it that way. I tried running the commands
you showed and one of them immediately dies with the new error message;
I can't cause the bogus constraint to show up anymore.

I'll clean this up for a real submission tomorrow.

--
�lvaro Herrera 39�49'30"S 73�17'W
"The Gord often wonders why people threaten never to come back after they've
been told never to return" (www.actsofgord.com)

Attachments:

dont.notpatchtext/plain; charset=us-asciiDownload
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 3fd0880ca0..83093f9730 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -535,3 +535,39 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool detached,
 
 	return found;
 }
+
+bool
+PartitionHasPendingDetach(Oid partoid)
+{
+	Relation	catalogRelation;
+	ScanKeyData key;
+	SysScanDesc scan;
+	HeapTuple	inheritsTuple;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(partoid));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		bool	detached;
+
+		detached = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetached;
+
+		/* Done */
+		systable_endscan(scan);
+		table_close(catalogRelation, RowExclusiveLock);
+
+		return detached;
+	}
+
+	elog(ERROR, "relation %u is not a partition", partoid);
+	return false;				/* keep compiler quiet */
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 5986b4dd56..5074c0ff2b 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -17106,6 +17106,8 @@ ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	partRel = table_openrv(name, concurrent ? ShareLock :
 						   AccessExclusiveLock);
+	if (PartitionHasPendingDetach(RelationGetRelid(partRel)))
+		elog(ERROR, "hey, don't!");
 
 	/*
 	 * Check inheritance conditions and either delete the pg_inherits row
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 155596907c..19243ea9e4 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -61,5 +61,6 @@ extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
 extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
 								const char *childname);
+extern bool PartitionHasPendingDetach(Oid partoid);
 
 #endif							/* PG_INHERITS_H */
#33Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#32)
2 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-15, Alvaro Herrera wrote:

Here's a fixup patch to do it that way. I tried running the commands
you showed and one of them immediately dies with the new error message;
I can't cause the bogus constraint to show up anymore.

Actually, that was a silly fix that didn't actually work correctly, as I
discovered immediately after sending it. The right fix is to forbid all
commands other than DETACH PARTITION FINALIZE in a partition that's in
the process of being detached.

In the attached v8, I did that; I also added a ton more tests that
hopefully show how the feature should work in concurrent cases,
including one case in which the transaction doing the detach is
cancelled. I also renamed "inhdetached" to "inhdetachpending", per
previous discussion, including changing how to looks in psql.

I am not aware of any other loose end in this patch; I consider this
version final. Barring further problem reports, I'll get this pushed
tomorrow morning.

psql completion is missing. If somebody would like to contribute that,
I'm grateful.

--
�lvaro Herrera 39�49'30"S 73�17'W
"If you have nothing to say, maybe you need just the right tool to help you
not say it." (New York Times, about Microsoft PowerPoint)

Attachments:

v8-0001-Let-ALTER-TABLE-exec-routines-deal-with-the-relat.patchtext/x-diff; charset=us-asciiDownload
From 101a20d1ccf86a255af166e198c7ac691af58c99 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 13 Jul 2020 20:15:30 -0400
Subject: [PATCH v8 1/2] Let ALTER TABLE exec routines deal with the relation

This means that ATExecCmd relies on AlteredRelationInfo->rel instead of
keeping the relation as a local variable; this is useful if the
subcommand needs to modify the relation internally.  For example, a
subcommand that internally commits its transaction needs this.
---
 src/backend/commands/tablecmds.c | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index ffb1308a0c..8e753f1efd 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -156,6 +156,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -353,7 +355,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -4512,7 +4514,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4521,10 +4522,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4536,7 +4537,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4562,11 +4567,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -5669,6 +5675,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
-- 
2.20.1

v8-0002-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From 6e893b5f2e7abae9e55ba0d1487d4621272d2cdf Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 3 Aug 2020 19:21:09 -0400
Subject: [PATCH v8 2/2] ALTER TABLE ... DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  25 +-
 src/backend/catalog/heap.c                    |  13 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/partition.c               |  31 +-
 src/backend/catalog/pg_inherits.c             |  91 ++-
 src/backend/commands/indexcmds.c              |   7 +-
 src/backend/commands/tablecmds.c              | 558 ++++++++++++++----
 src/backend/commands/trigger.c                |   5 +-
 src/backend/executor/execPartition.c          |  29 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/clauses.c          |   8 +-
 src/backend/optimizer/util/plancat.c          |  11 +-
 src/backend/parser/gram.y                     |  23 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/backend/utils/cache/partcache.c           |  16 +-
 src/bin/psql/describe.c                       |  42 +-
 src/include/catalog/pg_inherits.h             |   8 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 src/include/utils/snapmgr.h                   |   1 +
 .../detach-partition-concurrently-1.out       | 237 ++++++++
 .../detach-partition-concurrently-2.out       |  66 +++
 .../detach-partition-concurrently-3.out       | 211 +++++++
 src/test/isolation/isolation_schedule         |   3 +
 .../detach-partition-concurrently-1.spec      |  69 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 .../detach-partition-concurrently-3.spec      |  50 ++
 src/test/modules/delay_execution/Makefile     |   3 +-
 .../expected/partition-removal-1.out          | 175 ++++++
 .../specs/partition-removal-1.spec            |  58 ++
 src/test/regress/expected/alter_table.out     |  29 +
 src/test/regress/sql/alter_table.sql          |  20 +
 37 files changed, 1735 insertions(+), 165 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-3.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-3.spec
 create mode 100644 src/test/modules/delay_execution/expected/partition-removal-1.out
 create mode 100644 src/test/modules/delay_execution/specs/partition-removal-1.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 5c9f4af1d5..0cb846f408 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4485,6 +4485,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetachpending</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..09673dcf1c 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -938,7 +940,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -947,6 +950,24 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
      </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction, a
+      <literal>SHARE UPDATE EXCLUSIVE</literal> lock is taken on both parent table and
+      partition, and its partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, a previous
+      <literal>DETACH CONCURRENTLY</literal> invocation that was cancelled or
+      interrupted is completed.
+     </para>
     </listitem>
    </varlistentry>
 
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 9abc4a1f55..1a05f2a2fe 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -2555,10 +2555,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3827,7 +3829,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 4ef61b5efd..28f04a0700 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1836,7 +1836,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2486,7 +2486,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index af7754d6ab..797a7384ad 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -32,7 +32,7 @@
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
+static Oid	get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 										   List **ancestors);
 
@@ -42,6 +42,9 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
+ * If the partition is in the process of being detached, InvalidOid is
+ * returned and no error is thrown.
+ *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
@@ -51,12 +54,15 @@ get_partition_parent(Oid relid)
 {
 	Relation	catalogRelation;
 	Oid			result;
+	bool		detached;
 
 	catalogRelation = table_open(InheritsRelationId, AccessShareLock);
 
-	result = get_partition_parent_worker(catalogRelation, relid);
+	result = get_partition_parent_worker(catalogRelation, relid, &detached);
 
-	if (!OidIsValid(result))
+	if (detached)
+		result = InvalidOid;
+	else if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
 	table_close(catalogRelation, AccessShareLock);
@@ -68,15 +74,20 @@ get_partition_parent(Oid relid)
  * get_partition_parent_worker
  *		Scan the pg_inherits relation to return the OID of the parent of the
  *		given relation
+ *
+ * If the partition is being detached, InvalidOid is returned, and also
+ * *detached is set true.
  */
 static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
+get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
 {
 	SysScanDesc scan;
 	ScanKeyData key[2];
 	Oid			result = InvalidOid;
 	HeapTuple	tuple;
 
+	*detached = false;
+
 	ScanKeyInit(&key[0],
 				Anum_pg_inherits_inhrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
@@ -93,7 +104,14 @@ get_partition_parent_worker(Relation inhRel, Oid relid)
 	{
 		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
 
-		result = form->inhparent;
+		/* A partition being detached has no parent */
+		if (form->inhdetachpending)
+		{
+			*detached = true;
+			result = InvalidOid;
+		}
+		else
+			result = form->inhparent;
 	}
 
 	systable_endscan(scan);
@@ -134,9 +152,10 @@ static void
 get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 {
 	Oid			parentOid;
+	bool		detached;
 
 	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
+	parentOid = get_partition_parent_worker(inhRel, relid, &detached);
 	if (parentOid == InvalidOid)
 		return;
 
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 5ab7902827..f62720c647 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,7 +52,8 @@ typedef struct SeenRelsEntry
  * against possible DROPs of child relations.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,7 +92,13 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/* skip detached at caller's request */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending &&
+			!include_detached)
+			continue;
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
+
 		if (numoids >= maxoids)
 		{
 			maxoids *= 2;
@@ -160,6 +167,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -199,7 +209,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -430,6 +441,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetachpending - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -449,10 +461,17 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * expect_detach_pending is the expected state of the inhdetachpending flag.
+ * If the catalog row does not match that state, an error is raised.
+ *
+ * childname is the partition name, if a table; pass NULL for regular
+ * inheritance or when working with other relation kinds.
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool expect_detach_pending,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -479,6 +498,29 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	detach_pending;
+
+			detach_pending =
+				((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (detach_pending && !expect_detach_pending)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname ? childname : "unknown relation"),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!detach_pending && expect_detach_pending)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname ? childname : "unknown relation"),
+						 errdetail("There's no pending concurrent detach.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
@@ -490,3 +532,46 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 
 	return found;
 }
+
+/*
+ * Return whether the pg_inherits tuple for a partition has the "detach
+ * pending" flag set.
+ */
+bool
+PartitionHasPendingDetach(Oid partoid)
+{
+	Relation	catalogRelation;
+	ScanKeyData key;
+	SysScanDesc scan;
+	HeapTuple	inheritsTuple;
+
+	/* We don't have a good way to verify it is in fact a partition */
+
+	/*
+	 * Find the pg_inherits entry by inhrelid.  (There should only be one.)
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(partoid));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		bool	detached;
+
+		detached =
+			((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending;
+
+		/* Done */
+		systable_endscan(scan);
+		table_close(catalogRelation, RowExclusiveLock);
+
+		return detached;
+	}
+
+	elog(ERROR, "relation %u is not a partition", partoid);
+	return false;				/* keep compiler quiet */
+}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 8bc652ecd3..676edee233 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -410,7 +410,7 @@ CompareOpclassOptions(Datum *opts1, Datum *opts2, int natts)
  * GetCurrentVirtualXIDs.  If, during any iteration, a particular vxid
  * doesn't show up in the output, we know we can forget about it.
  */
-static void
+void
 WaitForOlderSnapshots(TransactionId limitXmin, bool progress)
 {
 	int			n_old_snapshots;
@@ -1123,7 +1123,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1180,7 +1180,7 @@ DefineIndex(Oid relationId,
 		 *
 		 * If we're called internally (no stmt->relation), recurse always.
 		 */
-		partdesc = RelationGetPartitionDesc(rel);
+		partdesc = RelationGetPartitionDesc(rel, false);
 		if ((!stmt->relation || stmt->relation->inh) && partdesc->nparts > 0)
 		{
 			int			nparts = partdesc->nparts;
@@ -4073,6 +4073,7 @@ IndexSetParentIndex(Relation partitionIdx, Oid parentOid)
 			values[Anum_pg_inherits_inhparent - 1] =
 				ObjectIdGetDatum(parentOid);
 			values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(1);
+			values[Anum_pg_inherits_inhdetachpending - 1] = BoolGetDatum(false);
 			memset(isnull, false, sizeof(isnull));
 
 			tuple = heap_form_tuple(RelationGetDescr(pg_inherits),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 8e753f1efd..81e866c1eb 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -541,7 +541,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd,
 										   AlterTableUtilityContext *context);
@@ -550,8 +551,14 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   List *partConstraint,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
+static void DetachAddConstraintIfNeeded(List **wqueue, Relation partRel);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid, bool wait);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -986,7 +993,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -3265,7 +3273,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3464,7 +3472,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -4083,7 +4091,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4167,6 +4182,19 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 	/* Find or create work queue entry for this table */
 	tab = ATGetQueueEntry(wqueue, rel);
 
+	/*
+	 * Disallow any ALTER TABLE other than ALTER TABLE DETACH FINALIZE on
+	 * partitions that are pending detach.
+	 */
+	if (rel->rd_rel->relispartition &&
+		cmd->subtype != AT_DetachPartitionFinalize &&
+		PartitionHasPendingDetach(RelationGetRelid(rel)))
+		ereport(ERROR,
+				errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				errmsg("cannot alter partition \"%s\" with an incomplete detach",
+					   RelationGetRelationName(rel)),
+				errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation."));
+
 	/*
 	 * Copy the original subcommand for each table.  This avoids conflicts
 	 * when different child tables need to make different parse
@@ -4474,6 +4502,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4859,7 +4892,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		case AT_AlterCollationRefreshVersion:
 			/* ATPrepCmd ensured it must be an index */
@@ -6291,7 +6329,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6524,7 +6562,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6678,7 +6717,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -7387,7 +7426,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), lockmode))
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -7970,7 +8009,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8434,7 +8474,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -9049,7 +9090,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9183,7 +9224,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -10967,7 +11008,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11351,7 +11393,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14172,7 +14215,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14183,12 +14226,70 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other
+ * transactions know to ignore the partition.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14202,7 +14303,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14218,7 +14319,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16303,7 +16406,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16363,7 +16466,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -16952,105 +17055,230 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,
+ * lock the partition against writes, and commit the transaction; anyone
+ * who requests the partition descriptor from that point onwards has to
+ * ignore such a partition.  In a second transaction, we wait until all
+ * transactions that could have seen the partition as attached are gone,
+ * then we remove the rest of partition metadata (pg_inherits and
+ * pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with share-update-exclusive
+	 * in the first transaction.  This allows concurrent transactions to be
+	 * doing DML to the partition.
+	 */
+	partRel = table_openrv(name, concurrent ? ShareLock :
+						   AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps
+	 * locks on the referencing tables, which prevents concurrent transactions
+	 * from adding rows that we wouldn't see.  For this to work in concurrent
+	 * mode, it is critical that the partition appears as no longer attached
+	 * for the RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to the
+	 * partition that matches the partition constraint, if there isn't a matching
+	 * one already.  The reason for this is that the planner may have made
+	 * optimizations that depend on the constraint.  XXX Isn't it sufficient to
+	 * invalidate the partition's relcache entry?
+	 *
+	 * Then we close our existing transaction, and in a new one wait for
+	 * all processes to catch up on the catalog updates we've done so far; at
+	 * that point we can complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/* Add constraint, if needed. XXX hopefully we can just remove this */
+		DetachAddConstraintIfNeeded(wqueue, partRel);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock on the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid, false);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid, bool wait)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	/*
+	 * If asked to, wait until existing snapshots are gone.  This is important
+	 * if the second transaction of DETACH PARTITION CONCURRENTLY is canceled:
+	 * the user could immediately run DETACH FINALIZE without actually waiting
+	 * for existing transactions.  We must not complete the detach action until
+	 * all such queries are complete (otherwise we would present them with an
+	 * inconsistent view of catalogs).
+	 */
+	if (wait)
+	{
+		Snapshot	snap = GetActiveSnapshot();
+
+		WaitForOlderSnapshots(snap->xmin, false);
+	}
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17123,22 +17351,150 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid, true);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
 }
 
+/*
+ * DetachAddConstraintIfNeeded
+ *		Subroutine for ATExecDetachPartition.  Create a constraint that
+ *		takes the place of the partition constraint, but avoid creating
+ *		a dupe if an equivalent constraint already exists.
+ */
+static void
+DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
+{
+	AlteredTableInfo *tab;
+	Expr	   *constraintExpr;
+	TupleDesc	td = RelationGetDescr(partRel);
+	Constraint *n;
+
+	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+
+	/* If an identical constraint exists, we don't need to create one */
+	if (td->constr && td->constr->num_check > 0)
+	{
+		for (int i = 0; i < td->constr->num_check; i++)
+		{
+			Node	*thisconstr;
+
+			thisconstr = stringToNode(td->constr->check[i].ccbin);
+
+			if (equal(constraintExpr, thisconstr))
+				return;
+		}
+	}
+
+	tab = ATGetQueueEntry(wqueue, partRel);
+
+	/* Add constraint on partition, equivalent to the partition constraint */
+	n = makeNode(Constraint);
+	n->contype = CONSTR_CHECK;
+	n->conname = NULL;
+	n->location = -1;
+	n->is_no_inherit = false;
+	n->raw_expr = NULL;
+	n->cooked_expr = nodeToString(constraintExpr);
+	n->initially_valid = true;
+	n->skip_validation = true;
+	/* It's a re-add, since it nominally already exists */
+	ATAddCheckConstraint(wqueue, tab, partRel, n,
+						 true, false, true, ShareUpdateExclusiveLock);
+}
+
 /*
  * DropClonedTriggersFromPartition
  *		subroutine for ATExecDetachPartition to remove any triggers that were
@@ -17334,7 +17690,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17488,7 +17844,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 4e4e05844c..7383d5994e 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1119,7 +1119,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1141,7 +1141,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index b8da4c5967..619aaffae4 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -569,6 +569,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	partrel;
 	int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -579,7 +580,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1065,9 +1066,21 @@ ExecInitPartitionDispatchInfo(EState *estate,
 	int			dispatchidx;
 	MemoryContext oldcxt;
 
+	/*
+	 * For data modification, it is better that executor does not include
+	 * partitions being detached, except in snapshot-isolation mode.  This
+	 * means that a read-committed transaction immediately gets a "no
+	 * partition for tuple" error when a tuple is inserted into a partition
+	 * that's being detached concurrently, but a transaction in repeatable-
+	 * read mode can still use the partition.  Note that because partition
+	 * detach uses ShareLock on the partition (which conflicts with DML),
+	 * we're certain that the detach won't be able to complete until any
+	 * inserting transaction is done.
+	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt,
+									 IsolationUsesXactSnapshot());
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1645,9 +1658,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
+	/* Executor must always include detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1713,9 +1727,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index da91cbd2b1..850e50f1a2 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4735,6 +4735,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index c2d73626fc..250a23b347 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2972,6 +2972,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ade66a541c..6ff5b03af2 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1184,11 +1184,15 @@ target_rel_partitions_max_parallel_hazard(Relation rel,
 
 	/* Recursively check each partition ... */
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already. Must
+	 * match args used in set_relation_partition_info.
+	 */
 	glob = context->planner_global;
 	if (glob->partition_directory == NULL)
 		glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
 
 	pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
 
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index c5947fa418..9ec7d7f518 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2142,10 +2142,17 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 * In snapshot-isolation transactions, we always include detached
+	 * partitions, because otherwise they become invisible too soon.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext,
+									 IsolationUsesXactSnapshot());
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 652be0b96d..b26d0a3d0b 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -646,7 +646,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2096,12 +2096,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2109,8 +2110,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2125,6 +2139,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15348,6 +15363,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
@@ -15888,6 +15904,7 @@ bare_label_keyword:
 			| EXTRACT
 			| FALSE_P
 			| FAMILY
+			| FINALIZE
 			| FIRST_P
 			| FLOAT_P
 			| FOLLOWING
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index e5f3482d52..2b2b1dc1ad 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2798,7 +2798,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec, ParseState *pstate)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	int			with = -1;
 	bool		overlap = false;
@@ -3990,7 +3990,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4190,7 +4190,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index f852b6e99d..58570fecfd 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -37,6 +37,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -46,7 +47,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -61,13 +62,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -88,7 +90,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -110,7 +112,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -238,6 +241,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -280,7 +284,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -295,6 +299,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -327,7 +332,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 05bb698cf4..729274b330 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1236,6 +1236,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index a6388d980e..0fe4f55b04 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -341,6 +341,7 @@ generate_partition_qual(Relation rel)
 	bool		isnull;
 	List	   *my_qual = NIL,
 			   *result = NIL;
+	Oid			parentrelid;
 	Relation	parent;
 
 	/* Guard against stack overflow due to overly deep partition tree */
@@ -350,9 +351,20 @@ generate_partition_qual(Relation rel)
 	if (rel->rd_partcheckvalid)
 		return copyObject(rel->rd_partcheck);
 
+	/*
+	 * Obtain parent relid; if it's invalid, then the partition is being
+	 * detached.  The constraint is NIL in that case, and let's cache that.
+	 */
+	parentrelid = get_partition_parent(RelationGetRelid(rel));
+	if (parentrelid == InvalidOid)
+	{
+		rel->rd_partcheckvalid = true;
+		rel->rd_partcheck = NIL;
+		return NIL;
+	}
+
 	/* Grab at least an AccessShareLock on the parent table */
-	parent = relation_open(get_partition_parent(RelationGetRelid(rel)),
-						   AccessShareLock);
+	parent = relation_open(parentrelid, AccessShareLock);
 
 	/* Get pg_class.relpartbound */
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 20af5a92b4..eb3e79adf7 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2117,7 +2117,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetachpending" :
+						  "false as inhdetachpending");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2135,17 +2140,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACH PENDING" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3197,9 +3204,20 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
 							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " inhdetachpending,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " false AS inhdetachpending,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3208,14 +3226,16 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " false AS inhdetachpending, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " false AS inhdetachpending, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3265,11 +3285,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACH PENDING)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 2b71cad9a2..6d07e1b302 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid BKI_LOOKUP(pg_class);
 	Oid			inhparent BKI_LOOKUP(pg_class);
 	int32		inhseqno;
+	bool		inhdetachpending;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -49,7 +50,8 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -57,6 +59,8 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
+extern bool PartitionHasPendingDetach(Oid partoid);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 236832a2ca..f97105ed3a 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -860,6 +860,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1896,6 +1897,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity,			/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 28083aaac9..7da248a385 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -165,6 +165,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD, AS_LABEL)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD, AS_LABEL)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD, BARE_LABEL)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD, BARE_LABEL)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index ff113199e5..7f03ff4271 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 2c8b881a09..901eccec25 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -133,6 +133,7 @@ extern void AtEOXact_Snapshot(bool isCommit, bool resetXmin);
 extern void ImportSnapshot(const char *idstr);
 extern bool XactHasExportedSnapshots(void);
 extern void DeleteAllExportedSnapshotFiles(void);
+extern void WaitForOlderSnapshots(TransactionId limitXmin, bool progress);
 extern bool ThereAreNoPriorRegisteredSnapshots(void);
 extern bool TransactionIdLimitedForOldSnapshots(TransactionId recentXmin,
 												Relation relation,
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..9b813b9a75
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,237 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2detach s1s s1c s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1s s1ins2 s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1ins s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1ins2 s2detach s3ins2 s1c
+step s1b: BEGIN;
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3ins2: INSERT INTO d_listp VALUES (2); <waiting ...>
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3ins2: <... completed>
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec1 s3s s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec1: EXECUTE f(1);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1prep s1exec2 s2detach s1s s1exec2 s3s s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1exec2: EXECUTE f(2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+2              
+step s1exec2: EXECUTE f(2);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep2 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep2: PREPARE f(int) AS INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/expected/detach-partition-concurrently-3.out b/src/test/isolation/expected/detach-partition-concurrently-3.out
new file mode 100644
index 0000000000..394ec2b40d
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-3.out
@@ -0,0 +1,211 @@
+Parsed test spec with 2 sessions
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1alter
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1alter: ALTER TABLE d3_listp1 ALTER a DROP NOT NULL;
+ERROR:  cannot alter partition "d3_listp1" with an incomplete detach
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1insert
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1insert: INSERT INTO d3_listp VALUES (1);
+ERROR:  no partition of relation "d3_listp" found for row
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1drop s1spart
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1drop: DROP TABLE d3_listp;
+step s1spart: SELECT * FROM d3_listp1;
+ERROR:  relation "d3_listp1" does not exist
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1trunc s1spart
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1trunc: TRUNCATE TABLE d3_listp;
+step s1spart: SELECT * FROM d3_listp1;
+a              
+
+1              
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1b s1s s2detachfinal s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1b s1spart s2detachfinal s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1b: BEGIN;
+step s1spart: SELECT * FROM d3_listp1;
+a              
+
+1              
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1b s1spart s2detachfinal s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1b: BEGIN;
+step s1spart: SELECT * FROM d3_listp1;
+a              
+
+1              
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2detachfinal s2commit
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s2begin: BEGIN;
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s2commit: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2detachfinal s1spart s2commit
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s2begin: BEGIN;
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s1spart: SELECT * FROM d3_listp1; <waiting ...>
+step s2commit: COMMIT;
+step s1spart: <... completed>
+a              
+
+1              
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2detachfinal s1insertpart s2commit
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s2begin: BEGIN;
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s1insertpart: INSERT INTO d3_listp1 VALUES (1); <waiting ...>
+step s2commit: COMMIT;
+step s1insertpart: <... completed>
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index 5d6b79e66e..25cd7d0a13 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,9 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
+test: detach-partition-concurrently-3
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..6ca0adac1d
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,69 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1ins2"	{ INSERT INTO d_listp VALUES (2); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1prep2"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (2); }
+step "s1exec1"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2detach"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3ins2"		{ INSERT INTO d_listp VALUES (2); }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate, as does anybody else.
+permutation "s1b" "s1s" "s2detach" "s1s" "s1c" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2detach" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# In read-committed mode, the partition disappears from view of concurrent
+# transactions immediately.  But if a write lock is held, then the detach
+# has to wait.
+permutation "s1b"   "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1b"   "s1s" "s1ins2" "s2detach" "s1ins" "s1s" "s1c"
+
+# In repeatable-read mode, the partition remains visible until commit even
+# if the to-be-detached partition is not locked for write.
+permutation "s1brr" "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1brr" "s1s"          "s2detach"         "s1s" "s1c"
+
+# Another process trying to acquire a write lock will be blocked behind the
+# detacher
+permutation "s1b" "s1ins2" "s2detach" "s3ins2" "s1c"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec1" "s3s" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1exec2" "s2detach" "s1s" "s1exec2" "s3s" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep2" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-3.spec b/src/test/isolation/specs/detach-partition-concurrently-3.spec
new file mode 100644
index 0000000000..5f5d650791
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-3.spec
@@ -0,0 +1,50 @@
+# Try various things to happen to a partition with an incomplete detach
+
+setup
+{
+  CREATE TABLE d3_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d3_listp1 PARTITION OF d3_listp FOR VALUES IN (1);
+  CREATE TABLE d3_pid (pid int);
+  INSERT INTO d3_listp VALUES (1);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_pid;
+}
+
+session "s1"
+step "s1b"			{ BEGIN; }
+step "s1s"			{ SELECT * FROM d3_listp; }
+step "s1spart"		{ SELECT * FROM d3_listp1; }
+step "s1cancel"		{ SELECT pg_cancel_backend(pid) FROM d3_pid; }
+step "s1c"			{ COMMIT; }
+step "s1alter"		{ ALTER TABLE d3_listp1 ALTER a DROP NOT NULL; }
+step "s1insert"		{ INSERT INTO d3_listp VALUES (1); }
+step "s1insertpart"		{ INSERT INTO d3_listp1 VALUES (1); }
+step "s1drop"		{ DROP TABLE d3_listp; }
+step "s1trunc"		{ TRUNCATE TABLE d3_listp; }
+
+session "s2"
+step "s2begin"		{ BEGIN; }
+step "s2snitch"		{ INSERT INTO d3_pid SELECT pg_backend_pid(); }
+step "s2detach"		{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; }
+step "s2detachfinal"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; }
+step "s2commit"		{ COMMIT; }
+
+# Try various things while the partition is in "being detached" state, with
+# no session waiting.
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1alter"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1insert"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1drop" "s1spart"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1trunc" "s1spart"
+
+# partially detach, then read from partition and try to complete the detach
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1b" "s1s" "s2detachfinal" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1b" "s1spart" "s2detachfinal" "s1c"
+
+# DETACH FINALIZE in a transaction block. No insert/select on the partition
+# is allowed concurrently with that.
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1b" "s1spart" "s2detachfinal" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2detachfinal" "s2commit"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2detachfinal" "s1spart" "s2commit"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2detachfinal" "s1insertpart" "s2commit"
diff --git a/src/test/modules/delay_execution/Makefile b/src/test/modules/delay_execution/Makefile
index f270aebf3a..70f24e846d 100644
--- a/src/test/modules/delay_execution/Makefile
+++ b/src/test/modules/delay_execution/Makefile
@@ -7,7 +7,8 @@ OBJS = \
 	$(WIN32RES) \
 	delay_execution.o
 
-ISOLATION = partition-addition
+ISOLATION = partition-addition \
+	    partition-removal-1
 
 ifdef USE_PGXS
 PG_CONFIG = pg_config
diff --git a/src/test/modules/delay_execution/expected/partition-removal-1.out b/src/test/modules/delay_execution/expected/partition-removal-1.out
new file mode 100644
index 0000000000..427f41c9aa
--- /dev/null
+++ b/src/test/modules/delay_execution/expected/partition-removal-1.out
@@ -0,0 +1,175 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s3lock s1b s1exec s2remp s3check s3unlock s3check s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec s2remp s3check s3unlock s3check s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1b s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2) AND a <> 1; <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2) AND a <> 1; <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1prepare s2remp s1execprep s3unlock s1check s1c s1check s1dealloc
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prepare: PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI');
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s1execprep: EXECUTE ins(2); <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1execprep: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+2              GHI            
+step s1c: COMMIT;
+step s2remp: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s1dealloc: DEALLOCATE ins;
+
+starting permutation: s1brr s1prepare s2remp s3lock s1execprep s3unlock s1check s1c s1check s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prepare: PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI');
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1execprep: EXECUTE ins(2); <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1execprep: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+2              GHI            
+step s1c: COMMIT;
+step s2remp: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s1dealloc: DEALLOCATE ins;
+
+starting permutation: s1brr s1check s3lock s2remp s1prepare s1execprep s3unlock s1check s1c s1check s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s1prepare: PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI');
+step s1execprep: EXECUTE ins(2); <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1execprep: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+2              GHI            
+step s1c: COMMIT;
+step s2remp: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s1dealloc: DEALLOCATE ins;
diff --git a/src/test/modules/delay_execution/specs/partition-removal-1.spec b/src/test/modules/delay_execution/specs/partition-removal-1.spec
new file mode 100644
index 0000000000..5ee2750129
--- /dev/null
+++ b/src/test/modules/delay_execution/specs/partition-removal-1.spec
@@ -0,0 +1,58 @@
+# Test removal of a partition with less-than-exclusive locking.
+
+setup
+{
+  CREATE TABLE partrem (a int, b text) PARTITION BY LIST(a);
+  CREATE TABLE partrem1 PARTITION OF partrem FOR VALUES IN (1);
+  CREATE TABLE partrem2 PARTITION OF partrem FOR VALUES IN (2);
+  CREATE TABLE partrem3 PARTITION OF partrem FOR VALUES IN (3);
+  INSERT INTO partrem VALUES (1, 'ABC');
+  INSERT INTO partrem VALUES (2, 'JKL');
+  INSERT INTO partrem VALUES (3, 'DEF');
+}
+
+teardown
+{
+  DROP TABLE IF EXISTS partrem, partrem2;
+}
+
+session "s1"
+setup		{ LOAD 'delay_execution';
+		  SET delay_execution.post_planning_lock_id = 12543; }
+step "s1b"	{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1exec"	{ SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); }
+step "s1exec2"	{ SELECT * FROM partrem WHERE a <> (SELECT 2) AND a <> 1; }
+step "s1prepare" { PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI'); }
+step "s1execprep" { EXECUTE ins(2); }
+step "s1check" { SELECT * FROM partrem WHERE b = 'GHI'; }
+step "s1c"	{ COMMIT; }
+step "s1dealloc" { DEALLOCATE ins; }
+
+session "s2"
+step "s2remp"	{ ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; }
+
+session "s3"
+step "s3lock"	{ SELECT pg_advisory_lock(12543); }
+step "s3unlock"	{ SELECT pg_advisory_unlock(12543); }
+step "s3check"	{ SELECT * FROM partrem; }
+
+# The SELECT will be planned with all three partitions shown above,
+# of which we expect partrem1 to be pruned at planning and partrem3 at
+# execution. Then we'll block, and by the time the query is actually
+# executed, detach of partrem2 is already underway (so its row doesn't
+# show up in s3's result), but we expect its row to still appear in the
+# result for s1.
+permutation "s3lock" "s1b" "s1exec" "s2remp" "s3check" "s3unlock" "s3check" "s1c"
+permutation "s3lock" "s1brr" "s1exec" "s2remp" "s3check" "s3unlock" "s3check" "s1c"
+
+# In this case we're testing that after pruning partrem2 at runtime, the
+# query still works correctly.
+permutation "s3lock" "s1b" "s1exec2" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec2" "s2remp" "s3unlock" "s1c"
+
+# In this case we test that an insert that's prepared in repeatable read
+# mode still works after detaching.
+permutation "s3lock" "s1brr" "s1prepare" "s2remp" "s1execprep" "s3unlock" "s1check" "s1c" "s1check" "s1dealloc"
+permutation "s1brr" "s1prepare" "s2remp" "s3lock" "s1execprep" "s3unlock" "s1check" "s1c" "s1check" "s1dealloc"
+permutation "s1brr" "s1check" "s3lock" "s2remp" "s1prepare" "s1execprep" "s3unlock" "s1check" "s1c" "s1check" "s1dealloc"
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index bb3f873f22..ec14b060a6 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -4163,6 +4163,35 @@ SELECT * from part_rp;
 (0 rows)
 
 DROP TABLE part_rp;
+-- concurrent detach
+CREATE TABLE range_parted2 (
+	a int
+) PARTITION BY RANGE(a);
+CREATE TABLE part_rp PARTITION OF range_parted2 FOR VALUES FROM (0) to (100);
+BEGIN;
+-- doesn't work in a partition block
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+ERROR:  ALTER TABLE ... DETACH CONCURRENTLY cannot run inside a transaction block
+COMMIT;
+CREATE TABLE part_rpd PARTITION OF range_parted2 DEFAULT;
+-- doesn't work if there's a default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+ERROR:  cannot detach partitions concurrently when a default partition exists
+-- doesn't work for the default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rpd CONCURRENTLY;
+ERROR:  cannot detach partitions concurrently when a default partition exists
+DROP TABLE part_rpd;
+-- works fine
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+\d+ range_parted2
+                         Partitioned table "public.range_parted2"
+ Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description 
+--------+---------+-----------+----------+---------+---------+--------------+-------------
+ a      | integer |           |          |         | plain   |              | 
+Partition key: RANGE (a)
+Number of partitions: 0
+
+DROP TABLE range_parted2;
 -- Check ALTER TABLE commands for partitioned tables and partitions
 -- cannot add/drop column to/from *only* the parent
 ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 4cc55d8525..7a9c9252dc 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2678,6 +2678,26 @@ DROP TABLE range_parted2;
 SELECT * from part_rp;
 DROP TABLE part_rp;
 
+-- concurrent detach
+CREATE TABLE range_parted2 (
+	a int
+) PARTITION BY RANGE(a);
+CREATE TABLE part_rp PARTITION OF range_parted2 FOR VALUES FROM (0) to (100);
+BEGIN;
+-- doesn't work in a partition block
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+COMMIT;
+CREATE TABLE part_rpd PARTITION OF range_parted2 DEFAULT;
+-- doesn't work if there's a default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+-- doesn't work for the default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rpd CONCURRENTLY;
+DROP TABLE part_rpd;
+-- works fine
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+\d+ range_parted2
+DROP TABLE range_parted2;
+
 -- Check ALTER TABLE commands for partitioned tables and partitions
 
 -- cannot add/drop column to/from *only* the parent
-- 
2.20.1

#34Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#33)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

The v8 patch has the "broken constraint" problem.

Also, it "fails to avoid" adding duplicate constraints:

Check constraints:
"c" CHECK (i IS NOT NULL AND i > 1 AND i < 2)
"cc" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)
"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 5c9f4af1d5..0cb846f408 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4485,6 +4485,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
when using declarative partitioning.
</para></entry>
</row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetachpending</structfield> <type>bool</type>
+      </para>
+      <para>
+       Set to true for a partition that is in the process of being detached;
+       false otherwise.
+      </para></entry>
+     </row>

Remove "Set to" ?
And say <literal>true</literal> and <literal>false</literal>

Probably you'll hate the suggestion, but maybe it should be "pendingdetach".
We already have pg_settings.pending_restart.

+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction, a
+      <literal>SHARE UPDATE EXCLUSIVE</literal> lock is taken on both parent table and
+      partition, and its partition is marked detached; at that point, the transaction
+      is committed and all transactions using the partitioned table are waited for.
+      Once all those transactions are gone, the second stage acquires

Instead of "gone", say "have completed" ?

+/*
+ * MarkInheritDetached
+ *
+ * When a partition is detached from its parent concurrently, we don't
+ * remove the pg_inherits row until a second transaction; as a preparatory
+ * step, this function marks the entry as 'detached', so that other

*pending detached

+ * The strategy for concurrency is to first modify the partition catalog
+ * rows to make it visible to everyone that the partition is detached,

the inherits catalog?

+	/*
+	 * In concurrent mode, the partition is locked with share-update-exclusive
+	 * in the first transaction.  This allows concurrent transactions to be
+	 * doing DML to the partition.
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhisdetached flag.

detachpending

#35Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#34)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-17, Justin Pryzby wrote:

The v8 patch has the "broken constraint" problem.

Yeah, I had misunderstood what the problem was. I think a good solution
to this is to have get_partition_parent return the true parent even when
a detach is pending, for one particular callsite. (This means adjusting
all other callsites.) Notpatch attached (applies on top of v8).

Also, it "fails to avoid" adding duplicate constraints:

Check constraints:
"c" CHECK (i IS NOT NULL AND i > 1 AND i < 2)
"cc" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)
"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

Do you mean the "cc" and "p1_i_check" one? I mean, if you already have
a constraint in the partition that duplicates the partition constraint,
then during attach we still create our new constraint? I guess a
solution to this would be to scan all constraints and see if any equals
the expression that the new one would have. Sounds easy enough now that
write it out loud.

Thanks

--
�lvaro Herrera 39�49'30"S 73�17'W
"Java is clearly an example of money oriented programming" (A. Stepanov)

Attachments:

0001-Fix-for-generation-of-invalid-constraints.notpatchtext/plain; charset=us-asciiDownload
From ecec91fc4744ddd9a231754afdf10ec09742f8d9 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Fri, 19 Mar 2021 10:52:26 -0300
Subject: [PATCH] Fix for generation of invalid constraints

per report from Justin Pryzby
---
 src/backend/catalog/heap.c          |  2 +-
 src/backend/catalog/partition.c     | 32 ++++++++++++++---------------
 src/backend/commands/tablecmds.c    | 12 +++++------
 src/backend/utils/cache/partcache.c | 11 ++--------
 src/include/catalog/partition.h     |  2 +-
 5 files changed, 26 insertions(+), 33 deletions(-)

diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 1a05f2a2fe..c6c5e40a39 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -1919,7 +1919,7 @@ heap_drop_with_catalog(Oid relid)
 		elog(ERROR, "cache lookup failed for relation %u", relid);
 	if (((Form_pg_class) GETSTRUCT(tuple))->relispartition)
 	{
-		parentOid = get_partition_parent(relid);
+		parentOid = get_partition_parent(relid, true);
 		LockRelationOid(parentOid, AccessExclusiveLock);
 
 		/*
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 797a7384ad..a7cf3d9e86 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -42,15 +42,15 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
- * If the partition is in the process of being detached, InvalidOid is
- * returned and no error is thrown.
+ * If the partition is in the process of being detached, we raise the same
+ * error unless even_if_detached is passed.
  *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
  */
 Oid
-get_partition_parent(Oid relid)
+get_partition_parent(Oid relid, bool even_if_detached)
 {
 	Relation	catalogRelation;
 	Oid			result;
@@ -60,10 +60,10 @@ get_partition_parent(Oid relid)
 
 	result = get_partition_parent_worker(catalogRelation, relid, &detached);
 
-	if (detached)
-		result = InvalidOid;
-	else if (!OidIsValid(result))
+	if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
+	if (detached && !even_if_detached)
+		elog(ERROR, "relation %u has no parent because it's being detached", relid);
 
 	table_close(catalogRelation, AccessShareLock);
 
@@ -75,8 +75,8 @@ get_partition_parent(Oid relid)
  *		Scan the pg_inherits relation to return the OID of the parent of the
  *		given relation
  *
- * If the partition is being detached, InvalidOid is returned, and also
- * *detached is set true.
+ * If the partition is being detached, *detached is set true (but the original
+ * parent is still returned.)
  */
 static Oid
 get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
@@ -106,12 +106,9 @@ get_partition_parent_worker(Relation inhRel, Oid relid, bool *detached)
 
 		/* A partition being detached has no parent */
 		if (form->inhdetachpending)
-		{
 			*detached = true;
-			result = InvalidOid;
-		}
-		else
-			result = form->inhparent;
+
+		result = form->inhparent;
 	}
 
 	systable_endscan(scan);
@@ -154,9 +151,12 @@ get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 	Oid			parentOid;
 	bool		detached;
 
-	/* Recursion ends at the topmost level, ie., when there's no parent */
+	/*
+	 * Recursion ends at the topmost level, ie., when there's no parent; also
+	 * when the partition is being detached.
+	 */
 	parentOid = get_partition_parent_worker(inhRel, relid, &detached);
-	if (parentOid == InvalidOid)
+	if (parentOid == InvalidOid || detached)
 		return;
 
 	*ancestors = lappend_oid(*ancestors, parentOid);
@@ -189,7 +189,7 @@ index_get_partition(Relation partition, Oid indexId)
 		ReleaseSysCache(tup);
 		if (!ispartition)
 			continue;
-		if (get_partition_parent(partIdx) == indexId)
+		if (get_partition_parent(partIdx, false) == indexId)
 		{
 			list_free(idxlist);
 			return partIdx;
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 21dc869b0b..853dc8d257 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -1547,7 +1547,7 @@ RangeVarCallbackForDropRelation(const RangeVar *rel, Oid relOid, Oid oldRelOid,
 	 */
 	if (is_partition && relOid != oldRelOid)
 	{
-		state->partParentOid = get_partition_parent(relOid);
+		state->partParentOid = get_partition_parent(relOid, false);
 		if (OidIsValid(state->partParentOid))
 			LockRelationOid(state->partParentOid, AccessExclusiveLock);
 	}
@@ -6816,7 +6816,7 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	/* If rel is partition, shouldn't drop NOT NULL if parent has the same */
 	if (rel->rd_rel->relispartition)
 	{
-		Oid			parentId = get_partition_parent(RelationGetRelid(rel));
+		Oid			parentId = get_partition_parent(RelationGetRelid(rel), false);
 		Relation	parent = table_open(parentId, AccessShareLock);
 		TupleDesc	tupDesc = RelationGetDescr(parent);
 		AttrNumber	parent_attnum;
@@ -17361,7 +17361,7 @@ DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
 		if (!has_superclass(idxid))
 			continue;
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
+		Assert((IndexGetRelation(get_partition_parent(idxid, false), false) ==
 				RelationGetRelid(rel)));
 
 		idx = index_open(idxid, AccessExclusiveLock);
@@ -17660,7 +17660,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 
 	/* Silently do nothing if already in the right state */
 	currParent = partIdx->rd_rel->relispartition ?
-		get_partition_parent(partIdxId) : InvalidOid;
+		get_partition_parent(partIdxId, false) : InvalidOid;
 	if (currParent != RelationGetRelid(parentIdx))
 	{
 		IndexInfo  *childInfo;
@@ -17872,8 +17872,8 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 		/* make sure we see the validation we just did */
 		CommandCounterIncrement();
 
-		parentIdxId = get_partition_parent(RelationGetRelid(partedIdx));
-		parentTblId = get_partition_parent(RelationGetRelid(partedTbl));
+		parentIdxId = get_partition_parent(RelationGetRelid(partedIdx), false);
+		parentTblId = get_partition_parent(RelationGetRelid(partedTbl), false);
 		parentIdx = relation_open(parentIdxId, AccessExclusiveLock);
 		parentTbl = relation_open(parentTblId, AccessExclusiveLock);
 		Assert(!parentIdx->rd_index->indisvalid);
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index 0fe4f55b04..6dfa3fb4a8 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -352,16 +352,9 @@ generate_partition_qual(Relation rel)
 		return copyObject(rel->rd_partcheck);
 
 	/*
-	 * Obtain parent relid; if it's invalid, then the partition is being
-	 * detached.  The constraint is NIL in that case, and let's cache that.
+	 * Obtain parent relid.  XXX explain why we need this
 	 */
-	parentrelid = get_partition_parent(RelationGetRelid(rel));
-	if (parentrelid == InvalidOid)
-	{
-		rel->rd_partcheckvalid = true;
-		rel->rd_partcheck = NIL;
-		return NIL;
-	}
+	parentrelid = get_partition_parent(RelationGetRelid(rel), true);
 
 	/* Grab at least an AccessShareLock on the parent table */
 	parent = relation_open(parentrelid, AccessShareLock);
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index fe3f66befa..c8c7bc1d99 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,7 +19,7 @@
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-extern Oid	get_partition_parent(Oid relid);
+extern Oid	get_partition_parent(Oid relid, bool even_if_detached);
 extern List *get_partition_ancestors(Oid relid);
 extern Oid	index_get_partition(Relation partition, Oid indexId);
 extern List *map_partition_varattnos(List *expr, int fromrel_varno,
-- 
2.20.1

#36Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#35)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Fri, Mar 19, 2021 at 10:57:37AM -0300, Alvaro Herrera wrote:

Also, it "fails to avoid" adding duplicate constraints:

Check constraints:
"c" CHECK (i IS NOT NULL AND i > 1 AND i < 2)
"cc" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)
"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

Do you mean the "cc" and "p1_i_check" one? I mean, if you already have

No, I started with c and cc, and it added the broken constraint p1_check (which
you say you've fixed) and the redundant constraint p1_i_check. I guess that's
what you meant.

a constraint in the partition that duplicates the partition constraint,
then during attach we still create our new constraint? I guess a
solution to this would be to scan all constraints and see if any equals
the expression that the new one would have. Sounds easy enough now that
write it out loud.

But it looks like DetachAddConstraintIfNeeded already intended to do that:

+            if (equal(constraintExpr, thisconstr))                                                                                                                                                                     
+                    return;                                                                                                                                                                                            

Actually, it appears your latest notpatch resolves both these issues.
But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

--
Justin

#37Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#36)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-21, Justin Pryzby wrote:

On Fri, Mar 19, 2021 at 10:57:37AM -0300, Alvaro Herrera wrote:

Also, it "fails to avoid" adding duplicate constraints:

Check constraints:
"c" CHECK (i IS NOT NULL AND i > 1 AND i < 2)
"cc" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)
"p1_check" CHECK (true)
"p1_i_check" CHECK (i IS NOT NULL AND i >= 1 AND i < 2)

Do you mean the "cc" and "p1_i_check" one? I mean, if you already have

No, I started with c and cc, and it added the broken constraint p1_check (which
you say you've fixed) and the redundant constraint p1_i_check. I guess that's
what you meant.

Yes, that's what I meant.

a constraint in the partition that duplicates the partition constraint,
then during attach we still create our new constraint? I guess a
solution to this would be to scan all constraints and see if any equals
the expression that the new one would have. Sounds easy enough now that
write it out loud.

But it looks like DetachAddConstraintIfNeeded already intended to do that:

+            if (equal(constraintExpr, thisconstr))
+                    return;

Hah, so I had already done it, but forgot.

Actually, it appears your latest notpatch resolves both these issues.

Great.

But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

Hm, I'm not sure I want to do that, because that means that if I later
have to attach the partition again with the same partition bounds, then
I might have to incur a scan to recheck the constraint. I think we want
to make the new constraint be as tight as possible ...

--
�lvaro Herrera 39�49'30"S 73�17'W

#38Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#35)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-19, Alvaro Herrera wrote:

diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index 0fe4f55b04..6dfa3fb4a8 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -352,16 +352,9 @@ generate_partition_qual(Relation rel)
return copyObject(rel->rd_partcheck);
/*
-	 * Obtain parent relid; if it's invalid, then the partition is being
-	 * detached.  The constraint is NIL in that case, and let's cache that.
+	 * Obtain parent relid.  XXX explain why we need this
*/
-	parentrelid = get_partition_parent(RelationGetRelid(rel));
-	if (parentrelid == InvalidOid)
-	{
-		rel->rd_partcheckvalid = true;
-		rel->rd_partcheck = NIL;
-		return NIL;
-	}
+	parentrelid = get_partition_parent(RelationGetRelid(rel), true);

One thing that makes me uneasy about this, is that I don't understand
how does this happen with your test of two psqls doing attach/detach.
(It is necessary for the case when the waiting concurrent detach is
canceled, and so this fix is necessary anyway). In your test, no
waiting transaction is ever cancelled; so what is the period during
which the relation is not locked that causes this code to be hit? I
fear that there's a bug in the lock protocol somewhere.

--
�lvaro Herrera Valdivia, Chile
"In Europe they call me Niklaus Wirth; in the US they call me Nickel's worth.
That's because in Europe they call me by name, and in the US by value!"

#39Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#37)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Sun, Mar 21, 2021 at 03:01:15PM -0300, Alvaro Herrera wrote:

But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

Hm, I'm not sure I want to do that, because that means that if I later
have to attach the partition again with the same partition bounds, then
I might have to incur a scan to recheck the constraint. I think we want
to make the new constraint be as tight as possible ...

The ATTACH PARTITION checks if any existing constraint impilies the (proposed)
partition bounds, not just if constraints are equal. So I'm suggesting to do
the same here.

--
Justin

#40Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#39)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-21, Justin Pryzby wrote:

On Sun, Mar 21, 2021 at 03:01:15PM -0300, Alvaro Herrera wrote:

But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

Hm, I'm not sure I want to do that, because that means that if I later
have to attach the partition again with the same partition bounds, then
I might have to incur a scan to recheck the constraint. I think we want
to make the new constraint be as tight as possible ...

The ATTACH PARTITION checks if any existing constraint impilies the (proposed)
partition bounds, not just if constraints are equal. So I'm suggesting to do
the same here.

So if we do that on DETACH, what would happen on ATTACH?

--
�lvaro Herrera 39�49'30"S 73�17'W

#41Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#40)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Sun, Mar 21, 2021 at 03:22:00PM -0300, Alvaro Herrera wrote:

On 2021-Mar-21, Justin Pryzby wrote:

On Sun, Mar 21, 2021 at 03:01:15PM -0300, Alvaro Herrera wrote:

But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

Hm, I'm not sure I want to do that, because that means that if I later
have to attach the partition again with the same partition bounds, then
I might have to incur a scan to recheck the constraint. I think we want
to make the new constraint be as tight as possible ...

The ATTACH PARTITION checks if any existing constraint impilies the (proposed)
partition bounds, not just if constraints are equal. So I'm suggesting to do
the same here.

So if we do that on DETACH, what would happen on ATTACH?

Do you mean what happens to the constraint that was already there ?
Nothing, since it's not ours to mess with. Checking ImpliedBy() rather than
equal() doesn't change that.

I proposed this a few years ago for DETACH (without concurrently), specifically
to avoid the partition scans.
/messages/by-id/20180601221428.GU5164@telsasoft.com
|The docs say: if detaching/re-attach a partition, should first ADD CHECK to
|avoid a slow ATTACH operation. Perhaps DETACHing a partition could
|implicitly CREATE a constraint which is usable when reATTACHing?

--
Justin

#42Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#41)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-21, Justin Pryzby wrote:

On Sun, Mar 21, 2021 at 03:22:00PM -0300, Alvaro Herrera wrote:

So if we do that on DETACH, what would happen on ATTACH?

Do you mean what happens to the constraint that was already there ?
Nothing, since it's not ours to mess with. Checking ImpliedBy() rather than
equal() doesn't change that.

No, I meant what happens regarding checking existing values in the
table: is the table scanned even if the partition constraint is implied
by existing table constraints?

I proposed this a few years ago for DETACH (without concurrently), specifically
to avoid the partition scans.
/messages/by-id/20180601221428.GU5164@telsasoft.com
|The docs say: if detaching/re-attach a partition, should first ADD CHECK to
|avoid a slow ATTACH operation. Perhaps DETACHing a partition could
|implicitly CREATE a constraint which is usable when reATTACHing?

Well, I agree with you that we should add such a constraint.

--
�lvaro Herrera Valdivia, Chile
"The problem with the future is that it keeps turning into the present"
(Hobbes)

#43Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#42)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Sun, Mar 21, 2021 at 01:14:20PM -0500, Justin Pryzby wrote:

On Sun, Mar 21, 2021 at 03:01:15PM -0300, Alvaro Herrera wrote:

But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

Hm, I'm not sure I want to do that, because that means that if I later
have to attach the partition again with the same partition bounds, then
I might have to incur a scan to recheck the constraint. I think we want
to make the new constraint be as tight as possible ...

The ATTACH PARTITION checks if any existing constraint impilies the (proposed)
partition bounds, not just if constraints are equal. So I'm suggesting to do
the same here.

On Sun, Mar 21, 2021 at 04:07:12PM -0300, Alvaro Herrera wrote:

On 2021-Mar-21, Justin Pryzby wrote:

On Sun, Mar 21, 2021 at 03:22:00PM -0300, Alvaro Herrera wrote:

So if we do that on DETACH, what would happen on ATTACH?

Do you mean what happens to the constraint that was already there ?
Nothing, since it's not ours to mess with. Checking ImpliedBy() rather than
equal() doesn't change that.

No, I meant what happens regarding checking existing values in the
table: is the table scanned even if the partition constraint is implied
by existing table constraints?

I'm still not sure we're talking about the same thing.

Your patch adds a CHECK constraint during DETACH CONCURRENTLY, and I suggested
that it should avoid adding it if it's redundant with an existing constraint,
even if not equal().

The current behavior (since v10) is this:

postgres=# ALTER TABLE p ATTACH PARTITION p1 FOR VALUES FROM (1)TO(2);
DEBUG: partition constraint for table "p1" is implied by existing constraints
ALTER TABLE

And that wouldn't change, except the CHECK constraint would be added
automatically during detach (if it wasn't already implied). Maybe the CHECK
constraint should be added without CONCURRENTLY, too. One fewer difference in
behavior.

--
Justin

#44Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#43)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

So I was about ready to get these patches pushed, when I noticed that in
REPEATABLE READ isolation mode it is possible to insert rows violating
an FK referencing the partition that is being detached. I'm not sure
what is a good solution to this problem.

The problem goes like this:

/* setup */
drop table if exists d4_primary, d4_primary1, d4_fk;
create table d4_primary (a int primary key) partition by list (a);
create table d4_primary1 partition of d4_primary for values in (1);
insert into d4_primary values (1);
create table d4_fk (a int references d4_primary);

/* session 1 */
begin isolation level repeatable read;
select * from d4_primary;

/* session 2 */
alter table d4_primary detach partition d4_primary1 concurrently;
-- blocks
-- Cancel wait: Ctrl-c

/* session 1 */
insert into d4_fk values (1);
commit;

At this point, d4_fk contains the value (1) which is not present in
d4_primary.

This doesn't happen in READ COMMITTED mode; the INSERT at the final step
fails with "insert or update in table f4_fk violates the foreign key",
which is what I expected to happen here too.

I had the idea that the RI code, in REPEATABLE READ mode, used a
different snapshot for the RI queries than the transaction snapshot.
Maybe I'm wrong about that.

I'm looking into that now.

--
�lvaro Herrera Valdivia, Chile
"Cuando ma�ana llegue pelearemos segun lo que ma�ana exija" (Mowgli)

#45Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#44)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-23, Alvaro Herrera wrote:

So I was about ready to get these patches pushed, when I noticed that in
REPEATABLE READ isolation mode it is possible to insert rows violating
an FK referencing the partition that is being detached. I'm not sure
what is a good solution to this problem.

...

I had the idea that the RI code, in REPEATABLE READ mode, used a
different snapshot for the RI queries than the transaction snapshot.

I am definitely right about this. So why doesn't it work? The reason
is that when SPI goes to execute the query, it obtains a new partition
directory, and we tell it to include detached partitions precisely
because we're in REPEATABLE READ mode.

In other words, the idea that we can blanket use the snapshot-isolation
condition to decide whether to include detached partitions or not, is
bogus and needs at least the refinement that for any query that comes
from the RI system, we need a partition directory that does not include
detached partitions.

--
�lvaro Herrera 39�49'30"S 73�17'W
"El sabio habla porque tiene algo que decir;
el tonto, porque tiene que decir algo" (Platon).

#46Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#45)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

I'm coming around to the idea that the fact that you can cancel the wait
phase of DETACH CONCURRENTLY creates quite a disaster, and it's not easy
to get away from it. The idea that REPEATABLE READ mode means that you
now see detached partitions as if they were in normal condition, is
completely at odds with that behavior.

I think a possible solution to this problem is that the "detach" flag in
pg_inherits is not a boolean anymore, but an Xid (or maybe two Xids).
Not sure exactly which Xid(s) yet, and I'm not sure what are the exact
rules, but the Xid becomes a marker that indicates an horizon past which
the partition is no longer visible. Then, REPEATABLE READ can see the
partition, but only if its snapshot is older than the Xid.

--
�lvaro Herrera Valdivia, Chile
"La persona que no quer�a pecar / estaba obligada a sentarse
en duras y empinadas sillas / desprovistas, por cierto
de blandos atenuantes" (Patricio Vogel)

#47Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#46)
2 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-23, Alvaro Herrera wrote:

I think a possible solution to this problem is that the "detach" flag in
pg_inherits is not a boolean anymore, but an Xid (or maybe two Xids).
Not sure exactly which Xid(s) yet, and I'm not sure what are the exact
rules, but the Xid becomes a marker that indicates an horizon past which
the partition is no longer visible. Then, REPEATABLE READ can see the
partition, but only if its snapshot is older than the Xid.

So a solution to this problem seems similar (but not quite the same) as
pg_index.indcheckxmin: the partition is included in the partition
directory, or not, depending on the pg_inherits tuple visibility for the
active snapshot. This solves the problem because the RI query uses a
fresh snapshot, for which the partition has already been detached, while
the normal REPEATABLE READ query is using the old snapshot for which the
'detach-pending' row is still seen as in progress. With this, the weird
hack in a couple of places that needed to check the isolation level is
gone, which makes me a bit more comfortable.

So attached is v9 with this problem solved.

I'll add one more torture test, and if it works correctly I'll push it:
have a cursor in the repeatable read transaction, which can read the
referenced partition table and see the row in the detached partition,
but the RI query must not see that row. Bonus: the RI query is run from
another cursor that is doing UPDATE WHERE CURRENT OF that cursor.

--
�lvaro Herrera 39�49'30"S 73�17'W
"Estoy de acuerdo contigo en que la verdad absoluta no existe...
El problema es que la mentira s� existe y tu est�s mintiendo" (G. Lama)

Attachments:

v9-0001-Let-ALTER-TABLE-exec-routines-deal-with-the-relat.patchtext/x-diff; charset=us-asciiDownload
From 61ff1c133bb4fa7e81f119e806af9760b2faf0e4 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 13 Jul 2020 20:15:30 -0400
Subject: [PATCH v9 1/2] Let ALTER TABLE exec routines deal with the relation

This means that ATExecCmd relies on AlteredRelationInfo->rel instead of
keeping the relation as a local variable; this is useful if the
subcommand needs to modify the relation internally.  For example, a
subcommand that internally commits its transaction needs this.
---
 src/backend/commands/tablecmds.c | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3349bcfaa7..bf7fd6e8ae 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -157,6 +157,8 @@ typedef struct AlteredTableInfo
 	Oid			relid;			/* Relation to work on */
 	char		relkind;		/* Its relkind */
 	TupleDesc	oldDesc;		/* Pre-modification tuple descriptor */
+	/* Transiently set during Phase 2, normally set to NULL */
+	Relation	rel;
 	/* Information saved by Phase 1 for Phase 2: */
 	List	   *subcmds[AT_NUM_PASSES]; /* Lists of AlterTableCmd */
 	/* Information saved by Phases 1/2 for Phase 3: */
@@ -354,7 +356,7 @@ static void ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  AlterTableUtilityContext *context);
 static void ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 							  AlterTableUtilityContext *context);
-static void ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+static void ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 					  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 					  AlterTableUtilityContext *context);
 static AlterTableCmd *ATParseTransformCmd(List **wqueue, AlteredTableInfo *tab,
@@ -4569,7 +4571,6 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 		{
 			AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 			List	   *subcmds = tab->subcmds[pass];
-			Relation	rel;
 			ListCell   *lcmd;
 
 			if (subcmds == NIL)
@@ -4578,10 +4579,10 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			/*
 			 * Appropriate lock was obtained by phase 1, needn't get it again
 			 */
-			rel = relation_open(tab->relid, NoLock);
+			tab->rel = relation_open(tab->relid, NoLock);
 
 			foreach(lcmd, subcmds)
-				ATExecCmd(wqueue, tab, rel,
+				ATExecCmd(wqueue, tab,
 						  castNode(AlterTableCmd, lfirst(lcmd)),
 						  lockmode, pass, context);
 
@@ -4593,7 +4594,11 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
 			if (pass == AT_PASS_ALTER_TYPE)
 				ATPostAlterTypeCleanup(wqueue, tab, lockmode);
 
-			relation_close(rel, NoLock);
+			if (tab->rel)
+			{
+				relation_close(tab->rel, NoLock);
+				tab->rel = NULL;
+			}
 		}
 	}
 
@@ -4619,11 +4624,12 @@ ATRewriteCatalogs(List **wqueue, LOCKMODE lockmode,
  * ATExecCmd: dispatch a subcommand to appropriate execution routine
  */
 static void
-ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
+ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 		  AlterTableCmd *cmd, LOCKMODE lockmode, int cur_pass,
 		  AlterTableUtilityContext *context)
 {
 	ObjectAddress address = InvalidObjectAddress;
+	Relation	rel = tab->rel;
 
 	switch (cmd->subtype)
 	{
@@ -5730,6 +5736,7 @@ ATGetQueueEntry(List **wqueue, Relation rel)
 	 */
 	tab = (AlteredTableInfo *) palloc0(sizeof(AlteredTableInfo));
 	tab->relid = relid;
+	tab->rel = NULL;			/* set later */
 	tab->relkind = rel->rd_rel->relkind;
 	tab->oldDesc = CreateTupleDescCopyConstr(RelationGetDescr(rel));
 	tab->newrelpersistence = RELPERSISTENCE_PERMANENT;
-- 
2.20.1

v9-0002-ALTER-TABLE-.-DETACH-CONCURRENTLY.patchtext/x-diff; charset=us-asciiDownload
From 9529835de0b3c1bde861a83d52e2741e231934a5 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Thu, 25 Mar 2021 12:44:08 -0300
Subject: [PATCH v9 2/2] ALTER TABLE ... DETACH CONCURRENTLY

---
 doc/src/sgml/catalogs.sgml                    |  10 +
 doc/src/sgml/ref/alter_table.sgml             |  28 +-
 src/backend/catalog/heap.c                    |  20 +-
 src/backend/catalog/index.c                   |   4 +-
 src/backend/catalog/partition.c               |  38 +-
 src/backend/catalog/pg_inherits.c             | 114 +++-
 src/backend/commands/indexcmds.c              |   6 +-
 src/backend/commands/tablecmds.c              | 569 ++++++++++++++----
 src/backend/commands/trigger.c                |   5 +-
 src/backend/executor/execPartition.c          |  29 +-
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/equalfuncs.c                |   1 +
 src/backend/optimizer/util/plancat.c          |   8 +-
 src/backend/parser/gram.y                     |  23 +-
 src/backend/partitioning/partbounds.c         |   6 +-
 src/backend/partitioning/partdesc.c           |  21 +-
 src/backend/tcop/utility.c                    |  19 +
 src/backend/utils/adt/ri_triggers.c           |   6 +-
 src/backend/utils/cache/partcache.c           |  12 +-
 src/bin/psql/describe.c                       |  42 +-
 src/include/catalog/partition.h               |   2 +-
 src/include/catalog/pg_inherits.h             |   8 +-
 src/include/nodes/parsenodes.h                |   2 +
 src/include/parser/kwlist.h                   |   1 +
 src/include/partitioning/partdesc.h           |   5 +-
 src/include/utils/snapmgr.h                   |   1 +
 .../detach-partition-concurrently-1.out       | 234 +++++++
 .../detach-partition-concurrently-2.out       |  66 ++
 .../detach-partition-concurrently-3.out       | 282 +++++++++
 .../detach-partition-concurrently-4.out       |  69 +++
 src/test/isolation/isolation_schedule         |   4 +
 .../detach-partition-concurrently-1.spec      |  69 +++
 .../detach-partition-concurrently-2.spec      |  41 ++
 .../detach-partition-concurrently-3.spec      |  66 ++
 .../detach-partition-concurrently-4.spec      |  36 ++
 src/test/modules/delay_execution/Makefile     |   3 +-
 .../expected/partition-removal-1.out          | 175 ++++++
 .../specs/partition-removal-1.spec            |  58 ++
 src/test/regress/expected/alter_table.out     |  29 +
 src/test/regress/sql/alter_table.sql          |  20 +
 40 files changed, 1959 insertions(+), 174 deletions(-)
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-1.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-2.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-3.out
 create mode 100644 src/test/isolation/expected/detach-partition-concurrently-4.out
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-1.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-2.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-3.spec
 create mode 100644 src/test/isolation/specs/detach-partition-concurrently-4.spec
 create mode 100644 src/test/modules/delay_execution/expected/partition-removal-1.out
 create mode 100644 src/test/modules/delay_execution/specs/partition-removal-1.spec

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bae4d8cdd3..cd00d9e3bb 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -4497,6 +4497,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        when using declarative partitioning.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>inhdetachpending</structfield> <type>bool</type>
+      </para>
+      <para>
+       <literal>true</literal> for a partition that is in the process of
+       being detached; <literal>false</literal> otherwise.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0bd0c1a503..6db505af45 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -36,7 +36,9 @@ ALTER TABLE ALL IN TABLESPACE <replaceable class="parameter">name</replaceable>
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
     ATTACH PARTITION <replaceable class="parameter">partition_name</replaceable> { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
-    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> FINALIZE
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -954,7 +956,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
    </varlistentry>
 
    <varlistentry>
-    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable></term>
+    <term><literal>DETACH PARTITION</literal> <replaceable class="parameter">partition_name</replaceable> [ { CONCURRENTLY | FINALIZE } ]</term>
+
     <listitem>
      <para>
       This form detaches the specified partition of the target table.  The detached
@@ -962,6 +965,27 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       ties to the table from which it was detached.  Any indexes that were
       attached to the target table's indexes are detached.  Any triggers that
       were created as clones of those in the target table are removed.
+      <literal>SHARE<literal> lock is obtained on any tables that reference
+      this partitioned table in foreign key constraints.
+     </para>
+     <para>
+      If <literal>CONCURRENTLY</literal> is specified, this process runs in two
+      transactions in order to avoid blocking other sessions that might be accessing
+      the partitioned table.  During the first transaction, a
+      <literal>SHARE UPDATE EXCLUSIVE</literal> lock is taken on both parent table and
+      partition, and the partition is marked as undergoing detach;
+      at that point, the transaction is committed and all other transactions using
+      the partitioned table are waited for.
+      Once all those transactions have completed, the second transaction acquires
+      <literal>ACCESS EXCLUSIVE</literal> on the partition, and the detach process
+      completes.
+      <literal>CONCURRENTLY</literal> is not allowed if the
+      partitioned table contains a default partition.
+     </para>
+     <para>
+      If <literal>FINALIZE</literal> is specified, a previous
+      <literal>DETACH CONCURRENTLY</literal> invocation that was cancelled or
+      interrupted is completed.
      </para>
     </listitem>
    </varlistentry>
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index d0ec44bb40..9f6303266f 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -1923,7 +1923,12 @@ heap_drop_with_catalog(Oid relid)
 		elog(ERROR, "cache lookup failed for relation %u", relid);
 	if (((Form_pg_class) GETSTRUCT(tuple))->relispartition)
 	{
-		parentOid = get_partition_parent(relid);
+		/*
+		 * We have to lock the parent if the partition is being detached,
+		 * because it's possible that some query still has a partition
+		 * descriptor that includes this partition.
+		 */
+		parentOid = get_partition_parent(relid, true);
 		LockRelationOid(parentOid, AccessExclusiveLock);
 
 		/*
@@ -2559,10 +2564,12 @@ StoreConstraints(Relation rel, List *cooked_constraints, bool is_internal)
  * Returns a list of CookedConstraint nodes that shows the cooked form of
  * the default and constraint expressions added to the relation.
  *
- * NB: caller should have opened rel with AccessExclusiveLock, and should
- * hold that lock till end of transaction.  Also, we assume the caller has
- * done a CommandCounterIncrement if necessary to make the relation's catalog
- * tuples visible.
+ * NB: caller should have opened rel with some self-conflicting lock mode,
+ * and should hold that lock till end of transaction; for normal cases that'll
+ * be AccessExclusiveLock, but if caller knows that the constraint is already
+ * enforced by some other means, it can be ShareUpdateExclusiveLock.  Also, we
+ * assume the caller has done a CommandCounterIncrement if necessary to make
+ * the relation's catalog tuples visible.
  */
 List *
 AddRelationNewConstraints(Relation rel,
@@ -3831,7 +3838,8 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	 * relcache entry for that partition every time a partition is added or
 	 * removed.
 	 */
-	defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+	defaultPartOid =
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, false));
 	if (OidIsValid(defaultPartOid))
 		CacheInvalidateRelcacheByRelid(defaultPartOid);
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 397d70d226..af30840c04 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -1837,7 +1837,7 @@ index_concurrently_swap(Oid newIndexId, Oid oldIndexId, const char *oldName)
 		List	   *ancestors = get_partition_ancestors(oldIndexId);
 		Oid			parentIndexRelid = linitial_oid(ancestors);
 
-		DeleteInheritsTuple(oldIndexId, parentIndexRelid);
+		DeleteInheritsTuple(oldIndexId, parentIndexRelid, false, NULL);
 		StoreSingleInheritance(newIndexId, parentIndexRelid, 1);
 
 		list_free(ancestors);
@@ -2487,7 +2487,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	/*
 	 * fix INHERITS relation
 	 */
-	DeleteInheritsTuple(indexId, InvalidOid);
+	DeleteInheritsTuple(indexId, InvalidOid, false, NULL);
 
 	/*
 	 * We are presently too lazy to attempt to compute the new correct value
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index af7754d6ab..790f4ccb92 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -32,7 +32,8 @@
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
+static Oid	get_partition_parent_worker(Relation inhRel, Oid relid,
+										bool *detach_pending);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 										   List **ancestors);
 
@@ -42,23 +43,32 @@ static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
  *
  * Returns inheritance parent of a partition by scanning pg_inherits
  *
+ * If the partition is in the process of being detached, an error is thrown,
+ * unless even_if_detached is passed as true.
+ *
  * Note: Because this function assumes that the relation whose OID is passed
  * as an argument will have precisely one parent, it should only be called
  * when it is known that the relation is a partition.
  */
 Oid
-get_partition_parent(Oid relid)
+get_partition_parent(Oid relid, bool even_if_detached)
 {
 	Relation	catalogRelation;
 	Oid			result;
+	bool		detach_pending;
 
 	catalogRelation = table_open(InheritsRelationId, AccessShareLock);
 
-	result = get_partition_parent_worker(catalogRelation, relid);
+	result = get_partition_parent_worker(catalogRelation, relid,
+										 &detach_pending);
 
 	if (!OidIsValid(result))
 		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
+	if (detach_pending && !even_if_detached)
+		elog(ERROR, "relation %u has no parent because it's being detached",
+			 relid);
+
 	table_close(catalogRelation, AccessShareLock);
 
 	return result;
@@ -68,15 +78,20 @@ get_partition_parent(Oid relid)
  * get_partition_parent_worker
  *		Scan the pg_inherits relation to return the OID of the parent of the
  *		given relation
+ *
+ * If the partition is being detached, *detach_pending is set true (but the
+ * original parent is still returned.)
  */
 static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
+get_partition_parent_worker(Relation inhRel, Oid relid, bool *detach_pending)
 {
 	SysScanDesc scan;
 	ScanKeyData key[2];
 	Oid			result = InvalidOid;
 	HeapTuple	tuple;
 
+	*detach_pending = false;
+
 	ScanKeyInit(&key[0],
 				Anum_pg_inherits_inhrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
@@ -93,6 +108,9 @@ get_partition_parent_worker(Relation inhRel, Oid relid)
 	{
 		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
 
+		/* Let caller know of partition being detached */
+		if (form->inhdetachpending)
+			*detach_pending = true;
 		result = form->inhparent;
 	}
 
@@ -134,10 +152,14 @@ static void
 get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
 {
 	Oid			parentOid;
+	bool		detach_pending;
 
-	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
-	if (parentOid == InvalidOid)
+	/*
+	 * Recursion ends at the topmost level, ie., when there's no parent; also
+	 * when the partition is being detached.
+	 */
+	parentOid = get_partition_parent_worker(inhRel, relid, &detach_pending);
+	if (parentOid == InvalidOid || detach_pending)
 		return;
 
 	*ancestors = lappend_oid(*ancestors, parentOid);
@@ -170,7 +192,7 @@ index_get_partition(Relation partition, Oid indexId)
 		ReleaseSysCache(tup);
 		if (!ispartition)
 			continue;
-		if (get_partition_parent(partIdx) == indexId)
+		if (get_partition_parent(partIdx, false) == indexId)
 		{
 			list_free(idxlist);
 			return partIdx;
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 5ab7902827..bedee069be 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -29,6 +29,7 @@
 #include "utils/builtins.h"
 #include "utils/fmgroids.h"
 #include "utils/memutils.h"
+#include "utils/snapmgr.h"
 #include "utils/syscache.h"
 
 /*
@@ -50,9 +51,14 @@ typedef struct SeenRelsEntry
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * include_detached says to include all partitions, even if they're marked
+ * detached.  Passing it as false means they might or might not be included,
+ * depending on the visibility of the pg_inherits row for the active snapshot.
  */
 List *
-find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool include_detached,
+						  LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -91,6 +97,30 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
 
 	while ((inheritsTuple = systable_getnext(scan)) != NULL)
 	{
+		/*
+		 * Cope with partitions concurrently being detached.  When we see a
+		 * partition marked "detach pending", we only include it in the set of
+		 * visible partitions if caller requested all detached partitions, or
+		 * if its pg_inherits tuple's xmin is still visible to the active
+		 * snapshot.
+		 *
+		 * The reason for this check is that we want to avoid seeing the
+		 * partition as alive in RI queries during REPEATABLE READ or
+		 * SERIALIZABLE transactions.
+		 */
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending &&
+			!include_detached)
+		{
+			TransactionId xmin;
+			Snapshot snap;
+
+			xmin = HeapTupleHeaderGetXmin(inheritsTuple->t_data);
+			snap = GetActiveSnapshot();
+
+			if (!XidInMVCCSnapshot(xmin, snap))
+				continue;
+		}
+
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
 		if (numoids >= maxoids)
 		{
@@ -160,6 +190,9 @@ find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
  * given rel; caller should already have locked it).  If lockmode is NoLock
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
+ *
+ * NB - No current callers of this routine are interested in children being
+ * concurrently detached, so there's no provision to include them.
  */
 List *
 find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
@@ -199,7 +232,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, lockmode);
+		currentchildren = find_inheritance_children(currentrel, false,
+													lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -430,6 +464,7 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
 	values[Anum_pg_inherits_inhrelid - 1] = ObjectIdGetDatum(relationId);
 	values[Anum_pg_inherits_inhparent - 1] = ObjectIdGetDatum(parentOid);
 	values[Anum_pg_inherits_inhseqno - 1] = Int32GetDatum(seqNumber);
+	values[Anum_pg_inherits_inhdetachpending - 1] = BoolGetDatum(false);
 
 	memset(nulls, 0, sizeof(nulls));
 
@@ -449,10 +484,17 @@ StoreSingleInheritance(Oid relationId, Oid parentOid, int32 seqNumber)
  * as InvalidOid, in which case all tuples matching inhrelid are deleted;
  * otherwise only delete tuples with the specified inhparent.
  *
+ * expect_detach_pending is the expected state of the inhdetachpending flag.
+ * If the catalog row does not match that state, an error is raised.
+ *
+ * childname is the partition name, if a table; pass NULL for regular
+ * inheritance or when working with other relation kinds.
+ *
  * Returns whether at least one row was deleted.
  */
 bool
-DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
+DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool expect_detach_pending,
+					const char *childname)
 {
 	bool		found = false;
 	Relation	catalogRelation;
@@ -479,6 +521,29 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
+			bool	detach_pending;
+
+			detach_pending =
+				((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending;
+
+			/*
+			 * Raise error depending on state.  This should only happen for
+			 * partitions, but we have no way to cross-check.
+			 */
+			if (detach_pending && !expect_detach_pending)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot detach partition \"%s\"",
+								childname ? childname : "unknown relation"),
+						 errdetail("The partition is being detached concurrently or has an unfinished detach."),
+						 errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation")));
+			if (!detach_pending && expect_detach_pending)
+				ereport(ERROR,
+						(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+						 errmsg("cannot complete detaching partition \"%s\"",
+								childname ? childname : "unknown relation"),
+						 errdetail("There's no pending concurrent detach.")));
+
 			CatalogTupleDelete(catalogRelation, &inheritsTuple->t_self);
 			found = true;
 		}
@@ -490,3 +555,46 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent)
 
 	return found;
 }
+
+/*
+ * Return whether the pg_inherits tuple for a partition has the "detach
+ * pending" flag set.
+ */
+bool
+PartitionHasPendingDetach(Oid partoid)
+{
+	Relation	catalogRelation;
+	ScanKeyData key;
+	SysScanDesc scan;
+	HeapTuple	inheritsTuple;
+
+	/* We don't have a good way to verify it is in fact a partition */
+
+	/*
+	 * Find the pg_inherits entry by inhrelid.  (There should only be one.)
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(partoid));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		bool	detached;
+
+		detached =
+			((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending;
+
+		/* Done */
+		systable_endscan(scan);
+		table_close(catalogRelation, RowExclusiveLock);
+
+		return detached;
+	}
+
+	elog(ERROR, "relation %u is not a partition", partoid);
+	return false;				/* keep compiler quiet */
+}
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index e4e1bbb7e0..166374cc0c 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -410,7 +410,7 @@ CompareOpclassOptions(Datum *opts1, Datum *opts2, int natts)
  * GetCurrentVirtualXIDs.  If, during any iteration, a particular vxid
  * doesn't show up in the output, we know we can forget about it.
  */
-static void
+void
 WaitForOlderSnapshots(TransactionId limitXmin, bool progress)
 {
 	int			n_old_snapshots;
@@ -1123,7 +1123,7 @@ DefineIndex(Oid relationId,
 	 */
 	if (partitioned && stmt->relation && !stmt->relation->inh)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
@@ -1180,7 +1180,7 @@ DefineIndex(Oid relationId,
 		 *
 		 * If we're called internally (no stmt->relation), recurse always.
 		 */
-		partdesc = RelationGetPartitionDesc(rel);
+		partdesc = RelationGetPartitionDesc(rel, false);
 		if ((!stmt->relation || stmt->relation->inh) && partdesc->nparts > 0)
 		{
 			int			nparts = partdesc->nparts;
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index bf7fd6e8ae..0966636fb2 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -544,7 +544,8 @@ static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partsp
 static void ComputePartitionAttrs(ParseState *pstate, Relation rel, List *partParams, AttrNumber *partattrs,
 								  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
-static void RemoveInheritance(Relation child_rel, Relation parent_rel);
+static void RemoveInheritance(Relation child_rel, Relation parent_rel,
+							  bool allow_detached);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
 										   PartitionCmd *cmd,
 										   AlterTableUtilityContext *context);
@@ -553,8 +554,14 @@ static void QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 											   List *partConstraint,
 											   bool validate_default);
 static void CloneRowTriggersToPartition(Relation parent, Relation partition);
+static void DetachAddConstraintIfNeeded(List **wqueue, Relation partRel);
 static void DropClonedTriggersFromPartition(Oid partitionId);
-static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static ObjectAddress ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab,
+										   Relation rel, RangeVar *name,
+										   bool concurrent);
+static void DetachPartitionFinalize(Relation rel, Relation partRel,
+									bool concurrent, Oid defaultPartOid, bool wait);
+static ObjectAddress ATExecDetachPartitionFinalize(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
 											  RangeVar *name);
 static void validatePartitionedIndex(Relation partedIdx, Relation partedTbl);
@@ -1002,7 +1009,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		 * lock the partition so as to avoid a deadlock.
 		 */
 		defaultPartOid =
-			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+			get_default_oid_from_partdesc(RelationGetPartitionDesc(parent,
+																   false));
 		if (OidIsValid(defaultPartOid))
 			defaultRel = table_open(defaultPartOid, AccessExclusiveLock);
 
@@ -1555,7 +1563,7 @@ RangeVarCallbackForDropRelation(const RangeVar *rel, Oid relOid, Oid oldRelOid,
 	 */
 	if (is_partition && relOid != oldRelOid)
 	{
-		state->partParentOid = get_partition_parent(relOid);
+		state->partParentOid = get_partition_parent(relOid, true);
 		if (OidIsValid(state->partParentOid))
 			LockRelationOid(state->partParentOid, AccessExclusiveLock);
 	}
@@ -3315,7 +3323,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3514,7 +3522,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -4134,7 +4142,14 @@ AlterTableGetLockLevel(List *cmds)
 				break;
 
 			case AT_DetachPartition:
-				cmd_lockmode = AccessExclusiveLock;
+				if (((PartitionCmd *) cmd->def)->concurrent)
+					cmd_lockmode = ShareUpdateExclusiveLock;
+				else
+					cmd_lockmode = AccessExclusiveLock;
+				break;
+
+			case AT_DetachPartitionFinalize:
+				cmd_lockmode = ShareUpdateExclusiveLock;
 				break;
 
 			case AT_CheckNotNull:
@@ -4218,6 +4233,19 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 	/* Find or create work queue entry for this table */
 	tab = ATGetQueueEntry(wqueue, rel);
 
+	/*
+	 * Disallow any ALTER TABLE other than ALTER TABLE DETACH FINALIZE on
+	 * partitions that are pending detach.
+	 */
+	if (rel->rd_rel->relispartition &&
+		cmd->subtype != AT_DetachPartitionFinalize &&
+		PartitionHasPendingDetach(RelationGetRelid(rel)))
+		ereport(ERROR,
+				errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				errmsg("cannot alter partition \"%s\" with an incomplete detach",
+					   RelationGetRelationName(rel)),
+				errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the pending detach operation."));
+
 	/*
 	 * Copy the original subcommand for each table.  This avoids conflicts
 	 * when different child tables need to make different parse
@@ -4531,6 +4559,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_DetachPartitionFinalize:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -4920,7 +4953,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
 			Assert(cmd != NULL);
 			/* ATPrepCmd ensures it must be a table */
 			Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
-			ATExecDetachPartition(rel, ((PartitionCmd *) cmd->def)->name);
+			ATExecDetachPartition(wqueue, tab, rel,
+								  ((PartitionCmd *) cmd->def)->name,
+								  ((PartitionCmd *) cmd->def)->concurrent);
+			break;
+		case AT_DetachPartitionFinalize:
+			ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
 			break;
 		case AT_AlterCollationRefreshVersion:
 			/* ATPrepCmd ensured it must be an index */
@@ -6352,7 +6390,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6597,7 +6635,8 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -6751,7 +6790,7 @@ ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
 	 */
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 
 		Assert(partdesc != NULL);
 		if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -6850,7 +6889,7 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	/* If rel is partition, shouldn't drop NOT NULL if parent has the same */
 	if (rel->rd_rel->relispartition)
 	{
-		Oid			parentId = get_partition_parent(RelationGetRelid(rel));
+		Oid			parentId = get_partition_parent(RelationGetRelid(rel), false);
 		Relation	parent = table_open(parentId, AccessShareLock);
 		TupleDesc	tupDesc = RelationGetDescr(parent);
 		AttrNumber	parent_attnum;
@@ -7460,7 +7499,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), lockmode))
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8067,7 +8106,8 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	if (children)
 	{
@@ -8531,7 +8571,8 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * routines, we have to do this one level of recursion at a time; we can't
 	 * use find_all_inheritors to do it in one pass.
 	 */
-	children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+	children =
+		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -9146,7 +9187,7 @@ addFkRecurseReferenced(List **wqueue, Constraint *fkconstraint, Relation rel,
 	 */
 	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(pkrel);
+		PartitionDesc pd = RelationGetPartitionDesc(pkrel, false);
 
 		for (int i = 0; i < pd->nparts; i++)
 		{
@@ -9280,7 +9321,7 @@ addFkRecurseReferencing(List **wqueue, Constraint *fkconstraint, Relation rel,
 	}
 	else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc pd = RelationGetPartitionDesc(rel);
+		PartitionDesc pd = RelationGetPartitionDesc(rel, false);
 
 		/*
 		 * Recurse to take appropriate action on each partition; either we
@@ -11064,7 +11105,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
+		children =
+			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
 	else
 		children = NIL;
 
@@ -11448,7 +11490,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), false,
+									   NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14286,7 +14329,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	 */
 
 	/* Off to RemoveInheritance() where most of the work happens */
-	RemoveInheritance(rel, parent_rel);
+	RemoveInheritance(rel, parent_rel, false);
 
 	ObjectAddressSet(address, RelationRelationId,
 					 RelationGetRelid(parent_rel));
@@ -14297,12 +14340,72 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
 	return address;
 }
 
+/*
+ * MarkInheritDetached
+ *
+ * Set inhdetachpending for a partition, for ATExecDetachPartition
+ * in concurrent mode.
+ */
+static void
+MarkInheritDetached(Relation child_rel, Relation parent_rel)
+{
+	Relation	catalogRelation;
+	SysScanDesc	scan;
+	ScanKeyData key;
+	HeapTuple	inheritsTuple;
+	bool		found = false;
+
+	Assert(child_rel->rd_rel->relkind == RELKIND_RELATION ||
+		   child_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
+	Assert(parent_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
+
+	/*
+	 * Find pg_inherits entries by inhrelid.
+	 */
+	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
+	ScanKeyInit(&key,
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationGetRelid(child_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+							  true, NULL, 1, &key);
+
+	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
+	{
+		HeapTuple	newtup;
+
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
+			RelationGetRelid(parent_rel))
+			elog(ERROR, "bad parent tuple found for partition %u",
+				 RelationGetRelid(child_rel));
+
+		newtup = heap_copytuple(inheritsTuple);
+		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+
+		CatalogTupleUpdate(catalogRelation,
+						   &inheritsTuple->t_self,
+						   newtup);
+		found = true;
+	}
+
+	/* Done */
+	systable_endscan(scan);
+	table_close(catalogRelation, RowExclusiveLock);
+
+	if (!found)
+		ereport(ERROR,
+				(errcode(ERRCODE_UNDEFINED_TABLE),
+				 errmsg("relation \"%s\" is not a partition of relation \"%s\"",
+						RelationGetRelationName(child_rel),
+						RelationGetRelationName(parent_rel))));
+}
+
 /*
  * RemoveInheritance
  *
  * Drop a parent from the child's parents. This just adjusts the attinhcount
  * and attislocal of the columns and removes the pg_inherit and pg_depend
- * entries.
+ * entries.  expect_detached is passed down to DeleteInheritsTuple, q.v..
  *
  * If attinhcount goes to 0 then attislocal gets set to true. If it goes back
  * up attislocal stays true, which means if a child is ever removed from a
@@ -14316,7 +14419,7 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * Common to ATExecDropInherit() and ATExecDetachPartition().
  */
 static void
-RemoveInheritance(Relation child_rel, Relation parent_rel)
+RemoveInheritance(Relation child_rel, Relation parent_rel, bool expect_detached)
 {
 	Relation	catalogRelation;
 	SysScanDesc scan;
@@ -14332,7 +14435,9 @@ RemoveInheritance(Relation child_rel, Relation parent_rel)
 		child_is_partition = true;
 
 	found = DeleteInheritsTuple(RelationGetRelid(child_rel),
-								RelationGetRelid(parent_rel));
+								RelationGetRelid(parent_rel),
+								expect_detached,
+								RelationGetRelationName(child_rel));
 	if (!found)
 	{
 		if (child_is_partition)
@@ -16498,7 +16603,7 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
 	}
 	else if (scanrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(scanrel, false);
 		int			i;
 
 		for (i = 0; i < partdesc->nparts; i++)
@@ -16558,7 +16663,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
 	 * new partition will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
@@ -17147,105 +17252,229 @@ CloneRowTriggersToPartition(Relation parent, Relation partition)
  * ALTER TABLE DETACH PARTITION
  *
  * Return the address of the relation that is no longer a partition of rel.
+ *
+ * If concurrent mode is requested, we run in two transactions.  A side-
+ * effect is that this command cannot run in a multi-part ALTER TABLE.
+ * Currently, that's enforced by the grammar.
+ *
+ * The strategy for concurrency is to first modify the partition's
+ * pg_inherit catalog row to make it visible to everyone that the
+ * partition is detached, lock the partition against writes, and commit
+ * the transaction; anyone who requests the partition descriptor from
+ * that point onwards has to ignore such a partition.  In a second
+ * transaction, we wait until all transactions that could have seen the
+ * partition as attached are gone, then we remove the rest of partition
+ * metadata (pg_inherits and pg_class.relpartbounds).
  */
 static ObjectAddress
-ATExecDetachPartition(Relation rel, RangeVar *name)
+ATExecDetachPartition(List **wqueue, AlteredTableInfo *tab, Relation rel,
+					  RangeVar *name, bool concurrent)
 {
-	Relation	partRel,
-				classRel;
-	HeapTuple	tuple,
-				newtuple;
-	Datum		new_val[Natts_pg_class];
-	bool		new_null[Natts_pg_class],
-				new_repl[Natts_pg_class];
+	Relation	partRel;
 	ObjectAddress address;
 	Oid			defaultPartOid;
-	List	   *indexes;
-	List	   *fks;
-	ListCell   *cell;
 
 	/*
 	 * We must lock the default partition, because detaching this partition
 	 * will change its partition constraint.
 	 */
 	defaultPartOid =
-		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
-	if (OidIsValid(defaultPartOid))
-		LockRelationOid(defaultPartOid, AccessExclusiveLock);
-
-	partRel = table_openrv(name, ShareUpdateExclusiveLock);
-
-	/* Ensure that foreign keys still hold after this detach */
-	ATDetachCheckNoForeignKeyRefs(partRel);
-
-	/* All inheritance related checks are performed within the function */
-	RemoveInheritance(partRel, rel);
-
-	/* Update pg_class tuple */
-	classRel = table_open(RelationRelationId, RowExclusiveLock);
-	tuple = SearchSysCacheCopy1(RELOID,
-								ObjectIdGetDatum(RelationGetRelid(partRel)));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(partRel));
-	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
-
-	/* Clear relpartbound and reset relispartition */
-	memset(new_val, 0, sizeof(new_val));
-	memset(new_null, false, sizeof(new_null));
-	memset(new_repl, false, sizeof(new_repl));
-	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
-	new_null[Anum_pg_class_relpartbound - 1] = true;
-	new_repl[Anum_pg_class_relpartbound - 1] = true;
-	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
-								 new_val, new_null, new_repl);
-
-	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
-	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
-	heap_freetuple(newtuple);
-
+		get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, false));
 	if (OidIsValid(defaultPartOid))
 	{
 		/*
-		 * If the relation being detached is the default partition itself,
-		 * remove it from the parent's pg_partitioned_table entry.
+		 * Concurrent detaching when a default partition exists is not
+		 * supported. The main problem is that the default partition
+		 * constraint would change.  And there's a definitional problem: what
+		 * should happen to the tuples that are being inserted that belong to
+		 * the partition being detached?  Putting them on the partition being
+		 * detached would be wrong, since they'd become "lost" after the but
+		 * we cannot put them in the default partition either until we alter
+		 * its partition constraint.
 		 *
-		 * If not, we must invalidate default partition's relcache entry, as
-		 * in StorePartitionBound: its partition constraint depends on every
-		 * other partition's partition constraint.
+		 * I think we could solve this problem if we effected the constraint
+		 * change before committing the first transaction.  But the lock would
+		 * have to remain AEL and it would cause concurrent query planning to
+		 * be blocked, so changing it that way would be even worse.
 		 */
-		if (RelationGetRelid(partRel) == defaultPartOid)
-			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
-		else
-			CacheInvalidateRelcacheByRelid(defaultPartOid);
+		if (concurrent)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot detach partitions concurrently when a default partition exists")));
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
-	/* detach indexes too */
-	indexes = RelationGetIndexList(partRel);
-	foreach(cell, indexes)
+	/*
+	 * In concurrent mode, the partition is locked with share-update-exclusive
+	 * in the first transaction.  This allows concurrent transactions to be
+	 * doing DML to the partition.
+	 */
+	partRel = table_openrv(name, concurrent ? ShareUpdateExclusiveLock :
+						   AccessExclusiveLock);
+
+	/*
+	 * Check inheritance conditions and either delete the pg_inherits row
+	 * (in non-concurrent mode) or just set the inhdetachpending flag.
+	 */
+	if (!concurrent)
+		RemoveInheritance(partRel, rel, false);
+	else
+		MarkInheritDetached(partRel, rel);
+
+	/*
+	 * Ensure that foreign keys still hold after this detach.  This keeps
+	 * locks on the referencing tables, which prevents concurrent transactions
+	 * from adding rows that we wouldn't see.  For this to work in concurrent
+	 * mode, it is critical that the partition appears as no longer attached
+	 * for the RI queries as soon as the first transaction commits.
+	 */
+	ATDetachCheckNoForeignKeyRefs(partRel);
+
+	/*
+	 * Concurrent mode has to work harder; first we add a new constraint to
+	 * the partition that matches the partition constraint.  Then we close our
+	 * existing transaction, and in a new one wait for all processes to catch
+	 * up on the catalog updates we've done so far; at that point we can
+	 * complete the operation.
+	 */
+	if (concurrent)
 	{
-		Oid			idxid = lfirst_oid(cell);
-		Relation	idx;
-		Oid			constrOid;
+		Oid		partrelid,
+				parentrelid;
+		LOCKTAG		tag;
+		char   *parentrelname;
+		char   *partrelname;
 
-		if (!has_superclass(idxid))
-			continue;
+		/*
+		 * Add a new constraint to the partition being detached, which
+		 * supplants the partition constraint (unless there is one already).
+		 */
+		DetachAddConstraintIfNeeded(wqueue, partRel);
 
-		Assert((IndexGetRelation(get_partition_parent(idxid), false) ==
-				RelationGetRelid(rel)));
+		/*
+		 * We're almost done now; the only traces that remain are the
+		 * pg_inherits tuple and the partition's relpartbounds.  Before we can
+		 * remove those, we need to wait until all transactions that know that
+		 * this is a partition are gone.
+		 */
 
-		idx = index_open(idxid, AccessExclusiveLock);
-		IndexSetParentIndex(idx, InvalidOid);
+		/*
+		 * Remember relation OIDs to re-acquire them later; and relation names
+		 * too, for error messages if something is dropped in between.
+		 */
+		partrelid = RelationGetRelid(partRel);
+		parentrelid = RelationGetRelid(rel);
+		parentrelname = MemoryContextStrdup(PortalContext,
+											RelationGetRelationName(rel));
+		partrelname = MemoryContextStrdup(PortalContext,
+										  RelationGetRelationName(partRel));
 
-		/* If there's a constraint associated with the index, detach it too */
-		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
-													idxid);
-		if (OidIsValid(constrOid))
-			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+		/* Invalidate relcache entries for the parent -- must be before close */
+		CacheInvalidateRelcache(rel);
+
+		table_close(partRel, NoLock);
+		table_close(rel, NoLock);
+		tab->rel = NULL;
+
+		/* Make updated catalog entry visible */
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+
+		StartTransactionCommand();
+
+		/*
+		 * Now wait.  This ensures that all queries that were planned including
+		 * the partition are finished before we remove the rest of catalog
+		 * entries.  We don't need or indeed want to acquire this lock, though
+		 * -- that would block later queries.
+		 *
+		 * We don't need to concern ourselves with waiting for a lock on the
+		 * partition itself, since we will acquire AccessExclusiveLock below.
+		 */
+		SET_LOCKTAG_RELATION(tag, MyDatabaseId, parentrelid);
+		WaitForLockersMultiple(list_make1(&tag), AccessExclusiveLock, false);
+
+		/*
+		 * Now acquire locks in both relations again.  Note they may have been
+		 * removed in the meantime, so care is required.
+		 */
+		rel = try_relation_open(parentrelid, ShareUpdateExclusiveLock);
+		partRel = try_relation_open(partrelid, AccessExclusiveLock);
+
+		/* If the relations aren't there, something bad happened; bail out */
+		if (rel == NULL)
+		{
+			if (partRel != NULL)	/* shouldn't happen */
+				elog(WARNING, "dangling partition \"%s\" remains, can't fix",
+					 partrelname);
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partitioned table \"%s\" was removed concurrently",
+							parentrelname)));
+		}
+		if (partRel == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("partition \"%s\" was removed concurrently", partrelname)));
+
+		tab->rel = rel;
 
-		index_close(idx, NoLock);
 	}
-	table_close(classRel, RowExclusiveLock);
+
+	/* Do the final part of detaching */
+	DetachPartitionFinalize(rel, partRel, concurrent, defaultPartOid, false);
+
+	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
+
+	/* keep our lock until commit */
+	table_close(partRel, NoLock);
+
+	return address;
+}
+
+/*
+ * Second part of ALTER TABLE .. DETACH.
+ *
+ * This is separate so that it can be run independently when the second
+ * transaction of the concurrent algorithm fails (crash or abort).
+ */
+static void
+DetachPartitionFinalize(Relation rel, Relation partRel, bool concurrent,
+						Oid defaultPartOid, bool wait)
+{
+	Relation	classRel;
+	List	   *fks;
+	ListCell   *cell;
+	List	   *indexes;
+	Datum		new_val[Natts_pg_class];
+	bool		new_null[Natts_pg_class],
+				new_repl[Natts_pg_class];
+	HeapTuple	tuple,
+				newtuple;
+
+	/*
+	 * If asked to, wait until existing snapshots are gone.  This is important
+	 * if the second transaction of DETACH PARTITION CONCURRENTLY is canceled:
+	 * the user could immediately run DETACH FINALIZE without actually waiting
+	 * for existing transactions.  We must not complete the detach action until
+	 * all such queries are complete (otherwise we would present them with an
+	 * inconsistent view of catalogs).
+	 */
+	if (wait)
+	{
+		Snapshot	snap = GetActiveSnapshot();
+
+		WaitForOlderSnapshots(snap->xmin, false);
+	}
+
+	if (concurrent)
+	{
+		/*
+		 * We can remove the pg_inherits row now. (In the non-concurrent case,
+		 * this was already done).
+		 */
+		RemoveInheritance(partRel, rel, true);
+	}
 
 	/* Drop any triggers that were cloned on creation/attach. */
 	DropClonedTriggersFromPartition(RelationGetRelid(partRel));
@@ -17318,22 +17547,150 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 		ObjectAddressSet(constraint, ConstraintRelationId, constrOid);
 		performDeletion(&constraint, DROP_RESTRICT, 0);
 	}
-	CommandCounterIncrement();
+
+	/* Now we can detach indexes */
+	indexes = RelationGetIndexList(partRel);
+	foreach(cell, indexes)
+	{
+		Oid			idxid = lfirst_oid(cell);
+		Relation	idx;
+		Oid			constrOid;
+
+		if (!has_superclass(idxid))
+			continue;
+
+		Assert((IndexGetRelation(get_partition_parent(idxid, false), false) ==
+				RelationGetRelid(rel)));
+
+		idx = index_open(idxid, AccessExclusiveLock);
+		IndexSetParentIndex(idx, InvalidOid);
+
+		/* If there's a constraint associated with the index, detach it too */
+		constrOid = get_relation_idx_constraint_oid(RelationGetRelid(partRel),
+													idxid);
+		if (OidIsValid(constrOid))
+			ConstraintSetParentConstraint(constrOid, InvalidOid, InvalidOid);
+
+		index_close(idx, NoLock);
+	}
+
+	/* Update pg_class tuple */
+	classRel = table_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID,
+								ObjectIdGetDatum(RelationGetRelid(partRel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(partRel));
+	Assert(((Form_pg_class) GETSTRUCT(tuple))->relispartition);
+
+	/* Clear relpartbound and reset relispartition */
+	memset(new_val, 0, sizeof(new_val));
+	memset(new_null, false, sizeof(new_null));
+	memset(new_repl, false, sizeof(new_repl));
+	new_val[Anum_pg_class_relpartbound - 1] = (Datum) 0;
+	new_null[Anum_pg_class_relpartbound - 1] = true;
+	new_repl[Anum_pg_class_relpartbound - 1] = true;
+	newtuple = heap_modify_tuple(tuple, RelationGetDescr(classRel),
+								 new_val, new_null, new_repl);
+
+	((Form_pg_class) GETSTRUCT(newtuple))->relispartition = false;
+	CatalogTupleUpdate(classRel, &newtuple->t_self, newtuple);
+	heap_freetuple(newtuple);
+	table_close(classRel, RowExclusiveLock);
+
+	if (OidIsValid(defaultPartOid))
+	{
+		/*
+		 * If the relation being detached is the default partition itself,
+		 * remove it from the parent's pg_partitioned_table entry.
+		 *
+		 * If not, we must invalidate default partition's relcache entry, as
+		 * in StorePartitionBound: its partition constraint depends on every
+		 * other partition's partition constraint.
+		 */
+		if (RelationGetRelid(partRel) == defaultPartOid)
+			update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+		else
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+	}
 
 	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
 	CacheInvalidateRelcache(rel);
+}
+
+/*
+ * ALTER TABLE ... DETACH PARTITION ... FINALIZE
+ *
+ * To use when a DETACH PARTITION command previously did not run to
+ * completion; this completes the detaching process.
+ */
+static ObjectAddress
+ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
+{
+	Relation    partRel;
+	ObjectAddress address;
+
+	partRel = table_openrv(name, AccessExclusiveLock);
+
+	DetachPartitionFinalize(rel, partRel, true, InvalidOid, true);
 
 	ObjectAddressSet(address, RelationRelationId, RelationGetRelid(partRel));
 
-	/* keep our lock until commit */
 	table_close(partRel, NoLock);
 
 	return address;
 }
 
+/*
+ * DetachAddConstraintIfNeeded
+ *		Subroutine for ATExecDetachPartition.  Create a constraint that
+ *		takes the place of the partition constraint, but avoid creating
+ *		a dupe if an equivalent constraint already exists.
+ */
+static void
+DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
+{
+	AlteredTableInfo *tab;
+	Expr	   *constraintExpr;
+	TupleDesc	td = RelationGetDescr(partRel);
+	Constraint *n;
+
+	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+
+	/* If an identical constraint exists, we don't need to create one */
+	if (td->constr && td->constr->num_check > 0)
+	{
+		for (int i = 0; i < td->constr->num_check; i++)
+		{
+			Node	*thisconstr;
+
+			thisconstr = stringToNode(td->constr->check[i].ccbin);
+
+			if (equal(constraintExpr, thisconstr))
+				return;
+		}
+	}
+
+	tab = ATGetQueueEntry(wqueue, partRel);
+
+	/* Add constraint on partition, equivalent to the partition constraint */
+	n = makeNode(Constraint);
+	n->contype = CONSTR_CHECK;
+	n->conname = NULL;
+	n->location = -1;
+	n->is_no_inherit = false;
+	n->raw_expr = NULL;
+	n->cooked_expr = nodeToString(constraintExpr);
+	n->initially_valid = true;
+	n->skip_validation = true;
+	/* It's a re-add, since it nominally already exists */
+	ATAddCheckConstraint(wqueue, tab, partRel, n,
+						 true, false, true, ShareUpdateExclusiveLock);
+}
+
 /*
  * DropClonedTriggersFromPartition
  *		subroutine for ATExecDetachPartition to remove any triggers that were
@@ -17501,7 +17858,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 
 	/* Silently do nothing if already in the right state */
 	currParent = partIdx->rd_rel->relispartition ?
-		get_partition_parent(partIdxId) : InvalidOid;
+		get_partition_parent(partIdxId, false) : InvalidOid;
 	if (currParent != RelationGetRelid(parentIdx))
 	{
 		IndexInfo  *childInfo;
@@ -17529,7 +17886,7 @@ ATExecAttachPartitionIdx(List **wqueue, Relation parentIdx, RangeVar *name)
 							   RelationGetRelationName(partIdx))));
 
 		/* Make sure it indexes a partition of the other index's table */
-		partDesc = RelationGetPartitionDesc(parentTbl);
+		partDesc = RelationGetPartitionDesc(parentTbl, false);
 		found = false;
 		for (i = 0; i < partDesc->nparts; i++)
 		{
@@ -17683,7 +18040,7 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 	 * If we found as many inherited indexes as the partitioned table has
 	 * partitions, we're good; update pg_index to set indisvalid.
 	 */
-	if (tuples == RelationGetPartitionDesc(partedTbl)->nparts)
+	if (tuples == RelationGetPartitionDesc(partedTbl, false)->nparts)
 	{
 		Relation	idxRel;
 		HeapTuple	newtup;
@@ -17713,8 +18070,8 @@ validatePartitionedIndex(Relation partedIdx, Relation partedTbl)
 		/* make sure we see the validation we just did */
 		CommandCounterIncrement();
 
-		parentIdxId = get_partition_parent(RelationGetRelid(partedIdx));
-		parentTblId = get_partition_parent(RelationGetRelid(partedTbl));
+		parentIdxId = get_partition_parent(RelationGetRelid(partedIdx), false);
+		parentTblId = get_partition_parent(RelationGetRelid(partedTbl), false);
 		parentIdx = relation_open(parentIdxId, AccessExclusiveLock);
 		parentTbl = relation_open(parentTblId, AccessExclusiveLock);
 		Assert(!parentIdx->rd_index->indisvalid);
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 4e4e05844c..7383d5994e 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1119,7 +1119,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	 */
 	if (partition_recurse)
 	{
-		PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+		PartitionDesc partdesc = RelationGetPartitionDesc(rel, false);
 		List	   *idxs = NIL;
 		List	   *childTbls = NIL;
 		ListCell   *l;
@@ -1141,7 +1141,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, false,
+											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index b8da4c5967..619aaffae4 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -569,6 +569,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 					  int partidx)
 {
 	ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
+	Oid			partOid = dispatch->partdesc->oids[partidx];
 	Relation	partrel;
 	int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
 	Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
@@ -579,7 +580,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate, EState *estate,
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
-	partrel = table_open(dispatch->partdesc->oids[partidx], RowExclusiveLock);
+	partrel = table_open(partOid, RowExclusiveLock);
 
 	leaf_part_rri = makeNode(ResultRelInfo);
 	InitResultRelInfo(leaf_part_rri,
@@ -1065,9 +1066,21 @@ ExecInitPartitionDispatchInfo(EState *estate,
 	int			dispatchidx;
 	MemoryContext oldcxt;
 
+	/*
+	 * For data modification, it is better that executor does not include
+	 * partitions being detached, except in snapshot-isolation mode.  This
+	 * means that a read-committed transaction immediately gets a "no
+	 * partition for tuple" error when a tuple is inserted into a partition
+	 * that's being detached concurrently, but a transaction in repeatable-
+	 * read mode can still use the partition.  Note that because partition
+	 * detach uses ShareLock on the partition (which conflicts with DML),
+	 * we're certain that the detach won't be able to complete until any
+	 * inserting transaction is done.
+	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt,
+									 IsolationUsesXactSnapshot());
 
 	oldcxt = MemoryContextSwitchTo(proute->memcxt);
 
@@ -1645,9 +1658,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
+	/* Executor must always include detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
-			CreatePartitionDirectory(estate->es_query_cxt);
+			CreatePartitionDirectory(estate->es_query_cxt, true);
 
 	n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
 	Assert(n_part_hierarchies > 0);
@@ -1713,9 +1727,12 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 												partrel);
 
 			/*
-			 * Initialize the subplan_map and subpart_map.  Since detaching a
-			 * partition requires AccessExclusiveLock, no partitions can have
-			 * disappeared, nor can the bounds for any partition have changed.
+			 * Initialize the subplan_map and subpart_map.
+			 *
+			 * Because we request detached partitions to be included, and
+			 * detaching waits for old transactions, it is safe to assume that
+			 * no partitions have disappeared since this query was planned.
+			 *
 			 * However, new partitions may have been added.
 			 */
 			Assert(partdesc->nparts >= pinfo->nparts);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 82d7cce5d5..38b56231b7 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4737,6 +4737,7 @@ _copyPartitionCmd(const PartitionCmd *from)
 
 	COPY_NODE_FIELD(name);
 	COPY_NODE_FIELD(bound);
+	COPY_SCALAR_FIELD(concurrent);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 3e980c457c..3292dda342 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2975,6 +2975,7 @@ _equalPartitionCmd(const PartitionCmd *a, const PartitionCmd *b)
 {
 	COMPARE_NODE_FIELD(name);
 	COMPARE_NODE_FIELD(bound);
+	COMPARE_SCALAR_FIELD(concurrent);
 
 	return true;
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 7f2e40ae39..6c39bf893f 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2141,10 +2141,14 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 {
 	PartitionDesc partdesc;
 
-	/* Create the PartitionDirectory infrastructure if we didn't already */
+	/*
+	 * Create the PartitionDirectory infrastructure if we didn't already.
+	 */
 	if (root->glob->partition_directory == NULL)
+	{
 		root->glob->partition_directory =
-			CreatePartitionDirectory(CurrentMemoryContext);
+			CreatePartitionDirectory(CurrentMemoryContext, false);
+	}
 
 	partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
 										relation);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 4843ff99e2..2132cf4d82 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -658,7 +658,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	EXCLUDE EXCLUDING EXCLUSIVE EXECUTE EXISTS EXPLAIN EXPRESSION
 	EXTENSION EXTERNAL EXTRACT
 
-	FALSE_P FAMILY FETCH FILTER FIRST_P FLOAT_P FOLLOWING FOR
+	FALSE_P FAMILY FETCH FILTER FINALIZE FIRST_P FLOAT_P FOLLOWING FOR
 	FORCE FOREIGN FORWARD FREEZE FROM FULL FUNCTION FUNCTIONS
 
 	GENERATED GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING GROUPS
@@ -2108,12 +2108,13 @@ partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = $4;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
 				}
-			/* ALTER TABLE <name> DETACH PARTITION <partition_name> */
-			| DETACH PARTITION qualified_name
+			/* ALTER TABLE <name> DETACH PARTITION <partition_name> [CONCURRENTLY] */
+			| DETACH PARTITION qualified_name opt_concurrently
 				{
 					AlterTableCmd *n = makeNode(AlterTableCmd);
 					PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2121,8 +2122,21 @@ partition_cmd:
 					n->subtype = AT_DetachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = $4;
 					n->def = (Node *) cmd;
 
+					$$ = (Node *) n;
+				}
+			| DETACH PARTITION qualified_name FINALIZE
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionCmd *cmd = makeNode(PartitionCmd);
+
+					n->subtype = AT_DetachPartitionFinalize;
+					cmd->name = $3;
+					cmd->bound = NULL;
+					cmd->concurrent = false;
+					n->def = (Node *) cmd;
 					$$ = (Node *) n;
 				}
 		;
@@ -2137,6 +2151,7 @@ index_partition_cmd:
 					n->subtype = AT_AttachPartition;
 					cmd->name = $3;
 					cmd->bound = NULL;
+					cmd->concurrent = false;
 					n->def = (Node *) cmd;
 
 					$$ = (Node *) n;
@@ -15395,6 +15410,7 @@ unreserved_keyword:
 			| EXTERNAL
 			| FAMILY
 			| FILTER
+			| FINALIZE
 			| FIRST_P
 			| FOLLOWING
 			| FORCE
@@ -15936,6 +15952,7 @@ bare_label_keyword:
 			| EXTRACT
 			| FALSE_P
 			| FAMILY
+			| FINALIZE
 			| FIRST_P
 			| FLOAT_P
 			| FOLLOWING
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index e5f3482d52..2b2b1dc1ad 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2798,7 +2798,7 @@ check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec, ParseState *pstate)
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent, true);
 	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	int			with = -1;
 	bool		overlap = false;
@@ -3990,7 +3990,7 @@ get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 	{
 		int			i;
 		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
 		if (boundinfo)
@@ -4190,7 +4190,7 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 	if (spec->is_default)
 	{
 		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent, true);	/* XXX correct? */
 		Oid		   *inhoids = pdesc->oids;
 		int			nparts = pdesc->nparts,
 					i;
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index f852b6e99d..58570fecfd 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -37,6 +37,7 @@ typedef struct PartitionDirectoryData
 {
 	MemoryContext pdir_mcxt;
 	HTAB	   *pdir_hash;
+	bool		include_detached;
 }			PartitionDirectoryData;
 
 typedef struct PartitionDirectoryEntry
@@ -46,7 +47,7 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel);
+static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 
 
 /*
@@ -61,13 +62,14 @@ static void RelationBuildPartitionDesc(Relation rel);
  * that the data doesn't become stale.
  */
 PartitionDesc
-RelationGetPartitionDesc(Relation rel)
+RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
 		return NULL;
 
-	if (unlikely(rel->rd_partdesc == NULL))
-		RelationBuildPartitionDesc(rel);
+	if (unlikely(rel->rd_partdesc == NULL ||
+				 rel->rd_partdesc->includes_detached != include_detached))
+		RelationBuildPartitionDesc(rel, include_detached);
 
 	return rel->rd_partdesc;
 }
@@ -88,7 +90,7 @@ RelationGetPartitionDesc(Relation rel)
  * permanently.
  */
 static void
-RelationBuildPartitionDesc(Relation rel)
+RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
 	PartitionBoundInfo boundinfo = NULL;
@@ -110,7 +112,8 @@ RelationBuildPartitionDesc(Relation rel)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
+										NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -238,6 +241,7 @@ RelationBuildPartitionDesc(Relation rel)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
+		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -280,7 +284,7 @@ RelationBuildPartitionDesc(Relation rel)
  *		Create a new partition directory object.
  */
 PartitionDirectory
-CreatePartitionDirectory(MemoryContext mcxt)
+CreatePartitionDirectory(MemoryContext mcxt, bool include_detached)
 {
 	MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
 	PartitionDirectory pdir;
@@ -295,6 +299,7 @@ CreatePartitionDirectory(MemoryContext mcxt)
 
 	pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
 								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+	pdir->include_detached = include_detached;
 
 	MemoryContextSwitchTo(oldcontext);
 	return pdir;
@@ -327,7 +332,7 @@ PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
 		 */
 		RelationIncrementReferenceCount(rel);
 		pde->rel = rel;
-		pde->pd = RelationGetPartitionDesc(rel);
+		pde->pd = RelationGetPartitionDesc(rel, pdir->include_detached);
 		Assert(pde->pd != NULL);
 	}
 	return pde->pd;
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 05bb698cf4..729274b330 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1236,6 +1236,25 @@ ProcessUtilitySlow(ParseState *pstate,
 					AlterTableStmt *atstmt = (AlterTableStmt *) parsetree;
 					Oid			relid;
 					LOCKMODE	lockmode;
+					ListCell   *cell;
+
+					/*
+					 * Disallow ALTER TABLE .. DETACH CONCURRENTLY in a
+					 * transaction block or function.  (Perhaps it could be
+					 * allowed in a procedure, but don't hold your breath.)
+					 */
+					foreach(cell, atstmt->cmds)
+					{
+						AlterTableCmd *cmd = (AlterTableCmd *) lfirst(cell);
+
+						/* Disallow DETACH CONCURRENTLY in a transaction block */
+						if (cmd->subtype == AT_DetachPartition)
+						{
+							if (((PartitionCmd *) cmd->def)->concurrent)
+								PreventInTransactionBlock(isTopLevel,
+														  "ALTER TABLE ... DETACH CONCURRENTLY");
+						}
+					}
 
 					/*
 					 * Figure out lock mode, and acquire lock.  This also does
diff --git a/src/backend/utils/adt/ri_triggers.c b/src/backend/utils/adt/ri_triggers.c
index 09a2ad2881..7c77c338ce 100644
--- a/src/backend/utils/adt/ri_triggers.c
+++ b/src/backend/utils/adt/ri_triggers.c
@@ -392,11 +392,15 @@ RI_FKey_check(TriggerData *trigdata)
 
 	/*
 	 * Now check that foreign key exists in PK table
+	 *
+	 * XXX detectNewRows must be true when a partitioned table is on the
+	 * referenced side.  The reason is that our snapshot must be fresh
+	 * in order for the hack in find_inheritance_children() to work.
 	 */
 	ri_PerformCheck(riinfo, &qkey, qplan,
 					fk_rel, pk_rel,
 					NULL, newslot,
-					false,
+					pk_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE,
 					SPI_OK_SELECT);
 
 	if (SPI_finish() != SPI_OK_FINISH)
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
index a6388d980e..21e60f0c5e 100644
--- a/src/backend/utils/cache/partcache.c
+++ b/src/backend/utils/cache/partcache.c
@@ -341,6 +341,7 @@ generate_partition_qual(Relation rel)
 	bool		isnull;
 	List	   *my_qual = NIL,
 			   *result = NIL;
+	Oid			parentrelid;
 	Relation	parent;
 
 	/* Guard against stack overflow due to overly deep partition tree */
@@ -350,9 +351,14 @@ generate_partition_qual(Relation rel)
 	if (rel->rd_partcheckvalid)
 		return copyObject(rel->rd_partcheck);
 
-	/* Grab at least an AccessShareLock on the parent table */
-	parent = relation_open(get_partition_parent(RelationGetRelid(rel)),
-						   AccessShareLock);
+	/*
+	 * Grab at least an AccessShareLock on the parent table.  Must do this
+	 * even if the partition has been partially detached, because transactions
+	 * concurrent with the detach might still be trying to use a partition
+	 * descriptor that includes it.
+	 */
+	parentrelid = get_partition_parent(RelationGetRelid(rel), true);
+	parent = relation_open(parentrelid, AccessShareLock);
 
 	/* Get pg_class.relpartbound */
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index eeac0efc4f..c9f7118a5d 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -2144,7 +2144,12 @@ describeOneTableDetails(const char *schemaname,
 
 		printfPQExpBuffer(&buf,
 						  "SELECT inhparent::pg_catalog.regclass,\n"
-						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid)");
+						  "  pg_catalog.pg_get_expr(c.relpartbound, c.oid),\n  ");
+
+		appendPQExpBuffer(&buf,
+						  pset.sversion >= 140000 ? "inhdetachpending" :
+						  "false as inhdetachpending");
+
 		/* If verbose, also request the partition constraint definition */
 		if (verbose)
 			appendPQExpBufferStr(&buf,
@@ -2162,17 +2167,19 @@ describeOneTableDetails(const char *schemaname,
 		{
 			char	   *parent_name = PQgetvalue(result, 0, 0);
 			char	   *partdef = PQgetvalue(result, 0, 1);
+			char	   *detached = PQgetvalue(result, 0, 2);
 
-			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
-							  partdef);
+			printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s%s"), parent_name,
+							  partdef,
+							  strcmp(detached, "t") == 0 ? " DETACH PENDING" : "");
 			printTableAddFooter(&cont, tmpbuf.data);
 
 			if (verbose)
 			{
 				char	   *partconstraintdef = NULL;
 
-				if (!PQgetisnull(result, 0, 2))
-					partconstraintdef = PQgetvalue(result, 0, 2);
+				if (!PQgetisnull(result, 0, 3))
+					partconstraintdef = PQgetvalue(result, 0, 3);
 				/* If there isn't any constraint, show that explicitly */
 				if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
 					printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
@@ -3224,9 +3231,20 @@ describeOneTableDetails(const char *schemaname,
 		}
 
 		/* print child tables (with additional info if partitions) */
-		if (pset.sversion >= 100000)
+		if (pset.sversion >= 140000)
 			printfPQExpBuffer(&buf,
 							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " inhdetachpending,"
+							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
+							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
+							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
+							  "ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT',"
+							  " c.oid::pg_catalog.regclass::pg_catalog.text;",
+							  oid);
+		else if (pset.sversion >= 100000)
+			printfPQExpBuffer(&buf,
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " false AS inhdetachpending,"
 							  " pg_catalog.pg_get_expr(c.relpartbound, c.oid)\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
@@ -3235,14 +3253,16 @@ describeOneTableDetails(const char *schemaname,
 							  oid);
 		else if (pset.sversion >= 80300)
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " false AS inhdetachpending, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.oid::pg_catalog.regclass::pg_catalog.text;",
 							  oid);
 		else
 			printfPQExpBuffer(&buf,
-							  "SELECT c.oid::pg_catalog.regclass, c.relkind, NULL\n"
+							  "SELECT c.oid::pg_catalog.regclass, c.relkind,"
+							  " false AS inhdetachpending, NULL\n"
 							  "FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i\n"
 							  "WHERE c.oid = i.inhrelid AND i.inhparent = '%s'\n"
 							  "ORDER BY c.relname;",
@@ -3292,11 +3312,13 @@ describeOneTableDetails(const char *schemaname,
 				else
 					printfPQExpBuffer(&buf, "%*s  %s",
 									  ctw, "", PQgetvalue(result, i, 0));
-				if (!PQgetisnull(result, i, 2))
-					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 2));
+				if (!PQgetisnull(result, i, 3))
+					appendPQExpBuffer(&buf, " %s", PQgetvalue(result, i, 3));
 				if (child_relkind == RELKIND_PARTITIONED_TABLE ||
 					child_relkind == RELKIND_PARTITIONED_INDEX)
 					appendPQExpBufferStr(&buf, ", PARTITIONED");
+				if (strcmp(PQgetvalue(result, i, 2), "t") == 0)
+					appendPQExpBuffer(&buf, " (DETACH PENDING)");
 				if (i < tuples - 1)
 					appendPQExpBufferChar(&buf, ',');
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index fe3f66befa..c8c7bc1d99 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,7 +19,7 @@
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-extern Oid	get_partition_parent(Oid relid);
+extern Oid	get_partition_parent(Oid relid, bool even_if_detached);
 extern List *get_partition_ancestors(Oid relid);
 extern Oid	index_get_partition(Relation partition, Oid indexId);
 extern List *map_partition_varattnos(List *expr, int fromrel_varno,
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 2b71cad9a2..6d07e1b302 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -34,6 +34,7 @@ CATALOG(pg_inherits,2611,InheritsRelationId)
 	Oid			inhrelid BKI_LOOKUP(pg_class);
 	Oid			inhparent BKI_LOOKUP(pg_class);
 	int32		inhseqno;
+	bool		inhdetachpending;
 } FormData_pg_inherits;
 
 /* ----------------
@@ -49,7 +50,8 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
+									   LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
@@ -57,6 +59,8 @@ extern bool has_superclass(Oid relationId);
 extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
 extern void StoreSingleInheritance(Oid relationId, Oid parentOid,
 								   int32 seqNumber);
-extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent);
+extern bool DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool allow_detached,
+								const char *childname);
+extern bool PartitionHasPendingDetach(Oid partoid);
 
 #endif							/* PG_INHERITS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 68425eb2c0..0ce19d98ec 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -871,6 +871,7 @@ typedef struct PartitionCmd
 	NodeTag		type;
 	RangeVar   *name;			/* name of partition to attach/detach */
 	PartitionBoundSpec *bound;	/* FOR VALUES, if attaching */
+	bool		concurrent;
 } PartitionCmd;
 
 /****************************************************************************
@@ -1909,6 +1910,7 @@ typedef enum AlterTableType
 	AT_GenericOptions,			/* OPTIONS (...) */
 	AT_AttachPartition,			/* ATTACH PARTITION */
 	AT_DetachPartition,			/* DETACH PARTITION */
+	AT_DetachPartitionFinalize,	/* DETACH PARTITION FINALIZE */
 	AT_AddIdentity,				/* ADD IDENTITY */
 	AT_SetIdentity,				/* SET identity column options */
 	AT_DropIdentity,			/* DROP IDENTITY */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index ca1f950cbe..4bbe53e852 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -166,6 +166,7 @@ PG_KEYWORD("false", FALSE_P, RESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("family", FAMILY, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("fetch", FETCH, RESERVED_KEYWORD, AS_LABEL)
 PG_KEYWORD("filter", FILTER, UNRESERVED_KEYWORD, AS_LABEL)
+PG_KEYWORD("finalize", FINALIZE, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("first", FIRST_P, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("float", FLOAT_P, COL_NAME_KEYWORD, BARE_LABEL)
 PG_KEYWORD("following", FOLLOWING, UNRESERVED_KEYWORD, BARE_LABEL)
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index ff113199e5..7f03ff4271 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -21,6 +21,7 @@
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
+	bool		includes_detached;	/* Does it include detached partitions */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
@@ -30,9 +31,9 @@ typedef struct PartitionDescData
 } PartitionDescData;
 
 
-extern PartitionDesc RelationGetPartitionDesc(Relation rel);
+extern PartitionDesc RelationGetPartitionDesc(Relation rel, bool include_detached);
 
-extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt, bool include_detached);
 extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
 extern void DestroyPartitionDirectory(PartitionDirectory pdir);
 
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index f66ac58188..44539fe15a 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -132,6 +132,7 @@ extern void AtEOXact_Snapshot(bool isCommit, bool resetXmin);
 extern void ImportSnapshot(const char *idstr);
 extern bool XactHasExportedSnapshots(void);
 extern void DeleteAllExportedSnapshotFiles(void);
+extern void WaitForOlderSnapshots(TransactionId limitXmin, bool progress);
 extern bool ThereAreNoPriorRegisteredSnapshots(void);
 extern bool TransactionIdLimitedForOldSnapshots(TransactionId recentXmin,
 												Relation relation,
diff --git a/src/test/isolation/expected/detach-partition-concurrently-1.out b/src/test/isolation/expected/detach-partition-concurrently-1.out
new file mode 100644
index 0000000000..9b4526773e
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-1.out
@@ -0,0 +1,234 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2detach s1s s1c s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1s s3s s3i s1c s3i s2drop s1s
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+f              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s3i: SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2';
+?column?       
+
+t              
+step s2drop: DROP TABLE d_listp2;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+
+starting permutation: s1b s1s s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1s s1ins2 s2detach s1ins s1s s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1ins s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1ins: INSERT INTO d_listp VALUES (1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1s s2detach s1s s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1b s1ins2 s2detach s3ins2 s1c
+step s1b: BEGIN;
+step s1ins2: INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s3ins2: INSERT INTO d_listp VALUES (2);
+ERROR:  no partition of relation "d_listp" found for row
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec1 s3s s1dealloc s1c
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec1: EXECUTE f(1);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1dealloc: DEALLOCATE f;
+step s1c: COMMIT;
+step s2detach: <... completed>
+
+starting permutation: s1brr s1prep s1exec2 s2detach s1s s1exec2 s3s s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1exec2: EXECUTE f(2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+2              
+step s1exec2: EXECUTE f(2);
+step s3s: SELECT * FROM d_listp;
+a              
+
+1              
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s1s s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep: PREPARE f(int) AS INSERT INTO d_listp VALUES ($1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep1 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep1: PREPARE f(int) AS INSERT INTO d_listp VALUES (1);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
+
+starting permutation: s1brr s1prep2 s2detach s1s s1exec2 s1c s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prep2: PREPARE f(int) AS INSERT INTO d_listp VALUES (2);
+step s2detach: ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; <waiting ...>
+step s1s: SELECT * FROM d_listp;
+a              
+
+1              
+2              
+step s1exec2: EXECUTE f(2);
+step s1c: COMMIT;
+step s2detach: <... completed>
+step s1dealloc: DEALLOCATE f;
diff --git a/src/test/isolation/expected/detach-partition-concurrently-2.out b/src/test/isolation/expected/detach-partition-concurrently-2.out
new file mode 100644
index 0000000000..85be707b40
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-2.out
@@ -0,0 +1,66 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s1b s1s s2d s3i1 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s2d s3i2 s3i2 s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3i1 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY;
+ERROR:  removing partition "d_lp_fk_1" violates foreign key constraint "d_lp_fk_r_a_fkey1"
+step s1c: COMMIT;
+
+starting permutation: s1b s1s s3i2 s2d s1c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3i2: INSERT INTO d_lp_fk_r VALUES (2);
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s1c: COMMIT;
+step s2d: <... completed>
+
+starting permutation: s1b s1s s3b s2d s3i1 s1c s3c
+step s1b: BEGIN;
+step s1s: SELECT * FROM d_lp_fk;
+a              
+
+1              
+2              
+step s3b: BEGIN;
+step s2d: ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; <waiting ...>
+step s3i1: INSERT INTO d_lp_fk_r VALUES (1);
+ERROR:  insert or update on table "d_lp_fk_r" violates foreign key constraint "d_lp_fk_r_a_fkey"
+step s1c: COMMIT;
+step s2d: <... completed>
+step s3c: COMMIT;
diff --git a/src/test/isolation/expected/detach-partition-concurrently-3.out b/src/test/isolation/expected/detach-partition-concurrently-3.out
new file mode 100644
index 0000000000..88e83638c7
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-3.out
@@ -0,0 +1,282 @@
+Parsed test spec with 2 sessions
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1describe s1alter
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1describe: SELECT 'd3_listp' AS root, * FROM pg_partition_tree('d3_listp')
+					  UNION ALL SELECT 'd3_listp1', * FROM pg_partition_tree('d3_listp1');
+root           relid          parentrelid    isleaf         level          
+
+d3_listp       d3_listp                      f              0              
+d3_listp1      d3_listp1                     t              0              
+step s1alter: ALTER TABLE d3_listp1 ALTER a DROP NOT NULL;
+ERROR:  cannot alter partition "d3_listp1" with an incomplete detach
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1insert s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1insert: INSERT INTO d3_listp VALUES (1);
+ERROR:  no partition of relation "d3_listp" found for row
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1brr s1s s2detach s1cancel s1insert s1c s1spart
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1insert: INSERT INTO d3_listp VALUES (1);
+step s1c: COMMIT;
+step s1spart: SELECT * FROM d3_listp1;
+a              
+
+1              
+1              
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1insertpart
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1insertpart: INSERT INTO d3_listp1 VALUES (1);
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1drop s1list
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1drop: DROP TABLE d3_listp;
+step s1list: SELECT relname FROM pg_catalog.pg_class
+					  WHERE relname LIKE 'd3_listp%' ORDER BY 1;
+relname        
+
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1trunc s1spart
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1trunc: TRUNCATE TABLE d3_listp;
+step s1spart: SELECT * FROM d3_listp1;
+a              
+
+1              
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2drop s1s s2commit
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s2begin: BEGIN;
+step s2drop: DROP TABLE d3_listp1;
+step s1s: SELECT * FROM d3_listp; <waiting ...>
+step s2commit: COMMIT;
+step s1s: <... completed>
+a              
+
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1b s1spart s2detachfinal s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1b: BEGIN;
+step s1spart: SELECT * FROM d3_listp1;
+a              
+
+1              
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1b s1s s2detachfinal s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1b s1spart s2detachfinal s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1b: BEGIN;
+step s1spart: SELECT * FROM d3_listp1;
+a              
+
+1              
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2detachfinal s2commit
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s2begin: BEGIN;
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s2commit: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2detachfinal s1spart s2commit
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s2begin: BEGIN;
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s1spart: SELECT * FROM d3_listp1; <waiting ...>
+step s2commit: COMMIT;
+step s1spart: <... completed>
+a              
+
+1              
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2detachfinal s1insertpart s2commit
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s2begin: BEGIN;
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
+step s1insertpart: INSERT INTO d3_listp1 VALUES (1); <waiting ...>
+step s2commit: COMMIT;
+step s1insertpart: <... completed>
+unused step name: s1droppart
diff --git a/src/test/isolation/expected/detach-partition-concurrently-4.out b/src/test/isolation/expected/detach-partition-concurrently-4.out
new file mode 100644
index 0000000000..e52ed99aa6
--- /dev/null
+++ b/src/test/isolation/expected/detach-partition-concurrently-4.out
@@ -0,0 +1,69 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1insert s1c
+step s2snitch: insert into d4_pid select pg_backend_pid();
+step s1b: begin;
+step s1s: select * from d4_primary;
+a              
+
+1              
+step s2detach: alter table d4_primary detach partition d4_primary1 concurrently; <waiting ...>
+step s1cancel: select pg_cancel_backend(pid) from d4_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
+step s1c: commit;
+
+starting permutation: s2snitch s1brr s1s s2detach s1cancel s1insert s1c
+step s2snitch: insert into d4_pid select pg_backend_pid();
+step s1brr: begin isolation level repeatable read;
+step s1s: select * from d4_primary;
+a              
+
+1              
+step s2detach: alter table d4_primary detach partition d4_primary1 concurrently; <waiting ...>
+step s1cancel: select pg_cancel_backend(pid) from d4_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
+step s1c: commit;
+
+starting permutation: s2snitch s1b s1s s2detach s3insert s1c
+step s2snitch: insert into d4_pid select pg_backend_pid();
+step s1b: begin;
+step s1s: select * from d4_primary;
+a              
+
+1              
+step s2detach: alter table d4_primary detach partition d4_primary1 concurrently; <waiting ...>
+step s3insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
+step s1c: commit;
+step s2detach: <... completed>
+
+starting permutation: s2snitch s1b s1s s2detach s3brr s3insert s3commit s1cancel
+step s2snitch: insert into d4_pid select pg_backend_pid();
+step s1b: begin;
+step s1s: select * from d4_primary;
+a              
+
+1              
+step s2detach: alter table d4_primary detach partition d4_primary1 concurrently; <waiting ...>
+step s3brr: begin isolation level repeatable read;
+step s3insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
+step s3commit: commit;
+step s1cancel: select pg_cancel_backend(pid) from d4_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index 5d6b79e66e..f4c01006fc 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -24,6 +24,10 @@ test: deadlock-hard
 test: deadlock-soft
 test: deadlock-soft-2
 test: deadlock-parallel
+test: detach-partition-concurrently-1
+test: detach-partition-concurrently-2
+test: detach-partition-concurrently-3
+test: detach-partition-concurrently-4
 test: fk-contention
 test: fk-deadlock
 test: fk-deadlock2
diff --git a/src/test/isolation/specs/detach-partition-concurrently-1.spec b/src/test/isolation/specs/detach-partition-concurrently-1.spec
new file mode 100644
index 0000000000..6ca0adac1d
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-1.spec
@@ -0,0 +1,69 @@
+# Test that detach partition concurrently makes the partition invisible at the
+# correct time.
+
+setup
+{
+  DROP TABLE IF EXISTS d_listp, d_listp1, d_listp2;
+  CREATE TABLE d_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d_listp1 PARTITION OF d_listp FOR VALUES IN (1);
+  CREATE TABLE d_listp2 PARTITION OF d_listp FOR VALUES IN (2);
+  INSERT INTO d_listp VALUES (1),(2);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d_listp, d_listp2, d_listp_foobar;
+}
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"		{ SELECT * FROM d_listp; }
+step "s1ins"	{ INSERT INTO d_listp VALUES (1); }
+step "s1ins2"	{ INSERT INTO d_listp VALUES (2); }
+step "s1prep"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES ($1); }
+step "s1prep1"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (1); }
+step "s1prep2"	{ PREPARE f(int) AS INSERT INTO d_listp VALUES (2); }
+step "s1exec1"	{ EXECUTE f(1); }
+step "s1exec2"	{ EXECUTE f(2); }
+step "s1dealloc" { DEALLOCATE f; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2detach"		{ ALTER TABLE d_listp DETACH PARTITION d_listp2 CONCURRENTLY; }
+step "s2drop"	{ DROP TABLE d_listp2; }
+
+session "s3"
+step "s3s"		{ SELECT * FROM d_listp; }
+step "s3i"		{ SELECT relpartbound IS NULL FROM pg_class where relname = 'd_listp2'; }
+step "s3ins2"		{ INSERT INTO d_listp VALUES (2); }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate, as does anybody else.
+permutation "s1b" "s1s" "s2detach" "s1s" "s1c" "s1s"
+
+# relpartbound remains set until s1 commits
+# XXX this could be timing dependent :-(
+permutation "s1b" "s1s" "s2detach" "s1s" "s3s" "s3i" "s1c" "s3i" "s2drop" "s1s"
+
+# In read-committed mode, the partition disappears from view of concurrent
+# transactions immediately.  But if a write lock is held, then the detach
+# has to wait.
+permutation "s1b"   "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1b"   "s1s" "s1ins2" "s2detach" "s1ins" "s1s" "s1c"
+
+# In repeatable-read mode, the partition remains visible until commit even
+# if the to-be-detached partition is not locked for write.
+permutation "s1brr" "s1s"          "s2detach" "s1ins" "s1s" "s1c"
+permutation "s1brr" "s1s"          "s2detach"         "s1s" "s1c"
+
+# Another process trying to acquire a write lock will be blocked behind the
+# detacher
+permutation "s1b" "s1ins2" "s2detach" "s3ins2" "s1c"
+
+# a prepared query is not blocked
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec1" "s3s" "s1dealloc" "s1c"
+permutation "s1brr" "s1prep" "s1exec2" "s2detach" "s1s" "s1exec2" "s3s" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s1s" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep1" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
+permutation "s1brr" "s1prep2" "s2detach" "s1s" "s1exec2" "s1c" "s1dealloc"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-2.spec b/src/test/isolation/specs/detach-partition-concurrently-2.spec
new file mode 100644
index 0000000000..9281c80a69
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-2.spec
@@ -0,0 +1,41 @@
+# Test that detach partition concurrently makes the partition safe
+# for foreign keys that reference it.
+
+setup
+{
+  DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r;
+
+  CREATE TABLE d_lp_fk (a int PRIMARY KEY) PARTITION BY LIST(a);
+  CREATE TABLE d_lp_fk_1 PARTITION OF d_lp_fk FOR VALUES IN (1);
+  CREATE TABLE d_lp_fk_2 PARTITION OF d_lp_fk FOR VALUES IN (2);
+  INSERT INTO d_lp_fk VALUES (1), (2);
+
+  CREATE TABLE d_lp_fk_r (a int references d_lp_fk);
+}
+
+teardown { DROP TABLE IF EXISTS d_lp_fk, d_lp_fk_1, d_lp_fk_2, d_lp_fk_r; }
+
+session "s1"
+step "s1b"		{ BEGIN; }
+step "s1s"		{ SELECT * FROM d_lp_fk; }
+step "s1c"		{ COMMIT; }
+
+session "s2"
+step "s2d"		{ ALTER TABLE d_lp_fk DETACH PARTITION d_lp_fk_1 CONCURRENTLY; }
+
+session "s3"
+step "s3b"		{ BEGIN; }
+step "s3i1"		{ INSERT INTO d_lp_fk_r VALUES (1); }
+step "s3i2"		{ INSERT INTO d_lp_fk_r VALUES (2); }
+step "s3c"		{ COMMIT; }
+
+# The transaction that detaches hangs until it sees any older transaction
+# terminate.
+permutation "s1b" "s1s" "s2d" "s3i1" "s1c"
+permutation "s1b" "s1s" "s2d" "s3i2" "s3i2" "s1c"
+
+permutation "s1b" "s1s" "s3i1" "s2d" "s1c"
+permutation "s1b" "s1s" "s3i2" "s2d" "s1c"
+
+# what if s3 has an uncommitted insertion?
+permutation "s1b" "s1s" "s3b" "s2d" "s3i1" "s1c" "s3c"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-3.spec b/src/test/isolation/specs/detach-partition-concurrently-3.spec
new file mode 100644
index 0000000000..4b706430e1
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-3.spec
@@ -0,0 +1,66 @@
+# Try various things to happen to a partition with an incomplete detach
+
+setup
+{
+  CREATE TABLE d3_listp (a int) PARTITION BY LIST(a);
+  CREATE TABLE d3_listp1 PARTITION OF d3_listp FOR VALUES IN (1);
+  CREATE TABLE d3_pid (pid int);
+  INSERT INTO d3_listp VALUES (1);
+}
+
+teardown {
+    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_pid;
+}
+
+session "s1"
+step "s1b"			{ BEGIN; }
+step "s1brr"		{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1s"			{ SELECT * FROM d3_listp; }
+step "s1spart"		{ SELECT * FROM d3_listp1; }
+step "s1cancel"		{ SELECT pg_cancel_backend(pid) FROM d3_pid; }
+step "s1c"			{ COMMIT; }
+step "s1alter"		{ ALTER TABLE d3_listp1 ALTER a DROP NOT NULL; }
+step "s1insert"		{ INSERT INTO d3_listp VALUES (1); }
+step "s1insertpart"	{ INSERT INTO d3_listp1 VALUES (1); }
+step "s1drop"		{ DROP TABLE d3_listp; }
+step "s1droppart"	{ DROP TABLE d3_listp1; }
+step "s1trunc"		{ TRUNCATE TABLE d3_listp; }
+step "s1list"		{ SELECT relname FROM pg_catalog.pg_class
+					  WHERE relname LIKE 'd3_listp%' ORDER BY 1; }
+step "s1describe"	{ SELECT 'd3_listp' AS root, * FROM pg_partition_tree('d3_listp')
+					  UNION ALL SELECT 'd3_listp1', * FROM pg_partition_tree('d3_listp1'); }
+
+session "s2"
+step "s2begin"		{ BEGIN; }
+step "s2snitch"		{ INSERT INTO d3_pid SELECT pg_backend_pid(); }
+step "s2detach"		{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; }
+step "s2detachfinal"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; }
+step "s2drop"		{ DROP TABLE d3_listp1; }
+step "s2commit"		{ COMMIT; }
+
+# Try various things while the partition is in "being detached" state, with
+# no session waiting.
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1describe" "s1alter"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1insert" "s1c"
+permutation "s2snitch" "s1brr" "s1s" "s2detach" "s1cancel" "s1insert" "s1c" "s1spart"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1insertpart"
+# "drop" here does both tables
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1drop" "s1list"
+# "truncate" only does parent, not partition
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1trunc" "s1spart"
+
+# When a partition with incomplete detach is dropped, we grab lock on parent too.
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2drop" "s1s" "s2commit"
+
+# Partially detach, then select and try to complete the detach.  Reading
+# from partition blocks (AEL is required on partition); reading from parent
+# does not block.
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1b" "s1spart" "s2detachfinal" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1b" "s1s" "s2detachfinal" "s1c"
+
+# DETACH FINALIZE in a transaction block. No insert/select on the partition
+# is allowed concurrently with that.
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1b" "s1spart" "s2detachfinal" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2detachfinal" "s2commit"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2detachfinal" "s1spart" "s2commit"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2detachfinal" "s1insertpart" "s2commit"
diff --git a/src/test/isolation/specs/detach-partition-concurrently-4.spec b/src/test/isolation/specs/detach-partition-concurrently-4.spec
new file mode 100644
index 0000000000..fa72386da7
--- /dev/null
+++ b/src/test/isolation/specs/detach-partition-concurrently-4.spec
@@ -0,0 +1,36 @@
+# This test exercises behavior of foreign keys in the face of concurrent
+# detach of partitions in the referenced table.
+setup {
+	drop table if exists d4_primary, d4_primary1, d4_fk, d4_pid;
+	create table d4_primary (a int primary key) partition by list (a);
+	create table d4_primary1 partition of d4_primary for values in (1);
+	insert into d4_primary values (1);
+	create table d4_fk (a int references d4_primary);
+	create table d4_pid (pid int);
+}
+
+session "s1"
+step "s1b"		{ begin; }
+step "s1brr"	{ begin isolation level repeatable read; }
+step "s1s"		{ select * from d4_primary; }
+step "s1cancel" { select pg_cancel_backend(pid) from d4_pid; }
+step "s1insert"	{ insert into d4_fk values (1); }
+step "s1c"		{ commit; }
+
+session "s2"
+step "s2snitch" { insert into d4_pid select pg_backend_pid(); }
+step "s2detach" { alter table d4_primary detach partition d4_primary1 concurrently; }
+
+session "s3"
+step "s3brr"	{ begin isolation level repeatable read; }
+step "s3insert" { insert into d4_fk values (1); }
+step "s3commit" { commit; }
+
+# Trying to insert into a partially detached partition is rejected
+permutation "s2snitch" "s1b"   "s1s" "s2detach" "s1cancel" "s1insert" "s1c"
+# ... even under REPEATABLE READ mode.
+permutation "s2snitch" "s1brr" "s1s" "s2detach" "s1cancel" "s1insert" "s1c"
+
+# These other cases are fairly trivial.
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s3insert" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s3brr" "s3insert" "s3commit" "s1cancel"
diff --git a/src/test/modules/delay_execution/Makefile b/src/test/modules/delay_execution/Makefile
index f270aebf3a..70f24e846d 100644
--- a/src/test/modules/delay_execution/Makefile
+++ b/src/test/modules/delay_execution/Makefile
@@ -7,7 +7,8 @@ OBJS = \
 	$(WIN32RES) \
 	delay_execution.o
 
-ISOLATION = partition-addition
+ISOLATION = partition-addition \
+	    partition-removal-1
 
 ifdef USE_PGXS
 PG_CONFIG = pg_config
diff --git a/src/test/modules/delay_execution/expected/partition-removal-1.out b/src/test/modules/delay_execution/expected/partition-removal-1.out
new file mode 100644
index 0000000000..427f41c9aa
--- /dev/null
+++ b/src/test/modules/delay_execution/expected/partition-removal-1.out
@@ -0,0 +1,175 @@
+Parsed test spec with 3 sessions
+
+starting permutation: s3lock s1b s1exec s2remp s3check s3unlock s3check s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec s2remp s3check s3unlock s3check s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec: SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec: <... completed>
+a              b              
+
+2              JKL            
+step s3check: SELECT * FROM partrem;
+a              b              
+
+1              ABC            
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1b s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1b: BEGIN;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2) AND a <> 1; <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1exec2 s2remp s3unlock s1c
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1exec2: SELECT * FROM partrem WHERE a <> (SELECT 2) AND a <> 1; <waiting ...>
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1exec2: <... completed>
+a              b              
+
+3              DEF            
+step s1c: COMMIT;
+step s2remp: <... completed>
+
+starting permutation: s3lock s1brr s1prepare s2remp s1execprep s3unlock s1check s1c s1check s1dealloc
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prepare: PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI');
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s1execprep: EXECUTE ins(2); <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1execprep: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+2              GHI            
+step s1c: COMMIT;
+step s2remp: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s1dealloc: DEALLOCATE ins;
+
+starting permutation: s1brr s1prepare s2remp s3lock s1execprep s3unlock s1check s1c s1check s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1prepare: PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI');
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s1execprep: EXECUTE ins(2); <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1execprep: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+2              GHI            
+step s1c: COMMIT;
+step s2remp: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s1dealloc: DEALLOCATE ins;
+
+starting permutation: s1brr s1check s3lock s2remp s1prepare s1execprep s3unlock s1check s1c s1check s1dealloc
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s3lock: SELECT pg_advisory_lock(12543);
+pg_advisory_lock
+
+               
+step s2remp: ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; <waiting ...>
+step s1prepare: PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI');
+step s1execprep: EXECUTE ins(2); <waiting ...>
+step s3unlock: SELECT pg_advisory_unlock(12543);
+pg_advisory_unlock
+
+t              
+step s1execprep: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+2              GHI            
+step s1c: COMMIT;
+step s2remp: <... completed>
+step s1check: SELECT * FROM partrem WHERE b = 'GHI';
+a              b              
+
+step s1dealloc: DEALLOCATE ins;
diff --git a/src/test/modules/delay_execution/specs/partition-removal-1.spec b/src/test/modules/delay_execution/specs/partition-removal-1.spec
new file mode 100644
index 0000000000..5ee2750129
--- /dev/null
+++ b/src/test/modules/delay_execution/specs/partition-removal-1.spec
@@ -0,0 +1,58 @@
+# Test removal of a partition with less-than-exclusive locking.
+
+setup
+{
+  CREATE TABLE partrem (a int, b text) PARTITION BY LIST(a);
+  CREATE TABLE partrem1 PARTITION OF partrem FOR VALUES IN (1);
+  CREATE TABLE partrem2 PARTITION OF partrem FOR VALUES IN (2);
+  CREATE TABLE partrem3 PARTITION OF partrem FOR VALUES IN (3);
+  INSERT INTO partrem VALUES (1, 'ABC');
+  INSERT INTO partrem VALUES (2, 'JKL');
+  INSERT INTO partrem VALUES (3, 'DEF');
+}
+
+teardown
+{
+  DROP TABLE IF EXISTS partrem, partrem2;
+}
+
+session "s1"
+setup		{ LOAD 'delay_execution';
+		  SET delay_execution.post_planning_lock_id = 12543; }
+step "s1b"	{ BEGIN; }
+step "s1brr"	{ BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step "s1exec"	{ SELECT * FROM partrem WHERE a <> 1 AND a <> (SELECT 3); }
+step "s1exec2"	{ SELECT * FROM partrem WHERE a <> (SELECT 2) AND a <> 1; }
+step "s1prepare" { PREPARE ins AS INSERT INTO partrem VALUES ($1, 'GHI'); }
+step "s1execprep" { EXECUTE ins(2); }
+step "s1check" { SELECT * FROM partrem WHERE b = 'GHI'; }
+step "s1c"	{ COMMIT; }
+step "s1dealloc" { DEALLOCATE ins; }
+
+session "s2"
+step "s2remp"	{ ALTER TABLE partrem DETACH PARTITION partrem2 CONCURRENTLY; }
+
+session "s3"
+step "s3lock"	{ SELECT pg_advisory_lock(12543); }
+step "s3unlock"	{ SELECT pg_advisory_unlock(12543); }
+step "s3check"	{ SELECT * FROM partrem; }
+
+# The SELECT will be planned with all three partitions shown above,
+# of which we expect partrem1 to be pruned at planning and partrem3 at
+# execution. Then we'll block, and by the time the query is actually
+# executed, detach of partrem2 is already underway (so its row doesn't
+# show up in s3's result), but we expect its row to still appear in the
+# result for s1.
+permutation "s3lock" "s1b" "s1exec" "s2remp" "s3check" "s3unlock" "s3check" "s1c"
+permutation "s3lock" "s1brr" "s1exec" "s2remp" "s3check" "s3unlock" "s3check" "s1c"
+
+# In this case we're testing that after pruning partrem2 at runtime, the
+# query still works correctly.
+permutation "s3lock" "s1b" "s1exec2" "s2remp" "s3unlock" "s1c"
+permutation "s3lock" "s1brr" "s1exec2" "s2remp" "s3unlock" "s1c"
+
+# In this case we test that an insert that's prepared in repeatable read
+# mode still works after detaching.
+permutation "s3lock" "s1brr" "s1prepare" "s2remp" "s1execprep" "s3unlock" "s1check" "s1c" "s1check" "s1dealloc"
+permutation "s1brr" "s1prepare" "s2remp" "s3lock" "s1execprep" "s3unlock" "s1check" "s1c" "s1check" "s1dealloc"
+permutation "s1brr" "s1check" "s3lock" "s2remp" "s1prepare" "s1execprep" "s3unlock" "s1check" "s1c" "s1check" "s1dealloc"
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index bb3f873f22..ec14b060a6 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -4163,6 +4163,35 @@ SELECT * from part_rp;
 (0 rows)
 
 DROP TABLE part_rp;
+-- concurrent detach
+CREATE TABLE range_parted2 (
+	a int
+) PARTITION BY RANGE(a);
+CREATE TABLE part_rp PARTITION OF range_parted2 FOR VALUES FROM (0) to (100);
+BEGIN;
+-- doesn't work in a partition block
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+ERROR:  ALTER TABLE ... DETACH CONCURRENTLY cannot run inside a transaction block
+COMMIT;
+CREATE TABLE part_rpd PARTITION OF range_parted2 DEFAULT;
+-- doesn't work if there's a default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+ERROR:  cannot detach partitions concurrently when a default partition exists
+-- doesn't work for the default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rpd CONCURRENTLY;
+ERROR:  cannot detach partitions concurrently when a default partition exists
+DROP TABLE part_rpd;
+-- works fine
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+\d+ range_parted2
+                         Partitioned table "public.range_parted2"
+ Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description 
+--------+---------+-----------+----------+---------+---------+--------------+-------------
+ a      | integer |           |          |         | plain   |              | 
+Partition key: RANGE (a)
+Number of partitions: 0
+
+DROP TABLE range_parted2;
 -- Check ALTER TABLE commands for partitioned tables and partitions
 -- cannot add/drop column to/from *only* the parent
 ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 4cc55d8525..7a9c9252dc 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2678,6 +2678,26 @@ DROP TABLE range_parted2;
 SELECT * from part_rp;
 DROP TABLE part_rp;
 
+-- concurrent detach
+CREATE TABLE range_parted2 (
+	a int
+) PARTITION BY RANGE(a);
+CREATE TABLE part_rp PARTITION OF range_parted2 FOR VALUES FROM (0) to (100);
+BEGIN;
+-- doesn't work in a partition block
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+COMMIT;
+CREATE TABLE part_rpd PARTITION OF range_parted2 DEFAULT;
+-- doesn't work if there's a default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+-- doesn't work for the default partition
+ALTER TABLE range_parted2 DETACH PARTITION part_rpd CONCURRENTLY;
+DROP TABLE part_rpd;
+-- works fine
+ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
+\d+ range_parted2
+DROP TABLE range_parted2;
+
 -- Check ALTER TABLE commands for partitioned tables and partitions
 
 -- cannot add/drop column to/from *only* the parent
-- 
2.20.1

#48Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#47)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

I added that test as promised, and I couldn't find any problems, so I
have pushed it.

Thanks!

--
�lvaro Herrera Valdivia, Chile

#49Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#25)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2020-Nov-30, Justin Pryzby wrote:

On Tue, Nov 03, 2020 at 08:56:06PM -0300, Alvaro Herrera wrote:

* On the first transaction (the one that marks the partition as
detached), the partition is locked with ShareLock rather than
ShareUpdateExclusiveLock. This means that DML is not allowed anymore.
This seems a reasonable restriction to me; surely by the time you're
detaching the partition you're not inserting data into it anymore.

I don't think it's an issue with your patch, but FYI that sounds like something
I had to do recently: detach *all* partitions of various tabls to promote their
partition key column from timestamp to timestamptz. And we insert directly
into child tables, not routed via parent.

I don't your patch is still useful, but not to us. So the documentation should
be clear about that.

FWIW since you mentioned this detail specifically: I backed away from
doing this (and use ShareUpdateExclusive), because it wasn't buying us
anything anyway. The reason for it is that I wanted to close the hole
for RI queries, and this seemed the simplest fix; but it really *wasn't*
a fix anyway. My later games with the active snapshot (which are
present in the version I pushed) better close this problem. So I don't
think this would be a problem.

--
�lvaro Herrera Valdivia, Chile

#50Tom Lane
tgl@sss.pgh.pa.us
In reply to: Alvaro Herrera (#48)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Alvaro Herrera <alvherre@alvh.no-ip.org> writes:

I added that test as promised, and I couldn't find any problems, so I
have pushed it.

Buildfarm testing suggests there's an issue under CLOBBER_CACHE_ALWAYS:

https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=trilobite&amp;dt=2021-03-29%2018%3A14%3A24

specifically

diff -U3 /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out
--- /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out	2021-03-29 20:14:21.258199311 +0200
+++ /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out	2021-03-30 18:54:34.272839428 +0200
@@ -324,6 +324,7 @@
 1              
 2              
 step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
 step s1c: commit;

starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c

regards, tom lane

#51Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#37)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Sun, Mar 21, 2021 at 03:01:15PM -0300, Alvaro Herrera wrote:

But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

Hm, I'm not sure I want to do that, because that means that if I later
have to attach the partition again with the same partition bounds, then
I might have to incur a scan to recheck the constraint. I think we want
to make the new constraint be as tight as possible ...

If it *implies* the partition constraint, then it's at least as tight (and
maybe tighter), yes ?

I think you're concerned with the case that someone has a partition with
"tight" bounds like (a>=200 and a<300) and a check constraint that's "less
tight" like (a>=100 AND a<400). In that case, the loose check constraint
doesn't imply the tighter partition constraint, so your patch would add a
non-redundant constraint.

I'm interested in the case that someone has a check constraint that almost but
not exactly matches the partition constraint, like (a<300 AND a>=200). In that
case, your patch adds a redundant constraint.

I wrote a patch which seems to effect my preferred behavior - please check.

--
Justin

Attachments:

0001-DETACH-CONCURRENTLY-avoid-creation-of-redundant-cons.patchtext/x-diff; charset=us-asciiDownload
From 1ecc8cb17192d13f7432fe582a43ab8597b15c00 Mon Sep 17 00:00:00 2001
From: Justin Pryzby <pryzbyj@telsasoft.com>
Date: Thu, 25 Mar 2021 21:24:10 -0500
Subject: [PATCH] DETACH CONCURRENTLY: avoid creation of redundant constraint

---
 src/backend/commands/tablecmds.c          | 29 ++++++++++-------------
 src/test/regress/expected/alter_table.out | 20 ++++++++++++++++
 src/test/regress/sql/alter_table.sql      |  6 +++++
 3 files changed, 39 insertions(+), 16 deletions(-)

diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 7d0bdc9ac4..62398b6882 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -17873,31 +17873,28 @@ ATExecDetachPartitionFinalize(Relation rel, RangeVar *name)
  * DetachAddConstraintIfNeeded
  *		Subroutine for ATExecDetachPartition.  Create a constraint that
  *		takes the place of the partition constraint, but avoid creating
- *		a dupe if an equivalent constraint already exists.
+ *		a dupe if an constraint already exists which implies the needed
+ *		constraint.
  */
 static void
 DetachAddConstraintIfNeeded(List **wqueue, Relation partRel)
 {
 	AlteredTableInfo *tab;
-	Expr	   *constraintExpr;
-	TupleDesc	td = RelationGetDescr(partRel);
+	List	   *constraintExpr;
 	Constraint *n;
 
-	constraintExpr = make_ands_explicit(RelationGetPartitionQual(partRel));
+	constraintExpr = (List *) eval_const_expressions(NULL,
+			(Node *) RelationGetPartitionQual(partRel));
 
-	/* If an identical constraint exists, we don't need to create one */
-	if (td->constr && td->constr->num_check > 0)
-	{
-		for (int i = 0; i < td->constr->num_check; i++)
-		{
-			Node	*thisconstr;
+	constraintExpr = (List *) make_ands_explicit(constraintExpr);
 
-			thisconstr = stringToNode(td->constr->check[i].ccbin);
-
-			if (equal(constraintExpr, thisconstr))
-				return;
-		}
-	}
+	/*
+	 * Avoid adding a new constraint if the needed constraint is implied by an
+	 * existing constraint
+	 */
+	if (PartConstraintImpliedByRelConstraint(partRel,
+				list_make1(constraintExpr)))
+		return;
 
 	tab = ATGetQueueEntry(wqueue, partRel);
 
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index ec14b060a6..f81bdf513b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -4191,6 +4191,26 @@ ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
 Partition key: RANGE (a)
 Number of partitions: 0
 
+-- constraint should be created
+\d part_rp
+              Table "public.part_rp"
+ Column |  Type   | Collation | Nullable | Default 
+--------+---------+-----------+----------+---------
+ a      | integer |           |          | 
+Check constraints:
+    "part_rp_a_check" CHECK (a IS NOT NULL AND a >= 0 AND a < 100)
+
+CREATE TABLE part_rp100 PARTITION OF range_parted2 (CHECK (a>=123 AND a<133 AND a IS NOT NULL)) FOR VALUES FROM (100) to (200);
+ALTER TABLE range_parted2 DETACH PARTITION part_rp100 CONCURRENTLY;
+-- redundant constraint should not be created
+\d part_rp100
+             Table "public.part_rp100"
+ Column |  Type   | Collation | Nullable | Default 
+--------+---------+-----------+----------+---------
+ a      | integer |           |          | 
+Check constraints:
+    "part_rp100_a_check" CHECK (a >= 123 AND a < 133 AND a IS NOT NULL)
+
 DROP TABLE range_parted2;
 -- Check ALTER TABLE commands for partitioned tables and partitions
 -- cannot add/drop column to/from *only* the parent
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 7a9c9252dc..dc0200adcb 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2696,6 +2696,12 @@ DROP TABLE part_rpd;
 -- works fine
 ALTER TABLE range_parted2 DETACH PARTITION part_rp CONCURRENTLY;
 \d+ range_parted2
+-- constraint should be created
+\d part_rp
+CREATE TABLE part_rp100 PARTITION OF range_parted2 (CHECK (a>=123 AND a<133 AND a IS NOT NULL)) FOR VALUES FROM (100) to (200);
+ALTER TABLE range_parted2 DETACH PARTITION part_rp100 CONCURRENTLY;
+-- redundant constraint should not be created
+\d part_rp100
 DROP TABLE range_parted2;
 
 -- Check ALTER TABLE commands for partitioned tables and partitions
-- 
2.17.0

#52Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Tom Lane (#50)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Mar-31, Tom Lane wrote:

diff -U3 /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out
--- /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out	2021-03-29 20:14:21.258199311 +0200
+++ /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out	2021-03-30 18:54:34.272839428 +0200
@@ -324,6 +324,7 @@
1              
2              
step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
step s1c: commit;

starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c

Hmm, actually, looking at this closely, I think the expected output is
bogus and trilobite is doing the right thing by throwing this error
here. The real question is why isn't this case behaving in that way in
every *other* animal.

--
�lvaro Herrera Valdivia, Chile
"Puedes vivir s�lo una vez, pero si lo haces bien, una vez es suficiente"

#53Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#52)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Sun, Apr 11, 2021 at 05:20:35PM -0400, Alvaro Herrera wrote:

On 2021-Mar-31, Tom Lane wrote:

diff -U3 /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out
--- /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out	2021-03-29 20:14:21.258199311 +0200
+++ /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out	2021-03-30 18:54:34.272839428 +0200
@@ -324,6 +324,7 @@
1              
2              
step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
step s1c: commit;

starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c

Hmm, actually, looking at this closely, I think the expected output is
bogus and trilobite is doing the right thing by throwing this error
here. The real question is why isn't this case behaving in that way in
every *other* animal.

I was looking/thinking at it, and wondered whether it could be a race condition
involving pg_cancel_backend()

--
Justin

#54Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#52)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Mon, Apr 12, 2021 at 6:20 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2021-Mar-31, Tom Lane wrote:

diff -U3 /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out
--- /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out      2021-03-29 20:14:21.258199311 +0200
+++ /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out    2021-03-30 18:54:34.272839428 +0200
@@ -324,6 +324,7 @@
1
2
step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
step s1c: commit;

starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c

Hmm, actually, looking at this closely, I think the expected output is
bogus and trilobite is doing the right thing by throwing this error
here. The real question is why isn't this case behaving in that way in
every *other* animal.

Indeed.

I can see a wrong behavior of RelationGetPartitionDesc() in a case
that resembles the above test case.

drop table if exists d4_primary, d4_primary1, d4_fk, d4_pid;
create table d4_primary (a int primary key) partition by list (a);
create table d4_primary1 partition of d4_primary for values in (1);
create table d4_primary2 partition of d4_primary for values in (2);
insert into d4_primary values (1);
insert into d4_primary values (2);
create table d4_fk (a int references d4_primary);
insert into d4_fk values (2);
create table d4_pid (pid int);

Session 1:
begin isolation level serializable;
select * from d4_primary;
a
---
1
2
(2 rows)

Session 2:
alter table d4_primary detach partition d4_primary1 concurrently;
<waits>

Session 1:
-- can see d4_primary1 as detached at this point, though still scans!
select * from d4_primary;
a
---
1
2
(2 rows)
insert into d4_fk values (1);
INSERT 0 1
end;

Session 2:
<alter-finishes>
ALTER TABLE

Session 1:
-- FK violated
select * from d4_primary;
a
---
2
(1 row)
select * from d4_fk;
a
---
1
(1 row)

The 2nd select that session 1 performs adds d4_primary1, whose detach
it *sees* is pending, to the PartitionDesc, but without setting its
includes_detached. The subsequent insert's RI query, because it uses
that PartitionDesc as-is, returns 1 as being present in d4_primary,
leading to the insert succeeding. When session 1's transaction
commits, the waiting ALTER proceeds with committing the 2nd part of
the DETACH, without having a chance again to check if the DETACH would
lead to the foreign key of d4_fk being violated.

It seems problematic to me that the logic of setting includes_detached
is oblivious of the special check in find_inheritance_children() to
decide whether "force"-include a detach-pending child based on
cross-checking its pg_inherit tuple's xmin against the active
snapshot. Maybe fixing that would help, although I haven't tried that
myself yet.

--
Amit Langote
EDB: http://www.enterprisedb.com

#55Amit Langote
amitlangote09@gmail.com
In reply to: Amit Langote (#54)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Mon, Apr 12, 2021 at 4:42 PM Amit Langote <amitlangote09@gmail.com> wrote:

On Mon, Apr 12, 2021 at 6:20 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2021-Mar-31, Tom Lane wrote:

diff -U3 /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out
--- /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out      2021-03-29 20:14:21.258199311 +0200
+++ /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out    2021-03-30 18:54:34.272839428 +0200
@@ -324,6 +324,7 @@
1
2
step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
step s1c: commit;

starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c

Hmm, actually, looking at this closely, I think the expected output is
bogus and trilobite is doing the right thing by throwing this error
here. The real question is why isn't this case behaving in that way in
every *other* animal.

Indeed.

I can see a wrong behavior of RelationGetPartitionDesc() in a case
that resembles the above test case.

drop table if exists d4_primary, d4_primary1, d4_fk, d4_pid;
create table d4_primary (a int primary key) partition by list (a);
create table d4_primary1 partition of d4_primary for values in (1);
create table d4_primary2 partition of d4_primary for values in (2);
insert into d4_primary values (1);
insert into d4_primary values (2);
create table d4_fk (a int references d4_primary);
insert into d4_fk values (2);
create table d4_pid (pid int);

Session 1:
begin isolation level serializable;
select * from d4_primary;
a
---
1
2
(2 rows)

Session 2:
alter table d4_primary detach partition d4_primary1 concurrently;
<waits>

Session 1:
-- can see d4_primary1 as detached at this point, though still scans!
select * from d4_primary;
a
---
1
2
(2 rows)
insert into d4_fk values (1);
INSERT 0 1
end;

Session 2:
<alter-finishes>
ALTER TABLE

Session 1:
-- FK violated
select * from d4_primary;
a
---
2
(1 row)
select * from d4_fk;
a
---
1
(1 row)

The 2nd select that session 1 performs adds d4_primary1, whose detach
it *sees* is pending, to the PartitionDesc, but without setting its
includes_detached. The subsequent insert's RI query, because it uses
that PartitionDesc as-is, returns 1 as being present in d4_primary,
leading to the insert succeeding. When session 1's transaction
commits, the waiting ALTER proceeds with committing the 2nd part of
the DETACH, without having a chance again to check if the DETACH would
lead to the foreign key of d4_fk being violated.

It seems problematic to me that the logic of setting includes_detached
is oblivious of the special check in find_inheritance_children() to
decide whether "force"-include a detach-pending child based on
cross-checking its pg_inherit tuple's xmin against the active
snapshot. Maybe fixing that would help, although I haven't tried that
myself yet.

I tried that in the attached. It is indeed the above failing
isolation test whose output needed to be adjusted.

While at it, I tried rewording the comment around that special
visibility check done to force-include detached partitions, as I got
confused by the way it's worded currently. Actually, it may be a good
idea to add some comments around the intended include_detached
behavior in the places where PartitionDesc is used; e.g.
set_relation_partition_info() lacks a one-liner on why it's okay for
the planner to not see detached partitions. Or perhaps, a comment for
includes_detached of PartitionDesc should describe the various cases
in which it is true and the cases in which it is not.

--
Amit Langote
EDB: http://www.enterprisedb.com

Attachments:

PartitionDesc-includes_detached-thinko-fix.patchapplication/octet-stream; name=PartitionDesc-includes_detached-thinko-fix.patchDownload
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index bb8b2249b1..ed75468a56 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -54,11 +54,12 @@ typedef struct SeenRelsEntry
  *
  * include_detached says to include all partitions, even if they're marked
  * detached.  Passing it as false means they might or might not be included,
- * depending on the visibility of the pg_inherits row for the active snapshot.
+ * depending on the visibility of the pg_inherits row for the active snapshot;
+ * whether actually included or not is returned in *detached_included.
  */
 List *
 find_inheritance_children(Oid parentrelId, bool include_detached,
-						  LOCKMODE lockmode)
+						  bool *detached_included, LOCKMODE lockmode)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -71,6 +72,9 @@ find_inheritance_children(Oid parentrelId, bool include_detached,
 				numoids,
 				i;
 
+	if (detached_included)
+		*detached_included = false;
+
 	/*
 	 * Can skip the scan if pg_class shows the relation has never had a
 	 * subclass.
@@ -104,11 +108,12 @@ find_inheritance_children(Oid parentrelId, bool include_detached,
 		 * if its pg_inherits tuple's xmin is still visible to the active
 		 * snapshot.
 		 *
-		 * The reason for this check is that we want to avoid seeing the
-		 * partition as alive in RI queries during REPEATABLE READ or
-		 * SERIALIZABLE transactions.  (If there's no active snapshot set,
-		 * that means we're not running a user query, so it's OK to always
-		 * include detached partitions in that case.)
+		 * The reason for this check is that we want the RI queries running in
+		 * REPEATABLE READ or SERIALIZABLE transactions to not see detach-
+		 * pending partitions that were detached after a given transaction
+		 * started.  (If there's no active snapshot set, that means we're not
+		 * running a user query, so it's OK to always include detached
+		 * partitions in that case.)
 		 */
 		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending &&
 			!include_detached &&
@@ -122,6 +127,9 @@ find_inheritance_children(Oid parentrelId, bool include_detached,
 
 			if (!XidInMVCCSnapshot(xmin, snap))
 				continue;
+
+			if (detached_included)
+				*detached_included = true;
 		}
 
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
@@ -236,7 +244,7 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 
 		/* Get the direct children of this rel */
 		currentchildren = find_inheritance_children(currentrel, false,
-													lockmode);
+													NULL, lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 096a6f2891..291ebbd3be 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3507,7 +3507,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, false, NoLock) != NIL)
+			find_inheritance_children(myrelid, false, NULL, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3706,7 +3706,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, false, NoLock) != NIL)
+				find_inheritance_children(myrelid, false, NULL, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6580,7 +6580,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, false, NoLock) != NIL)
+		find_inheritance_children(myrelid, false, NULL, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6826,7 +6826,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), false, NULL, lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7689,7 +7689,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
+		find_inheritance_children(RelationGetRelid(rel), false, NULL, lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8297,7 +8297,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), false, NULL, lockmode);
 
 	if (children)
 	{
@@ -8785,7 +8785,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), false, NULL, lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11319,7 +11319,7 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 */
 	if (!is_no_inherit_constraint)
 		children =
-			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+			find_inheritance_children(RelationGetRelid(rel), false, NULL, lockmode);
 	else
 		children = NIL;
 
@@ -11704,7 +11704,7 @@ ATPrepAlterColumnType(List **wqueue,
 	}
 	else if (!recursing &&
 			 find_inheritance_children(RelationGetRelid(rel), false,
-									   NoLock) != NIL)
+									   NULL, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 3421014e47..b216a0a9a9 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,7 +1141,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, false,
+			idxs = find_inheritance_children(indexOid, false, NULL,
 											 ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 58570fecfd..35dd10c6b2 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -105,6 +105,7 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	MemoryContext new_pdcxt;
 	MemoryContext oldcxt;
 	int		   *mapping;
+	bool		includes_detached;
 
 	/*
 	 * Get partition oids from pg_inherits.  This uses a single snapshot to
@@ -113,7 +114,7 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	 * some well-defined point in time.
 	 */
 	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
-										NoLock);
+										&includes_detached, NoLock);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -241,7 +242,7 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
-		partdesc->includes_detached = include_detached;
+		partdesc->includes_detached = includes_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 6d07e1b302..26f5489ac8 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -51,7 +51,7 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 
 
 extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
-									   LOCKMODE lockmode);
+									   bool *detached_included, LOCKMODE lockmode);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/test/isolation/expected/detach-partition-concurrently-4.out b/src/test/isolation/expected/detach-partition-concurrently-4.out
index 90a75cb077..2167675374 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-4.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-4.out
@@ -324,6 +324,7 @@ a
 1              
 2              
 step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
 step s1c: commit;
 
 starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c
#56Amit Langote
amitlangote09@gmail.com
In reply to: Justin Pryzby (#53)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Mon, Apr 12, 2021 at 6:23 AM Justin Pryzby <pryzby@telsasoft.com> wrote:

On Sun, Apr 11, 2021 at 05:20:35PM -0400, Alvaro Herrera wrote:

On 2021-Mar-31, Tom Lane wrote:

diff -U3 /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out
--- /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/expected/detach-partition-concurrently-4.out    2021-03-29 20:14:21.258199311 +0200
+++ /home/buildfarm/trilobite/buildroot/HEAD/pgsql.build/src/test/isolation/output_iso/results/detach-partition-concurrently-4.out  2021-03-30 18:54:34.272839428 +0200
@@ -324,6 +324,7 @@
1
2
step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
step s1c: commit;

starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c

Hmm, actually, looking at this closely, I think the expected output is
bogus and trilobite is doing the right thing by throwing this error
here. The real question is why isn't this case behaving in that way in
every *other* animal.

I was looking/thinking at it, and wondered whether it could be a race condition
involving pg_cancel_backend()

I thought about it some and couldn't come up with an explanation as to
why pg_cancel_backend() race might be a problem.

Actually it occurred to me this morning that CLOBBER_CACHE_ALWAYS is
what exposed this problem on this animal (not sure if other such
animals did too though). With CLOBBER_CACHE_ALWAYS, a PartitionDesc
will be built afresh on most uses. In this particular case, the RI
query executed by the insert has to build a new one (for d4_primary),
correctly excluding the detach-pending partition (d4_primary1) per the
snapshot with which it is run. In normal builds, it would reuse the
one built by an earlier query in the transaction, which does include
the detach-pending partition, thus allowing the insert trying to
insert a row referencing that partition to succeed. There is a
provision in RelationGetPartitionDesc() to rebuild if any
detach-pending partitions in the existing copy of PartitionDesc are
not to be seen by the current query, but a bug mentioned in my earlier
reply prevents that from kicking in.

--
Amit Langote
EDB: http://www.enterprisedb.com

#57Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#56)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-13, Amit Langote wrote:

Actually it occurred to me this morning that CLOBBER_CACHE_ALWAYS is
what exposed this problem on this animal (not sure if other such
animals did too though). With CLOBBER_CACHE_ALWAYS, a PartitionDesc
will be built afresh on most uses. In this particular case, the RI
query executed by the insert has to build a new one (for d4_primary),
correctly excluding the detach-pending partition (d4_primary1) per the
snapshot with which it is run. In normal builds, it would reuse the
one built by an earlier query in the transaction, which does include
the detach-pending partition, thus allowing the insert trying to
insert a row referencing that partition to succeed. There is a
provision in RelationGetPartitionDesc() to rebuild if any
detach-pending partitions in the existing copy of PartitionDesc are
not to be seen by the current query, but a bug mentioned in my earlier
reply prevents that from kicking in.

Right -- that explanation makes perfect sense: the problem stems from
the fact that the cached copy of the partition descriptor is not valid
depending on the visibility of detached partitions for the operation
that requests the descriptor. I think your patch is a critical part to
a complete solution, but one thing is missing: we don't actually know
that the detached partitions we see now are the same detached partitions
we saw a moment ago. After all, a partitioned table can have several
partitions in the process of being detached; so if you just go with the
boolean "does it have any detached or not" bit, you could end up with a
descriptor that doesn't include/ignore *all* detached partitions, just
the older one(s).

I think you could fix this aspect easily by decreeing that you can only
have only one partition-being-detached at one point. So if you try to
DETACH CONCURRENTLY and there's another one in that state, raise an
error. Maybe for simplicity we should do that anyway.

But I think there's another hidden assumption in your patch, which is
that the descriptor is rebuilt every now and then *anyway* because the
flag for detached flips between parser and executor, and because we send
invalidation messages for each detach. I don't think we would ever
change things that would break this flipping (it sounds like planner and
executor necessarily have to be doing things differently all the time),
but it seems fragile as heck. I would feel much safer if we just
avoided caching the wrong thing ... or perhaps keep a separate cache
entry (one descriptor including detached, another one not), to avoid
pointless rebuilds.

--
�lvaro Herrera Valdivia, Chile

#58Justin Pryzby
pryzby@telsasoft.com
In reply to: Alvaro Herrera (#57)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Sat, Apr 10, 2021 at 01:42:26PM -0500, Justin Pryzby wrote:

On Sun, Mar 21, 2021 at 03:01:15PM -0300, Alvaro Herrera wrote:

But note that it doesn't check if an existing constraint "implies" the new
constraint - maybe it should.

Hm, I'm not sure I want to do that, because that means that if I later
have to attach the partition again with the same partition bounds, then
I might have to incur a scan to recheck the constraint. I think we want
to make the new constraint be as tight as possible ...

If it *implies* the partition constraint, then it's at least as tight (and
maybe tighter), yes ?

I think you're concerned with the case that someone has a partition with
"tight" bounds like (a>=200 and a<300) and a check constraint that's "less
tight" like (a>=100 AND a<400). In that case, the loose check constraint
doesn't imply the tighter partition constraint, so your patch would add a
non-redundant constraint.

I'm interested in the case that someone has a check constraint that almost but
not exactly matches the partition constraint, like (a<300 AND a>=200). In that
case, your patch adds a redundant constraint.

I wrote a patch which seems to effect my preferred behavior - please check.

On Sat, Apr 10, 2021 at 02:13:26PM -0500, Justin Pryzby wrote:

I suppose the docs should be updated for the exact behavior, maybe even without
this patch:

|A <literal>CHECK</literal> constraint
|that duplicates the partition constraint is added to the partition.

I added this as an Opened Item, since it affects user-visible behavior:
whether or not a redundant, non-equal constraint is added.

--
Justin

#59Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#57)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

OK so after mulling this over for a long time, here's a patch for this.
It solves the problem by making the partition descriptor no longer be
cached if a detached partition is omitted. Any transaction that needs a
partition descriptor that excludes detached partitions, will have to
recreate the partdesc from scratch. To support this, I changed the
output boolean semantics: instead of "does this partdesc include an
detached partitions" as in your patch, it now is "are there any detached
partitions". But whenever no detached partitions exist, or when all
partitions including detached are requested, then the cached descriptor
is returned (because that necessarily has to be correct). The main
difference this has to your patch is that we always keep the descriptor
in the cache and don't rebuild anything, unless a detached partition is
present.

I flipped the find_inheritance_children() input boolean, from
"include_detached" to "omit_detached". This is more natural, given the
internal behavior. You could argue to propagate that naming change to
the partdesc.h API and PartitionDirectory, but I don't think there's a
need for that.

I ran all the detach-partition-concurrently tests under
debug_invalidate_system_caches_always=1 and everything passes.

I experimented with keeping a separate cached partition descriptor that
omits the detached partition, but that brings back some trouble; I
couldn't find a way to invalidate such a cached entry in a reasonable
way. I have the patch for that, if somebody wants to play with it.

--
�lvaro Herrera Valdivia, Chile
"That sort of implies that there are Emacs keystrokes which aren't obscure.
I've been using it daily for 2 years now and have yet to discover any key
sequence which makes any sense." (Paul Thomas)

Attachments:

0001-Fix-relcache-hazard-with-detached-partitions.patchtext/x-diff; charset=iso-8859-1Download
From f95149f0fee451b8a2b49861dc813aeddf384f8f Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Tue, 20 Apr 2021 18:01:18 -0400
Subject: [PATCH] Fix relcache hazard with detached partitions
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

During queries run by ri_triggers.c queries, we need to omit partitions
that are marked pending detach -- otherwise, the RI query is tricked
into allowing a row into the referencing table whose corresponding row
is in the detached partition.  Which is bogus: once the detach operation
completes, the row becomes an orphan.

However, the code was not doing that in repeatable-read transactions,
because relcache kept a copy of the partition descriptor that included
the partition, and used it in the RI query.  This commit changes the
caching code so that the partition descriptors we keep in relcache are
only those that do not contain any detached partition, *or* those that
are requested to contain all partitions.  When a partdesc-without-
detached-partitions is requested, we create one afresh each time.

This was noticed because a buildfarm member that runs with relcache
clobbering, which would not keep the improperly cached partdesc, broke
one test, which led us to realize that the expected output of that test
was bogus.  This commit also corrects that expected output.

Author: Amit Langote <amitlangote09@gmail.com>
Author: Álvaro Herrera <alvherre@alvh.no-ip.org>
Discussion: https://postgr.es/m/3269784.1617215412@sss.pgh.pa.us
---
 src/backend/catalog/pg_inherits.c             | 64 +++++++++++--------
 src/backend/commands/tablecmds.c              | 22 +++----
 src/backend/commands/trigger.c                |  4 +-
 src/backend/executor/execPartition.c          | 15 ++---
 src/backend/partitioning/partdesc.c           | 30 +++++----
 src/include/catalog/pg_inherits.h             |  4 +-
 src/include/partitioning/partdesc.h           | 10 ++-
 .../detach-partition-concurrently-4.out       |  1 +
 8 files changed, 88 insertions(+), 62 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index bb8b2249b1..bb6b1686ee 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,13 +52,19 @@ typedef struct SeenRelsEntry
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
  *
- * include_detached says to include all partitions, even if they're marked
- * detached.  Passing it as false means they might or might not be included,
- * depending on the visibility of the pg_inherits row for the active snapshot.
+ * If a partition's pg_inherits row is marked "detach pending",
+ * *detached_exist (if not null) is set true, otherwise it is set false.
+ *
+ * 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.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool include_detached,
-						  LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool omit_detached,
+						  LOCKMODE lockmode, bool *detached_exist)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -78,6 +84,9 @@ find_inheritance_children(Oid parentrelId, bool include_detached,
 	if (!has_subclass(parentrelId))
 		return NIL;
 
+	if (detached_exist)
+		*detached_exist = false;
+
 	/*
 	 * Scan pg_inherits and build a working array of subclass OIDs.
 	 */
@@ -99,29 +108,34 @@ find_inheritance_children(Oid parentrelId, bool include_detached,
 	{
 		/*
 		 * Cope with partitions concurrently being detached.  When we see a
-		 * partition marked "detach pending", we only include it in the set of
-		 * visible partitions if caller requested all detached partitions, or
-		 * if its pg_inherits tuple's xmin is still visible to the active
-		 * snapshot.
+		 * partition marked "detach pending", we omit it from the returned
+		 * descriptor if caller requested that and the tuple's xmin does not
+		 * appear in progress to the active snapshot.  (If there's no active
+		 * snapshot set, that means we're not running a user query, so it's OK
+		 * to always include detached partitions in that case; if the xmin is
+		 * still running to the active snapshot, then the partition has not
+		 * been detached yet and so we include it.)
 		 *
-		 * The reason for this check is that we want to avoid seeing the
+		 * The reason for this hack is that we want to avoid seeing the
 		 * partition as alive in RI queries during REPEATABLE READ or
-		 * SERIALIZABLE transactions.  (If there's no active snapshot set,
-		 * that means we're not running a user query, so it's OK to always
-		 * include detached partitions in that case.)
+		 * SERIALIZABLE transactions.
 		 */
-		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending &&
-			!include_detached &&
-			ActiveSnapshotSet())
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending)
 		{
-			TransactionId xmin;
-			Snapshot snap;
+			if (detached_exist)
+				*detached_exist = true;
 
-			xmin = HeapTupleHeaderGetXmin(inheritsTuple->t_data);
-			snap = GetActiveSnapshot();
+			if (omit_detached && ActiveSnapshotSet())
+			{
+				TransactionId xmin;
+				Snapshot	snap;
 
-			if (!XidInMVCCSnapshot(xmin, snap))
-				continue;
+				xmin = HeapTupleHeaderGetXmin(inheritsTuple->t_data);
+				snap = GetActiveSnapshot();
+
+				if (!XidInMVCCSnapshot(xmin, snap))
+					continue;
+			}
 		}
 
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
@@ -235,8 +249,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, false,
-													lockmode);
+		currentchildren = find_inheritance_children(currentrel, true,
+													lockmode, NULL);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -524,7 +538,7 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool expect_detach_pending,
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
-			bool	detach_pending;
+			bool		detach_pending;
 
 			detach_pending =
 				((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending;
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 096a6f2891..31da913d2b 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3507,7 +3507,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, false, NoLock) != NIL)
+			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3706,7 +3706,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, false, NoLock) != NIL)
+				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6580,7 +6580,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, false, NoLock) != NIL)
+		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6826,7 +6826,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7689,7 +7689,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8297,7 +8297,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
 
 	if (children)
 	{
@@ -8785,7 +8785,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11318,8 +11318,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children =
-			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		children = find_inheritance_children(RelationGetRelid(rel), true,
+											 lockmode, NULL);
 	else
 		children = NIL;
 
@@ -11703,8 +11703,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), false,
-									   NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), true,
+									   NoLock, NULL) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 3421014e47..2dce62999d 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, false,
-											 ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, true,
+											 ShareRowExclusiveLock, NULL);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 99780ebb96..bbfc9ef3fa 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -991,14 +991,11 @@ ExecInitPartitionDispatchInfo(EState *estate,
 
 	/*
 	 * For data modification, it is better that executor does not include
-	 * partitions being detached, except in snapshot-isolation mode.  This
-	 * means that a read-committed transaction immediately gets a "no
-	 * partition for tuple" error when a tuple is inserted into a partition
-	 * that's being detached concurrently, but a transaction in repeatable-
-	 * read mode can still use the partition.  Note that because partition
-	 * detach uses ShareLock on the partition (which conflicts with DML),
-	 * we're certain that the detach won't be able to complete until any
-	 * inserting transaction is done.
+	 * partitions being detached, except when running in snapshot-isolation
+	 * mode.  This means that a read-committed transaction immediately gets a
+	 * "no partition for tuple" error when a tuple is inserted into a
+	 * partition that's being detached concurrently, but a transaction in
+	 * repeatable-read mode can still use such a partition.
 	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
@@ -1571,7 +1568,7 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
-	/* Executor must always include detached partitions */
+	/* For data reading, executor always includes detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
 			CreatePartitionDirectory(estate->es_query_cxt, true);
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 58570fecfd..7d019a72be 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -47,7 +47,8 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
+static PartitionDesc RelationBuildPartitionDesc(Relation rel,
+												bool include_detached);
 
 
 /*
@@ -64,14 +65,13 @@ static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 PartitionDesc
 RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
-	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return NULL;
+	Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
-	if (unlikely(rel->rd_partdesc == NULL ||
-				 rel->rd_partdesc->includes_detached != include_detached))
-		RelationBuildPartitionDesc(rel, include_detached);
+	if (likely(rel->rd_partdesc &&
+			   (!rel->rd_partdesc->detached_exist || include_detached)))
+		return rel->rd_partdesc;
 
-	return rel->rd_partdesc;
+	return RelationBuildPartitionDesc(rel, include_detached);
 }
 
 /*
@@ -89,7 +89,7 @@ RelationGetPartitionDesc(Relation rel, bool include_detached)
  * that some of our callees allocate memory on their own which would be leaked
  * permanently.
  */
-static void
+static PartitionDesc
 RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
@@ -98,6 +98,7 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	PartitionBoundSpec **boundspecs = NULL;
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
+	bool		detached_exist;
 	ListCell   *cell;
 	int			i,
 				nparts;
@@ -112,8 +113,8 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
-										NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), !include_detached,
+										NoLock, &detached_exist);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -234,6 +235,7 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	partdesc = (PartitionDescData *)
 		MemoryContextAllocZero(new_pdcxt, sizeof(PartitionDescData));
 	partdesc->nparts = nparts;
+	partdesc->detached_exist = detached_exist;
 	/* If there are no partitions, the rest of the partdesc can stay zero */
 	if (nparts > 0)
 	{
@@ -241,7 +243,6 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
-		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -276,7 +277,12 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	if (rel->rd_pdcxt != NULL)
 		MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
 	rel->rd_pdcxt = new_pdcxt;
-	rel->rd_partdesc = partdesc;
+
+	/* Store it into relcache, but only if no detached partitions exist */
+	if (!detached_exist)
+		rel->rd_partdesc = partdesc;
+
+	return partdesc;
 }
 
 /*
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 6d07e1b302..4d28ede5a6 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,8 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
-									   LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
+									   LOCKMODE lockmode, bool *detached_exist);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 7f03ff4271..41485c5b10 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -17,11 +17,19 @@
 
 /*
  * Information about partitions of a partitioned table.
+ *
+ * For partitioned tables where detached partitions exist, we only cache the
+ * descriptor that includes all partitions, including detached; when we're
+ * requested a descriptor without the detached partitions, we create one
+ * afresh each time.  (The reason for this is that the set of detached
+ * partitions that are visible to each caller depends on the snapshot it has,
+ * so it's pretty much impossible to evict a descriptor from cache at the
+ * right time.)
  */
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
-	bool		includes_detached;	/* Does it include detached partitions */
+	bool		detached_exist; /* Are there any detached partitions? */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
diff --git a/src/test/isolation/expected/detach-partition-concurrently-4.out b/src/test/isolation/expected/detach-partition-concurrently-4.out
index 90a75cb077..2167675374 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-4.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-4.out
@@ -324,6 +324,7 @@ a
 1              
 2              
 step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
 step s1c: commit;
 
 starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c
-- 
2.20.1

#60Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#59)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Actually I had a silly bug in the version that attempted to cache a
partdesc that omits detached partitions. This one, while not fully
baked, seems to work correctly (on top of the previous one).

The thing that I don't fully understand is why we have to require to
have built the regular one first.

--
�lvaro Herrera 39�49'30"S 73�17'W
"This is what I like so much about PostgreSQL. Most of the surprises
are of the "oh wow! That's cool" Not the "oh shit!" kind. :)"
Scott Marlowe, http://archives.postgresql.org/pgsql-admin/2008-10/msg00152.php

Attachments:

0001-separately-cache-a-partdesc-that-omits-detached-part.patchtext/x-diff; charset=us-asciiDownload
From b45e9db9cd08282f7a7fabcef944880cc3a3d415 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Tue, 20 Apr 2021 20:42:30 -0400
Subject: [PATCH] separately cache a partdesc that omits detached partitions,
 too

---
 src/backend/partitioning/partdesc.c | 21 +++++++++++++++++++--
 src/backend/utils/cache/relcache.c  |  4 ++++
 src/include/utils/rel.h             |  1 +
 3 files changed, 24 insertions(+), 2 deletions(-)

diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 7d019a72be..f245cbd664 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -67,10 +67,25 @@ RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
 	Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
+	/*
+	 * The common case is that there are no partitions pending detach and
+	 * that the appropriate descriptor has already been built; use that.
+	 */
 	if (likely(rel->rd_partdesc &&
 			   (!rel->rd_partdesc->detached_exist || include_detached)))
 		return rel->rd_partdesc;
 
+	/*
+	 * If we're asked for a descriptor with detached partitions omitted, and
+	 * we have already built it, return that.
+	 */
+	if (!include_detached &&
+		rel->rd_partdesc &&
+		rel->rd_partdesc->detached_exist &&
+		rel->rd_partdesc_nodetached)
+		return rel->rd_partdesc_nodetached;
+
+	/* Otherwise build one afresh and store it for next time */
 	return RelationBuildPartitionDesc(rel, include_detached);
 }
 
@@ -278,9 +293,11 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 		MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
 	rel->rd_pdcxt = new_pdcxt;
 
-	/* Store it into relcache, but only if no detached partitions exist */
-	if (!detached_exist)
+	/* Store it into the appropriate member of the relcache entry */
+	if (!detached_exist || include_detached)
 		rel->rd_partdesc = partdesc;
+	else
+		rel->rd_partdesc_nodetached = partdesc;
 
 	return partdesc;
 }
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 29702d6eab..87a7613584 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -1157,6 +1157,7 @@ RelationBuildDesc(Oid targetRelId, bool insertIt)
 	relation->rd_partkey = NULL;
 	relation->rd_partkeycxt = NULL;
 	relation->rd_partdesc = NULL;
+	relation->rd_partdesc_nodetached = NULL;
 	relation->rd_pdcxt = NULL;
 	relation->rd_partcheck = NIL;
 	relation->rd_partcheckvalid = false;
@@ -2672,12 +2673,14 @@ RelationClearRelation(Relation relation, bool rebuild)
 			 * newrel.
 			 */
 			relation->rd_partdesc = NULL;	/* ensure rd_partdesc is invalid */
+			relation->rd_partdesc_nodetached = NULL;
 			if (relation->rd_pdcxt != NULL) /* probably never happens */
 				MemoryContextSetParent(newrel->rd_pdcxt, relation->rd_pdcxt);
 			else
 				relation->rd_pdcxt = newrel->rd_pdcxt;
 			/* drop newrel's pointers so we don't destroy it below */
 			newrel->rd_partdesc = NULL;
+			newrel->rd_partdesc_nodetached = NULL;
 			newrel->rd_pdcxt = NULL;
 		}
 
@@ -5942,6 +5945,7 @@ load_relcache_init_file(bool shared)
 		rel->rd_partkey = NULL;
 		rel->rd_partkeycxt = NULL;
 		rel->rd_partdesc = NULL;
+		rel->rd_partdesc_nodetached = NULL;
 		rel->rd_pdcxt = NULL;
 		rel->rd_partcheck = NIL;
 		rel->rd_partcheckvalid = false;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 34e25eb597..3d480a3ab2 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -127,6 +127,7 @@ typedef struct RelationData
 
 	/* data managed by RelationGetPartitionDesc: */
 	PartitionDesc rd_partdesc;	/* partition descriptor, or NULL */
+	PartitionDesc rd_partdesc_nodetached;	/* partition descriptor omitting detached partitions */
 	MemoryContext rd_pdcxt;		/* private context for rd_partdesc, if any */
 
 	/* data managed by RelationGetPartitionQual: */
-- 
2.20.1

#61Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#60)
2 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

While the approach in the previous email does pass the tests, I think
(but couldn't find a test case to prove) it does so coincidentally, not
because it is correct. If I make the test for "detached exist" use the
cached omits-partitions-partdesc, it does fail, because we had
previously cached one that was not yet omitting the partition. So what
I said earlier in the thread stands: the set of partitions that are
considered detached changes depending on what the active snapshot is,
and therefore we *must not* cache any such descriptor.

So I backtracked to my previous proposal, which saves in relcache only
the partdesc that includes all partitions. If any partdesc is built
that omits partitions being detached, that one must be rebuilt afresh
each time. And to avoid potentially saving a lot of single-use
partdescs in CacheMemoryContext, in the attached second patch (which I
attach separately only to make it more obvious to review) I store such
partdescs in PortalContext.

Barring objections, I will get this pushed early tomorrow.

--
�lvaro Herrera Valdivia, Chile
"Just treat us the way you want to be treated + some extra allowance
for ignorance." (Michael Brusser)

Attachments:

0001-Fix-relcache-hazard-with-detached-partitions.patchtext/x-diff; charset=iso-8859-1Download
From a6e316fd12e5afe4dbb736e27ac83f688c7138ab Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Tue, 20 Apr 2021 18:01:18 -0400
Subject: [PATCH 1/2] Fix relcache hazard with detached partitions
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

During queries coming from ri_triggers.c, we need to omit partitions
that are marked pending detach -- otherwise, the RI query is tricked
into allowing a row into the referencing table whose corresponding row
is in the detached partition.  Which is bogus: once the detach operation
completes, the row becomes an orphan.

However, the code was not doing that in repeatable-read transactions,
because relcache kept a copy of the partition descriptor that included
the partition, and used it in the RI query.  This commit changes the
partdesc cache code to only keep descriptors that aren't dependent on
a snapshot (namely: those where no detached partition exist, and those
where detached partitions are included).  When a partdesc-without-
detached-partitions is requested, we create one afresh each time.

find_inheritance_children gets a new output *detached_exist boolean,
which indicates whether any partition marked pending-detach is found.
Its "include_detached" input flag is changed to "omit_detached", because
that captures desired the semantics more naturally.

This was noticed because a buildfarm member that runs with relcache
clobbering, which would not keep the improperly cached partdesc, broke
one test, which led us to realize that the expected output of that test
was bogus.  This commit also corrects that expected output.

Author: Amit Langote <amitlangote09@gmail.com>
Author: Álvaro Herrera <alvherre@alvh.no-ip.org>
Discussion: https://postgr.es/m/3269784.1617215412@sss.pgh.pa.us
---
 src/backend/catalog/pg_inherits.c             | 64 +++++++++++--------
 src/backend/commands/tablecmds.c              | 22 +++----
 src/backend/commands/trigger.c                |  4 +-
 src/backend/executor/execPartition.c          | 15 ++---
 src/backend/partitioning/partdesc.c           | 30 +++++----
 src/include/catalog/pg_inherits.h             |  4 +-
 src/include/partitioning/partdesc.h           | 10 ++-
 .../detach-partition-concurrently-4.out       |  1 +
 8 files changed, 88 insertions(+), 62 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index bb8b2249b1..bb6b1686ee 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,13 +52,19 @@ typedef struct SeenRelsEntry
  * then no locks are acquired, but caller must beware of race conditions
  * against possible DROPs of child relations.
  *
- * include_detached says to include all partitions, even if they're marked
- * detached.  Passing it as false means they might or might not be included,
- * depending on the visibility of the pg_inherits row for the active snapshot.
+ * If a partition's pg_inherits row is marked "detach pending",
+ * *detached_exist (if not null) is set true, otherwise it is set false.
+ *
+ * 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.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool include_detached,
-						  LOCKMODE lockmode)
+find_inheritance_children(Oid parentrelId, bool omit_detached,
+						  LOCKMODE lockmode, bool *detached_exist)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -78,6 +84,9 @@ find_inheritance_children(Oid parentrelId, bool include_detached,
 	if (!has_subclass(parentrelId))
 		return NIL;
 
+	if (detached_exist)
+		*detached_exist = false;
+
 	/*
 	 * Scan pg_inherits and build a working array of subclass OIDs.
 	 */
@@ -99,29 +108,34 @@ find_inheritance_children(Oid parentrelId, bool include_detached,
 	{
 		/*
 		 * Cope with partitions concurrently being detached.  When we see a
-		 * partition marked "detach pending", we only include it in the set of
-		 * visible partitions if caller requested all detached partitions, or
-		 * if its pg_inherits tuple's xmin is still visible to the active
-		 * snapshot.
+		 * partition marked "detach pending", we omit it from the returned
+		 * descriptor if caller requested that and the tuple's xmin does not
+		 * appear in progress to the active snapshot.  (If there's no active
+		 * snapshot set, that means we're not running a user query, so it's OK
+		 * to always include detached partitions in that case; if the xmin is
+		 * still running to the active snapshot, then the partition has not
+		 * been detached yet and so we include it.)
 		 *
-		 * The reason for this check is that we want to avoid seeing the
+		 * The reason for this hack is that we want to avoid seeing the
 		 * partition as alive in RI queries during REPEATABLE READ or
-		 * SERIALIZABLE transactions.  (If there's no active snapshot set,
-		 * that means we're not running a user query, so it's OK to always
-		 * include detached partitions in that case.)
+		 * SERIALIZABLE transactions.
 		 */
-		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending &&
-			!include_detached &&
-			ActiveSnapshotSet())
+		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending)
 		{
-			TransactionId xmin;
-			Snapshot snap;
+			if (detached_exist)
+				*detached_exist = true;
 
-			xmin = HeapTupleHeaderGetXmin(inheritsTuple->t_data);
-			snap = GetActiveSnapshot();
+			if (omit_detached && ActiveSnapshotSet())
+			{
+				TransactionId xmin;
+				Snapshot	snap;
 
-			if (!XidInMVCCSnapshot(xmin, snap))
-				continue;
+				xmin = HeapTupleHeaderGetXmin(inheritsTuple->t_data);
+				snap = GetActiveSnapshot();
+
+				if (!XidInMVCCSnapshot(xmin, snap))
+					continue;
+			}
 		}
 
 		inhrelid = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhrelid;
@@ -235,8 +249,8 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, false,
-													lockmode);
+		currentchildren = find_inheritance_children(currentrel, true,
+													lockmode, NULL);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
@@ -524,7 +538,7 @@ DeleteInheritsTuple(Oid inhrelid, Oid inhparent, bool expect_detach_pending,
 		parent = ((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent;
 		if (!OidIsValid(inhparent) || parent == inhparent)
 		{
-			bool	detach_pending;
+			bool		detach_pending;
 
 			detach_pending =
 				((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhdetachpending;
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 096a6f2891..31da913d2b 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3507,7 +3507,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, false, NoLock) != NIL)
+			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3706,7 +3706,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, false, NoLock) != NIL)
+				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6580,7 +6580,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, false, NoLock) != NIL)
+		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6826,7 +6826,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7689,7 +7689,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode))
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8297,7 +8297,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
 
 	if (children)
 	{
@@ -8785,7 +8785,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11318,8 +11318,8 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children =
-			find_inheritance_children(RelationGetRelid(rel), false, lockmode);
+		children = find_inheritance_children(RelationGetRelid(rel), true,
+											 lockmode, NULL);
 	else
 		children = NIL;
 
@@ -11703,8 +11703,8 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), false,
-									   NoLock) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), true,
+									   NoLock, NULL) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index 3421014e47..2dce62999d 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, false,
-											 ShareRowExclusiveLock);
+			idxs = find_inheritance_children(indexOid, true,
+											 ShareRowExclusiveLock, NULL);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 99780ebb96..bbfc9ef3fa 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -991,14 +991,11 @@ ExecInitPartitionDispatchInfo(EState *estate,
 
 	/*
 	 * For data modification, it is better that executor does not include
-	 * partitions being detached, except in snapshot-isolation mode.  This
-	 * means that a read-committed transaction immediately gets a "no
-	 * partition for tuple" error when a tuple is inserted into a partition
-	 * that's being detached concurrently, but a transaction in repeatable-
-	 * read mode can still use the partition.  Note that because partition
-	 * detach uses ShareLock on the partition (which conflicts with DML),
-	 * we're certain that the detach won't be able to complete until any
-	 * inserting transaction is done.
+	 * partitions being detached, except when running in snapshot-isolation
+	 * mode.  This means that a read-committed transaction immediately gets a
+	 * "no partition for tuple" error when a tuple is inserted into a
+	 * partition that's being detached concurrently, but a transaction in
+	 * repeatable-read mode can still use such a partition.
 	 */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
@@ -1571,7 +1568,7 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	ListCell   *lc;
 	int			i;
 
-	/* Executor must always include detached partitions */
+	/* For data reading, executor always includes detached partitions */
 	if (estate->es_partition_directory == NULL)
 		estate->es_partition_directory =
 			CreatePartitionDirectory(estate->es_query_cxt, true);
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 58570fecfd..7d019a72be 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -47,7 +47,8 @@ typedef struct PartitionDirectoryEntry
 	PartitionDesc pd;
 } PartitionDirectoryEntry;
 
-static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
+static PartitionDesc RelationBuildPartitionDesc(Relation rel,
+												bool include_detached);
 
 
 /*
@@ -64,14 +65,13 @@ static void RelationBuildPartitionDesc(Relation rel, bool include_detached);
 PartitionDesc
 RelationGetPartitionDesc(Relation rel, bool include_detached)
 {
-	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return NULL;
+	Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
-	if (unlikely(rel->rd_partdesc == NULL ||
-				 rel->rd_partdesc->includes_detached != include_detached))
-		RelationBuildPartitionDesc(rel, include_detached);
+	if (likely(rel->rd_partdesc &&
+			   (!rel->rd_partdesc->detached_exist || include_detached)))
+		return rel->rd_partdesc;
 
-	return rel->rd_partdesc;
+	return RelationBuildPartitionDesc(rel, include_detached);
 }
 
 /*
@@ -89,7 +89,7 @@ RelationGetPartitionDesc(Relation rel, bool include_detached)
  * that some of our callees allocate memory on their own which would be leaked
  * permanently.
  */
-static void
+static PartitionDesc
 RelationBuildPartitionDesc(Relation rel, bool include_detached)
 {
 	PartitionDesc partdesc;
@@ -98,6 +98,7 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	PartitionBoundSpec **boundspecs = NULL;
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
+	bool		detached_exist;
 	ListCell   *cell;
 	int			i,
 				nparts;
@@ -112,8 +113,8 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	 * concurrently, whatever this function returns will be accurate as of
 	 * some well-defined point in time.
 	 */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), include_detached,
-										NoLock);
+	inhoids = find_inheritance_children(RelationGetRelid(rel), !include_detached,
+										NoLock, &detached_exist);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -234,6 +235,7 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	partdesc = (PartitionDescData *)
 		MemoryContextAllocZero(new_pdcxt, sizeof(PartitionDescData));
 	partdesc->nparts = nparts;
+	partdesc->detached_exist = detached_exist;
 	/* If there are no partitions, the rest of the partdesc can stay zero */
 	if (nparts > 0)
 	{
@@ -241,7 +243,6 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 		partdesc->boundinfo = partition_bounds_copy(boundinfo, key);
 		partdesc->oids = (Oid *) palloc(nparts * sizeof(Oid));
 		partdesc->is_leaf = (bool *) palloc(nparts * sizeof(bool));
-		partdesc->includes_detached = include_detached;
 
 		/*
 		 * Assign OIDs from the original array into mapped indexes of the
@@ -276,7 +277,12 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	if (rel->rd_pdcxt != NULL)
 		MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
 	rel->rd_pdcxt = new_pdcxt;
-	rel->rd_partdesc = partdesc;
+
+	/* Store it into relcache, but only if no detached partitions exist */
+	if (!detached_exist)
+		rel->rd_partdesc = partdesc;
+
+	return partdesc;
 }
 
 /*
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 6d07e1b302..4d28ede5a6 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,8 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool include_detached,
-									   LOCKMODE lockmode);
+extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
+									   LOCKMODE lockmode, bool *detached_exist);
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/partitioning/partdesc.h b/src/include/partitioning/partdesc.h
index 7f03ff4271..41485c5b10 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -17,11 +17,19 @@
 
 /*
  * Information about partitions of a partitioned table.
+ *
+ * For partitioned tables where detached partitions exist, we only cache the
+ * descriptor that includes all partitions, including detached; when we're
+ * requested a descriptor without the detached partitions, we create one
+ * afresh each time.  (The reason for this is that the set of detached
+ * partitions that are visible to each caller depends on the snapshot it has,
+ * so it's pretty much impossible to evict a descriptor from cache at the
+ * right time.)
  */
 typedef struct PartitionDescData
 {
 	int			nparts;			/* Number of partitions */
-	bool		includes_detached;	/* Does it include detached partitions */
+	bool		detached_exist; /* Are there any detached partitions? */
 	Oid		   *oids;			/* Array of 'nparts' elements containing
 								 * partition OIDs in order of the their bounds */
 	bool	   *is_leaf;		/* Array of 'nparts' elements storing whether
diff --git a/src/test/isolation/expected/detach-partition-concurrently-4.out b/src/test/isolation/expected/detach-partition-concurrently-4.out
index 90a75cb077..2167675374 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-4.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-4.out
@@ -324,6 +324,7 @@ a
 1              
 2              
 step s1insert: insert into d4_fk values (1);
+ERROR:  insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"
 step s1c: commit;
 
 starting permutation: s2snitch s1b s1s s2detach s1cancel s3vacfreeze s1s s1insert s1c
-- 
2.20.1

0002-Make-partitions-omitted-partdescs-have-stmt-lifetime.patchtext/x-diff; charset=us-asciiDownload
From 1b1106cace29025ef96b364e5bcd5bc7930c2eb7 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Wed, 21 Apr 2021 15:42:03 -0400
Subject: [PATCH 2/2] Make partitions-omitted partdescs have stmt lifetime

---
 src/backend/partitioning/partdesc.c | 50 ++++++++++++++++++++---------
 1 file changed, 35 insertions(+), 15 deletions(-)

diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 7d019a72be..399390b42c 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -88,6 +88,12 @@ RelationGetPartitionDesc(Relation rel, bool include_detached)
  * context the current context except in very brief code sections, out of fear
  * that some of our callees allocate memory on their own which would be leaked
  * permanently.
+ *
+ * As a special case, partition descriptors that are requested to omit
+ * partitions being detached (and which contain such partitions) are transient
+ * and are not to be associated with the relcache entry.  Such descriptors only
+ * last through the requesting Portal, so we use the corresponding memory
+ * context for them.
  */
 static PartitionDesc
 RelationBuildPartitionDesc(Relation rel, bool include_detached)
@@ -262,25 +268,39 @@ RelationBuildPartitionDesc(Relation rel, bool include_detached)
 	}
 
 	/*
-	 * We have a fully valid partdesc ready to store into the relcache.
-	 * Reparent it so it has the right lifespan.
+	 * We have a fully valid partdesc.  Reparent it so that it has the right
+	 * lifespan, and if appropriate put it into the relation's relcache entry.
 	 */
-	MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
+	if (!include_detached && detached_exist)
+	{
+		/*
+		 * A transient partition descriptor is only good for the current
+		 * statement, so make it a child of the current portal's context.
+		 */
+		MemoryContextSetParent(new_pdcxt, PortalContext);
+	}
+	else
+	{
+		/*
+		 * This partdesc goes into relcache.
+		 */
 
-	/*
-	 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
-	 * partition descriptor that may still be referenced somewhere.  Preserve
-	 * it, while not leaking it, by reattaching it as a child context of the
-	 * new rd_pdcxt.  Eventually it will get dropped by either RelationClose
-	 * or RelationClearRelation.
-	 */
-	if (rel->rd_pdcxt != NULL)
-		MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
-	rel->rd_pdcxt = new_pdcxt;
+		MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
 
-	/* Store it into relcache, but only if no detached partitions exist */
-	if (!detached_exist)
+		/*
+		 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
+		 * partition descriptor that may still be referenced somewhere.
+		 * Preserve it, while not leaking it, by reattaching it as a child
+		 * context of the new rd_pdcxt.  Eventually it will get dropped by
+		 * either RelationClose or RelationClearRelation.
+		 */
+		if (rel->rd_pdcxt != NULL)
+			MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
+		rel->rd_pdcxt = new_pdcxt;
+
+		/* Store it into relcache */
 		rel->rd_partdesc = partdesc;
+	}
 
 	return partdesc;
 }
-- 
2.20.1

#62Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Justin Pryzby (#51)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-10, Justin Pryzby wrote:

If it *implies* the partition constraint, then it's at least as tight (and
maybe tighter), yes ?

I think you're concerned with the case that someone has a partition with
"tight" bounds like (a>=200 and a<300) and a check constraint that's "less
tight" like (a>=100 AND a<400). In that case, the loose check constraint
doesn't imply the tighter partition constraint, so your patch would add a
non-redundant constraint.

... yeah, you're right, we can do as you suggest and it seems an
improvement. I verified, as is obvious in hindsight, that the existing
constraint makes a future ATTACH of the partition with the same bounds
as before not scan the partition.

I pushed the patch with a small change:
PartConstraintImpliedByRelConstraint wants the constraint in
implicit-AND form (that is, a list) which is what we already have, so we
can postpone make_ands_explicit() until later.

Pushed, thanks,

--
�lvaro Herrera Valdivia, Chile

#63Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#61)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

(Sorry about being away from this for over a week.)

On Thu, Apr 22, 2021 at 5:39 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

While the approach in the previous email does pass the tests, I think
(but couldn't find a test case to prove) it does so coincidentally, not
because it is correct. If I make the test for "detached exist" use the
cached omits-partitions-partdesc, it does fail, because we had
previously cached one that was not yet omitting the partition. So what
I said earlier in the thread stands: the set of partitions that are
considered detached changes depending on what the active snapshot is,
and therefore we *must not* cache any such descriptor.

So I backtracked to my previous proposal, which saves in relcache only
the partdesc that includes all partitions. If any partdesc is built
that omits partitions being detached, that one must be rebuilt afresh
each time. And to avoid potentially saving a lot of single-use
partdescs in CacheMemoryContext, in the attached second patch (which I
attach separately only to make it more obvious to review) I store such
partdescs in PortalContext.

Barring objections, I will get this pushed early tomorrow.

Thanks for updating the patch. I have mostly cosmetic comments.

Reading through the latest one, seeing both include_detached and
omit_detached being used in different parts of the code makes it a bit
hard to keep in mind what a given code path is doing wrt detached
partitions. How about making it all omit_detached?

         * Cope with partitions concurrently being detached.  When we see a
-        * partition marked "detach pending", we only include it in the set of
-        * visible partitions if caller requested all detached partitions, or
-        * if its pg_inherits tuple's xmin is still visible to the active
-        * snapshot.
+        * partition marked "detach pending", we omit it from the returned
+        * descriptor if caller requested that and the tuple's xmin does not
+        * appear in progress to the active snapshot.

It seems odd for a comment in find_inheritance_children() to talk
about the "descriptor". Maybe the earlier "set of visible
partitions" wording was fine?

-        * The reason for this check is that we want to avoid seeing the
+        * The reason for this hack is that we want to avoid seeing the
         * partition as alive in RI queries during REPEATABLE READ or
<snip>
+        * SERIALIZABLE transactions.

The comment doesn't quite make it clear why it is the RI query case
that necessitates this hack in the first case. Maybe the relation to
what's going on with the partdesc

+   if (likely(rel->rd_partdesc &&
+              (!rel->rd_partdesc->detached_exist || include_detached)))
+       return rel->rd_partdesc;

I think it would help to have a comment about what's going here, in
addition to the description you already wrote for
PartitionDescData.detached_exist. Maybe something along the lines of:

===
Under normal circumstances, we just return the partdesc that was
already built. However, if the partdesc was built at a time when
there existed detach-pending partition(s), which the current caller
would rather not see (omit_detached), then we build one afresh
omitting any such partitions and return that one.
RelationBuildPartitionDesc() makes sure that any such partdescs will
disappear when the query finishes.
===

That's maybe a bit verbose but I am sure you will find a way to write
it more succinctly.

BTW, I do feel a bit alarmed by the potential performance impact of
this. If detached_exist of a cached partdesc is true, then RI queries
invoked during a bulk DML operation would have to rebuild one for
every tuple to be checked, right? I haven't tried an actual example
yet though.

--
Amit Langote
EDB: http://www.enterprisedb.com

#64Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#63)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-22, Amit Langote wrote:

On Thu, Apr 22, 2021 at 5:39 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Reading through the latest one, seeing both include_detached and
omit_detached being used in different parts of the code makes it a bit
hard to keep in mind what a given code path is doing wrt detached
partitions. How about making it all omit_detached?

Yeah, I hesitated but wanted to do that too. Done.

* Cope with partitions concurrently being detached.  When we see a
-        * partition marked "detach pending", we only include it in the set of
-        * visible partitions if caller requested all detached partitions, or
-        * if its pg_inherits tuple's xmin is still visible to the active
-        * snapshot.
+        * partition marked "detach pending", we omit it from the returned
+        * descriptor if caller requested that and the tuple's xmin does not
+        * appear in progress to the active snapshot.

It seems odd for a comment in find_inheritance_children() to talk
about the "descriptor". Maybe the earlier "set of visible
partitions" wording was fine?

Absolutely -- done that way.

-        * The reason for this check is that we want to avoid seeing the
+        * The reason for this hack is that we want to avoid seeing the
* partition as alive in RI queries during REPEATABLE READ or
<snip>
+        * SERIALIZABLE transactions.

The comment doesn't quite make it clear why it is the RI query case
that necessitates this hack in the first case.

I added "such queries use a different snapshot than the one used by
regular (user) queries." I hope that's sufficient.

Maybe the relation to what's going on with the partdesc

+   if (likely(rel->rd_partdesc &&
+              (!rel->rd_partdesc->detached_exist || include_detached)))
+       return rel->rd_partdesc;

I think it would help to have a comment about what's going here, in
addition to the description you already wrote for
PartitionDescData.detached_exist. Maybe something along the lines of:

===
Under normal circumstances, we just return the partdesc that was
already built. However, if the partdesc was built at a time when
there existed detach-pending partition(s), which the current caller
would rather not see (omit_detached), then we build one afresh
omitting any such partitions and return that one.
RelationBuildPartitionDesc() makes sure that any such partdescs will
disappear when the query finishes.
===

That's maybe a bit verbose but I am sure you will find a way to write
it more succinctly.

I added some text in this spot, and also wrote some more in the comment
above RelationGetPartitionDesc and RelationBuildPartitionDesc.

BTW, I do feel a bit alarmed by the potential performance impact of
this. If detached_exist of a cached partdesc is true, then RI queries
invoked during a bulk DML operation would have to rebuild one for
every tuple to be checked, right? I haven't tried an actual example
yet though.

Yeah, I was scared about that too (which is why I insisted on trying to
add a cached copy of the partdesc omitting detached partitions). But
AFAICS what happens is that the plan for the RI query gets cached after
a few tries; so we do build the partdesc a few times at first, but later
we use the cached plan and so we no longer use that one. So at least in
the normal cases this isn't a serious problem that I can see.

I pushed it now. Thanks for your help,

--
�lvaro Herrera Valdivia, Chile
"How strange it is to find the words "Perl" and "saner" in such close
proximity, with no apparent sense of irony. I doubt that Larry himself
could have managed it." (ncm, http://lwn.net/Articles/174769/)

#65Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#64)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Fri, Apr 23, 2021 at 4:26 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2021-Apr-22, Amit Langote wrote:

-        * The reason for this check is that we want to avoid seeing the
+        * The reason for this hack is that we want to avoid seeing the
* partition as alive in RI queries during REPEATABLE READ or
<snip>
+        * SERIALIZABLE transactions.

The comment doesn't quite make it clear why it is the RI query case
that necessitates this hack in the first case.

I added "such queries use a different snapshot than the one used by
regular (user) queries." I hope that's sufficient.

Yeah, that makes sense.

Maybe the relation to what's going on with the partdesc

(I had to leave my desk while in the middle of typing this, but I
forget what I was going to add :()

BTW, I do feel a bit alarmed by the potential performance impact of
this. If detached_exist of a cached partdesc is true, then RI queries
invoked during a bulk DML operation would have to rebuild one for
every tuple to be checked, right? I haven't tried an actual example
yet though.

Yeah, I was scared about that too (which is why I insisted on trying to
add a cached copy of the partdesc omitting detached partitions). But
AFAICS what happens is that the plan for the RI query gets cached after
a few tries; so we do build the partdesc a few times at first, but later
we use the cached plan and so we no longer use that one. So at least in
the normal cases this isn't a serious problem that I can see.

Actually, ri_trigger.c (or really plancache.c) is not very good at
caching the plan when querying partitioned tables; it always chooses
to replan because a generic plan, even with runtime pruning built into
it, looks very expensive compared to a custom one. Now that's a
problem we will have to fix sooner than later, but until then we have
to work around it.

Here is an example that shows the problem:

create unlogged table pk_parted (a int primary key) partition by range (a);
select 'create unlogged table pk_parted_' || i || ' partition of
pk_parted for values from (' || (i-1) * 1000 + 1 || ') to (' || i *
1000 + 1 || ');' from generate_series(1, 1000) i;
\gexec
create unlogged table fk (a int references pk_parted);
insert into pk_parted select generate_series(1, 10000);
begin;
select * from fk_parted where a = 1;

In another session:

alter table pk_parted detach partition pk_parted_1000 concurrently;
<blocks; cancel using ctrl-c>

Back in the 1st session:

end;
insert into fk select generate_series(1, 10000);
INSERT 0 10000
Time: 47400.792 ms (00:47.401)

The insert took unusually long, because the PartitionDesc for
pk_parted had to be built exactly 10000 times, because there's a
detach-pending partition lying around. There is also a danger of an
OOM with such an insert because of leaking into PortalContext the
memory of every PartitionDesc thus built, especially with larger
counts of PK partitions and rows inserted into the FK table.

Also, I noticed that all queries on pk_parted, not just the RI
queries, have to build the PartitionDesc every time, so take that much
longer:

-- note the planning time
explain analyze select * from pk_parted where a = 1;
QUERY
PLAN

-------------------------------------------------------------------------------------------------------------------------------
--------------
Index Only Scan using pk_parted_1_pkey on pk_parted_1 pk_parted
(cost=0.28..8.29 rows=1 width=4) (actual time=0.016..0.017 ro
ws=1 loops=1)
Index Cond: (a = 1)
Heap Fetches: 1
Planning Time: 7.543 ms
Execution Time: 0.044 ms
(5 rows)

Finalizing the detach makes the insert and the query finish in normal
time, because the PartitionDesc can be cached again:

alter table pk_parted detach partition pk_parted_1000 finalize;
insert into fk select generate_series(1, 10000);
INSERT 0 10000
Time: 855.336 ms

explain analyze select * from pk_parted where a = 1;
QUERY
PLAN

-------------------------------------------------------------------------------------------------------------------------------
--------------
Index Only Scan using pk_parted_1_pkey on pk_parted_1 pk_parted
(cost=0.28..8.29 rows=1 width=4) (actual time=0.033..0.036 ro
ws=1 loops=1)
Index Cond: (a = 1)
Heap Fetches: 1
Planning Time: 0.202 ms
Execution Time: 0.075 ms
(5 rows)

I am afraid we may have to fix this in the code after all, because
there does not seem a good way to explain this away in the
documentation. If I read correctly, you did try an approach of
caching the PartitionDesc that we currently don't, no?

--
Amit Langote
EDB: http://www.enterprisedb.com

#66Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#65)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-23, Amit Langote wrote:

Back in the 1st session:

end;
insert into fk select generate_series(1, 10000);
INSERT 0 10000
Time: 47400.792 ms (00:47.401)

I guess I was wrong about that ... the example I tried didn't have 1000s
of partitions, and I used debug print-outs to show when a new partdesc
was being rebuilt, and only six were occurring. I'm not sure why my
case behaves so differently from yours, but clearly from the timing this
is not good.

I am afraid we may have to fix this in the code after all, because
there does not seem a good way to explain this away in the
documentation.

Yeah, looking at this case, I agree that it needs a fix of some kind.

If I read correctly, you did try an approach of caching the
PartitionDesc that we currently don't, no?

I think the patch I posted was too simple. I think a real fix requires
us to keep track of exactly in what way the partdesc is outdated, so
that we can compare to the current situation in deciding to use that
partdesc or build a new one. For example, we could keep track of a list
of OIDs of detached partitions (and it simplifies things a lot if allow
only a single partition in this situation, because it's either zero OIDs
or one OID); or we can save the Xmin of the pg_inherits tuple for the
detached partition (and we can compare that Xmin to our current active
snapshot and decide whether to use that partdesc or not).

I'll experiment with this a little more and propose a patch later today.

I don't think it's too much of a problem to state that you need to
finalize one detach before you can do the next one. After all, with
regular detach, you'd have the tables exclusively locked so you can't do
them in parallel anyway. (It also increases the chances that people
will finalize detach operations that went unnoticed.)

--
�lvaro Herrera 39�49'30"S 73�17'W

#67Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#66)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-23, Alvaro Herrera wrote:

I think the patch I posted was too simple. I think a real fix requires
us to keep track of exactly in what way the partdesc is outdated, so
that we can compare to the current situation in deciding to use that
partdesc or build a new one. For example, we could keep track of a list
of OIDs of detached partitions (and it simplifies things a lot if allow
only a single partition in this situation, because it's either zero OIDs
or one OID); or we can save the Xmin of the pg_inherits tuple for the
detached partition (and we can compare that Xmin to our current active
snapshot and decide whether to use that partdesc or not).

I'll experiment with this a little more and propose a patch later today.

This (POC-quality) seems to do the trick.

(I restored the API of find_inheritance_children, because it was getting
a little obnoxious. I haven't thought this through but I think we
should do something like it.)

I don't think it's too much of a problem to state that you need to
finalize one detach before you can do the next one. After all, with
regular detach, you'd have the tables exclusively locked so you can't do
them in parallel anyway. (It also increases the chances that people
will finalize detach operations that went unnoticed.)

I haven't added a mechanism to verify this; but with asserts on, this
patch will crash if you have more than one. I think the behavior is not
necessarily sane with asserts off, since you'll get an arbitrary
detach-Xmin assigned to the partdesc, depending on catalog scan order.

--
�lvaro Herrera Valdivia, Chile
"El hombre nunca sabe de lo que es capaz hasta que lo intenta" (C. Dickens)

Attachments:

partdesc-faster.patchtext/x-diff; charset=us-asciiDownload
diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 6447b52854..e3ee72c0b7 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -63,8 +63,16 @@ typedef struct SeenRelsEntry
  * committed to the active snapshot.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool omit_detached,
-						  LOCKMODE lockmode, bool *detached_exist)
+find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+{
+	return find_inheritance_children_extended(parentrelId, true, lockmode,
+											  NULL, NULL);
+}
+
+List *
+find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
+								   LOCKMODE lockmode, bool *detached_exist,
+								   TransactionId *detached_xmin)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -132,7 +140,15 @@ find_inheritance_children(Oid parentrelId, bool omit_detached,
 				snap = GetActiveSnapshot();
 
 				if (!XidInMVCCSnapshot(xmin, snap))
+				{
+					/* XXX only one detached partition allowed */
+					if (detached_xmin)
+					{
+						Assert(*detached_xmin == InvalidTransactionId);
+						*detached_xmin = xmin;
+					}
 					continue;
+				}
 			}
 		}
 
@@ -247,8 +263,7 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, true,
-													lockmode, NULL);
+		currentchildren = find_inheritance_children(currentrel, lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 7d00f4eb25..dd931d09af 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3507,7 +3507,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+			find_inheritance_children(myrelid, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3706,7 +3706,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+				find_inheritance_children(myrelid, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6580,7 +6580,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+		find_inheritance_children(myrelid, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6826,7 +6826,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7689,7 +7689,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
+		find_inheritance_children(RelationGetRelid(rel), lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8297,7 +8297,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	if (children)
 	{
@@ -8785,7 +8785,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11318,8 +11318,7 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), true,
-											 lockmode, NULL);
+		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
 	else
 		children = NIL;
 
@@ -11703,8 +11702,7 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), true,
-									   NoLock, NULL) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index d8393aa4de..f305f8bc0f 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, true,
-											 ShareRowExclusiveLock, NULL);
+			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 2305dff407..73fa0459a1 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -78,11 +78,33 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
 	 * If relcache has a partition descriptor, use that.  However, we can only
 	 * 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.
 	 */
 	if (likely(rel->rd_partdesc &&
-			   (!rel->rd_partdesc->detached_exist || !omit_detached)))
+			   (!rel->rd_partdesc->detached_exist || !omit_detached ||
+				!ActiveSnapshotSet())))
 		return rel->rd_partdesc;
 
+	/*
+	 * If we're asked to omit detached partitions, we may be able to use a
+	 * cached version of it, if it was built with the same conditions that
+	 * we see now.
+	 */
+	if (omit_detached &&
+		rel->rd_partdesc_nodetached &&
+		TransactionIdIsValid(rel->rd_partdesc_nodetached_xmin) &&
+		ActiveSnapshotSet())
+	{
+		Snapshot	activesnap;
+
+		activesnap = GetActiveSnapshot();
+
+		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin, activesnap))
+			return rel->rd_partdesc_nodetached;
+	}
+
 	return RelationBuildPartitionDesc(rel, omit_detached);
 }
 
@@ -117,6 +139,7 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
 	bool		detached_exist;
+	TransactionId detached_xmin;
 	ListCell   *cell;
 	int			i,
 				nparts;
@@ -132,8 +155,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	 * some well-defined point in time.
 	 */
 	detached_exist = false;
-	inhoids = find_inheritance_children(RelationGetRelid(rel), omit_detached,
-										NoLock, &detached_exist);
+	detached_xmin = InvalidTransactionId;
+	inhoids = find_inheritance_children_extended(RelationGetRelid(rel),
+												 omit_detached, NoLock,
+												 &detached_exist,
+												 &detached_xmin);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -282,36 +308,29 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 
 	/*
 	 * We have a fully valid partdesc.  Reparent it so that it has the right
-	 * lifespan, and if appropriate put it into the relation's relcache entry.
+	 * lifespan.
 	 */
-	if (omit_detached && detached_exist)
+	MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
+
+	/*
+	 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
+	 * partition descriptor that may still be referenced somewhere.
+	 * Preserve it, while not leaking it, by reattaching it as a child
+	 * context of the new rd_pdcxt.  Eventually it will get dropped by
+	 * either RelationClose or RelationClearRelation.
+	 */
+	if (rel->rd_pdcxt != NULL)
+		MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
+	rel->rd_pdcxt = new_pdcxt;
+
+	/* Store it into relcache */
+	if (omit_detached && detached_exist && ActiveSnapshotSet())
 	{
-		/*
-		 * A transient partition descriptor is only good for the current
-		 * statement, so make it a child of the current portal's context.
-		 */
-		MemoryContextSetParent(new_pdcxt, PortalContext);
+		rel->rd_partdesc_nodetached = partdesc;
+		rel->rd_partdesc_nodetached_xmin = detached_xmin;
 	}
 	else
 	{
-		/*
-		 * This partdesc goes into relcache.
-		 */
-
-		MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
-
-		/*
-		 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
-		 * partition descriptor that may still be referenced somewhere.
-		 * Preserve it, while not leaking it, by reattaching it as a child
-		 * context of the new rd_pdcxt.  Eventually it will get dropped by
-		 * either RelationClose or RelationClearRelation.
-		 */
-		if (rel->rd_pdcxt != NULL)
-			MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
-		rel->rd_pdcxt = new_pdcxt;
-
-		/* Store it into relcache */
 		rel->rd_partdesc = partdesc;
 	}
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 4d28ede5a6..f47925588a 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,10 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
-									   LOCKMODE lockmode, bool *detached_exist);
+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);
+
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 34e25eb597..6e5a5bf4a7 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -127,6 +127,9 @@ typedef struct RelationData
 
 	/* data managed by RelationGetPartitionDesc: */
 	PartitionDesc rd_partdesc;	/* partition descriptor, or NULL */
+	PartitionDesc rd_partdesc_nodetached;	/* partition descriptor */
+	TransactionId rd_partdesc_nodetached_xmin;	/* xmin of partdesc */
+
 	MemoryContext rd_pdcxt;		/* private context for rd_partdesc, if any */
 
 	/* data managed by RelationGetPartitionQual: */
#68Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#67)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hi Alvaro,

On Sat, Apr 24, 2021 at 8:31 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2021-Apr-23, Alvaro Herrera wrote:

I think the patch I posted was too simple. I think a real fix requires
us to keep track of exactly in what way the partdesc is outdated, so
that we can compare to the current situation in deciding to use that
partdesc or build a new one. For example, we could keep track of a list
of OIDs of detached partitions (and it simplifies things a lot if allow
only a single partition in this situation, because it's either zero OIDs
or one OID); or we can save the Xmin of the pg_inherits tuple for the
detached partition (and we can compare that Xmin to our current active
snapshot and decide whether to use that partdesc or not).

I'll experiment with this a little more and propose a patch later today.

This (POC-quality) seems to do the trick.

Thanks for the patch.

(I restored the API of find_inheritance_children, because it was getting
a little obnoxious. I haven't thought this through but I think we
should do something like it.)

+1.

I don't think it's too much of a problem to state that you need to
finalize one detach before you can do the next one. After all, with
regular detach, you'd have the tables exclusively locked so you can't do
them in parallel anyway. (It also increases the chances that people
will finalize detach operations that went unnoticed.)

That sounds reasonable.

I haven't added a mechanism to verify this; but with asserts on, this
patch will crash if you have more than one. I think the behavior is not
necessarily sane with asserts off, since you'll get an arbitrary
detach-Xmin assigned to the partdesc, depending on catalog scan order.

Maybe this is an ignorant question but is the plan to add an elog() in
this code path or a check (and an ereport()) somewhere in
ATExecDetachPartition() to prevent more than one partition ending up
in detach-pending state?

Please allow me to study the patch a bit more closely and get back tomorrow.

--
Amit Langote
EDB: http://www.enterprisedb.com

#69Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#68)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hello Amit,

On 2021-Apr-26, Amit Langote wrote:

On Sat, Apr 24, 2021 at 8:31 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I haven't added a mechanism to verify this; but with asserts on, this
patch will crash if you have more than one. I think the behavior is not
necessarily sane with asserts off, since you'll get an arbitrary
detach-Xmin assigned to the partdesc, depending on catalog scan order.

Maybe this is an ignorant question but is the plan to add an elog() in
this code path or a check (and an ereport()) somewhere in
ATExecDetachPartition() to prevent more than one partition ending up
in detach-pending state?

Yeah, that's what I'm planning to do.

Please allow me to study the patch a bit more closely and get back tomorrow.

Sure, thanks!

--
�lvaro Herrera Valdivia, Chile
"But static content is just dynamic content that isn't moving!"
http://smylers.hates-software.com/2007/08/15/fe244d0c.html

#70Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#69)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-26, Alvaro Herrera wrote:

Please allow me to study the patch a bit more closely and get back tomorrow.

Sure, thanks!

Here's a more polished version.

After trying the version with the elog(ERROR) when two detached
partitions are present, I decided against it; it is unhelpful because
it doesn't let you build partition descriptors for anything. So I made
it an elog(WARNING) (not an ereport, note), and keep the most recent
pg_inherits.xmin value. This is not great, but it lets you out of the
situation by finalizing one of the detaches.

The other check (at ALTER TABLE .. DETACH time) is an ereport(ERROR) and
should make the first one unreachable.

--
�lvaro Herrera 39�49'30"S 73�17'W

Attachments:

0001-Allow-a-partdesc-with-omitted-partitions-to-be-cache.patchtext/x-diff; charset=us-asciiDownload
From 89e8a7d170fc0c6701e536b9c5830e3a58f303cc Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 26 Apr 2021 14:53:04 -0400
Subject: [PATCH] Allow a partdesc-with-omitted-partitions to be cached

Makes partdesc acquisition faster during the transient period during
which a partition is in the process of being dropped.

Per gripe from Amit Langote
Discussion: https://postgr.es/m/CA+HiwqFgpP1LxJZOBYGt9rpvTjXXkg5qG2+Xch2Z1Q7KrqZR1A@mail.gmail.com
---
 src/backend/catalog/pg_inherits.c             | 52 +++++++++++--
 src/backend/commands/tablecmds.c              | 66 +++++++++-------
 src/backend/commands/trigger.c                |  3 +-
 src/backend/partitioning/partdesc.c           | 75 ++++++++++++-------
 src/include/catalog/pg_inherits.h             |  6 +-
 src/include/utils/rel.h                       |  3 +
 .../detach-partition-concurrently-3.out       | 57 +++++++++++++-
 .../detach-partition-concurrently-3.spec      |  9 ++-
 8 files changed, 205 insertions(+), 66 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 6447b52854..c373faf2d6 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,6 +52,22 @@ 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
+ * find_inheritance_children_extended for details.
+ */
+List *
+find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+{
+	return find_inheritance_children_extended(parentrelId, true, lockmode,
+											  NULL, NULL);
+}
+
+/*
+ * find_inheritance_children_extended
+ *
+ * As find_inheritance_children, with more options regarding detached
+ * partitions.
+ *
  * If a partition's pg_inherits row is marked "detach pending",
  * *detached_exist (if not null) is set true.
  *
@@ -60,11 +76,13 @@ typedef struct SeenRelsEntry
  * 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.
+ * committed to the active snapshot.  In addition, *detached_xmin (if not null)
+ * is set to the xmin of the row of the detached partition.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool omit_detached,
-						  LOCKMODE lockmode, bool *detached_exist)
+find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
+								   LOCKMODE lockmode, bool *detached_exist,
+								   TransactionId *detached_xmin)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -132,7 +150,32 @@ find_inheritance_children(Oid parentrelId, bool omit_detached,
 				snap = GetActiveSnapshot();
 
 				if (!XidInMVCCSnapshot(xmin, snap))
+				{
+					if (detached_xmin)
+					{
+						/*
+						 * Two detached partitions should not occur (see
+						 * checks in MarkInheritDetached), but if they do,
+						 * track the newer of the two.  Make sure to warn the
+						 * user, so that they can clean up.  Since this is
+						 * just a cross-check against potentially corrupt
+						 * catalogs, we don't make it a full-fledged error
+						 * message.
+						 */
+						if (*detached_xmin != InvalidTransactionId)
+						{
+							elog(WARNING, "more than one partition pending detach found for table with OID %u",
+								 parentrelId);
+							if (TransactionIdFollows(xmin, *detached_xmin))
+								*detached_xmin = xmin;
+						}
+						else
+							*detached_xmin = xmin;
+					}
+
+					/* Don't add the partition to the output list */
 					continue;
+				}
 			}
 		}
 
@@ -247,8 +290,7 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, true,
-													lockmode, NULL);
+		currentchildren = find_inheritance_children(currentrel, lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 7d00f4eb25..ab3c61898c 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3507,7 +3507,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+			find_inheritance_children(myrelid, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3706,7 +3706,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+				find_inheritance_children(myrelid, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6580,7 +6580,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+		find_inheritance_children(myrelid, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6826,7 +6826,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7689,7 +7689,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
+		find_inheritance_children(RelationGetRelid(rel), lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8297,7 +8297,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	if (children)
 	{
@@ -8785,7 +8785,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11318,8 +11318,7 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), true,
-											 lockmode, NULL);
+		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
 	else
 		children = NIL;
 
@@ -11703,8 +11702,7 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), true,
-									   NoLock, NULL) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14616,7 +14614,8 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * MarkInheritDetached
  *
  * Set inhdetachpending for a partition, for ATExecDetachPartition
- * in concurrent mode.
+ * in concurrent mode.  While at it, verify that no other partition is
+ * already pending detach.
  */
 static void
 MarkInheritDetached(Relation child_rel, Relation parent_rel)
@@ -14632,32 +14631,45 @@ MarkInheritDetached(Relation child_rel, Relation parent_rel)
 	Assert(parent_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
 	/*
-	 * Find pg_inherits entries by inhrelid.
+	 * Find pg_inherits entries by inhparent.  (We need to scan them all in
+	 * order to verify that no other partition is pending detach.)
 	 */
 	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
 	ScanKeyInit(&key,
-				Anum_pg_inherits_inhrelid,
+				Anum_pg_inherits_inhparent,
 				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(RelationGetRelid(child_rel)));
-	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+				ObjectIdGetDatum(RelationGetRelid(parent_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsParentIndexId,
 							  true, NULL, 1, &key);
 
 	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
 	{
-		HeapTuple	newtup;
+		Form_pg_inherits inhForm;
 
-		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
-			RelationGetRelid(parent_rel))
-			elog(ERROR, "bad parent tuple found for partition %u",
-				 RelationGetRelid(child_rel));
+		inhForm = (Form_pg_inherits) GETSTRUCT(inheritsTuple);
+		if (inhForm->inhdetachpending)
+			ereport(ERROR,
+					errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("partition \"%s\" already pending detach in partitioned table \"%s.%s\"",
+						   get_rel_name(inhForm->inhrelid),
+						   get_namespace_name(parent_rel->rd_rel->relnamespace),
+						   RelationGetRelationName(parent_rel)),
+					errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the detach operation."));
 
-		newtup = heap_copytuple(inheritsTuple);
-		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+		if (inhForm->inhrelid == RelationGetRelid(child_rel))
+		{
+			HeapTuple	newtup;
 
-		CatalogTupleUpdate(catalogRelation,
-						   &inheritsTuple->t_self,
-						   newtup);
-		found = true;
+			newtup = heap_copytuple(inheritsTuple);
+			((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+
+			CatalogTupleUpdate(catalogRelation,
+							   &inheritsTuple->t_self,
+							   newtup);
+			found = true;
+			heap_freetuple(newtup);
+			/* keep looking, to ensure we catch others pending detach */
+		}
 	}
 
 	/* Done */
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index d8393aa4de..f305f8bc0f 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, true,
-											 ShareRowExclusiveLock, NULL);
+			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 2305dff407..9af740f6e1 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -78,11 +78,33 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
 	 * If relcache has a partition descriptor, use that.  However, we can only
 	 * 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.
 	 */
 	if (likely(rel->rd_partdesc &&
-			   (!rel->rd_partdesc->detached_exist || !omit_detached)))
+			   (!rel->rd_partdesc->detached_exist || !omit_detached ||
+				!ActiveSnapshotSet())))
 		return rel->rd_partdesc;
 
+	/*
+	 * If we're asked to omit detached partitions, we may be able to use a
+	 * cached version of it, if it was built with the same conditions that we
+	 * see now.
+	 */
+	if (omit_detached &&
+		rel->rd_partdesc_nodetached &&
+		TransactionIdIsValid(rel->rd_partdesc_nodetached_xmin) &&
+		ActiveSnapshotSet())
+	{
+		Snapshot	activesnap;
+
+		activesnap = GetActiveSnapshot();
+
+		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin, activesnap))
+			return rel->rd_partdesc_nodetached;
+	}
+
 	return RelationBuildPartitionDesc(rel, omit_detached);
 }
 
@@ -117,6 +139,7 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
 	bool		detached_exist;
+	TransactionId detached_xmin;
 	ListCell   *cell;
 	int			i,
 				nparts;
@@ -132,8 +155,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	 * some well-defined point in time.
 	 */
 	detached_exist = false;
-	inhoids = find_inheritance_children(RelationGetRelid(rel), omit_detached,
-										NoLock, &detached_exist);
+	detached_xmin = InvalidTransactionId;
+	inhoids = find_inheritance_children_extended(RelationGetRelid(rel),
+												 omit_detached, NoLock,
+												 &detached_exist,
+												 &detached_xmin);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -282,36 +308,29 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 
 	/*
 	 * We have a fully valid partdesc.  Reparent it so that it has the right
-	 * lifespan, and if appropriate put it into the relation's relcache entry.
+	 * lifespan.
 	 */
-	if (omit_detached && detached_exist)
+	MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
+
+	/*
+	 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
+	 * partition descriptor that may still be referenced somewhere.
+	 * Preserve it, while not leaking it, by reattaching it as a child
+	 * context of the new rd_pdcxt.  Eventually it will get dropped by
+	 * either RelationClose or RelationClearRelation.
+	 */
+	if (rel->rd_pdcxt != NULL)
+		MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
+	rel->rd_pdcxt = new_pdcxt;
+
+	/* Store it into relcache */
+	if (omit_detached && detached_exist && ActiveSnapshotSet())
 	{
-		/*
-		 * A transient partition descriptor is only good for the current
-		 * statement, so make it a child of the current portal's context.
-		 */
-		MemoryContextSetParent(new_pdcxt, PortalContext);
+		rel->rd_partdesc_nodetached = partdesc;
+		rel->rd_partdesc_nodetached_xmin = detached_xmin;
 	}
 	else
 	{
-		/*
-		 * This partdesc goes into relcache.
-		 */
-
-		MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
-
-		/*
-		 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
-		 * partition descriptor that may still be referenced somewhere.
-		 * Preserve it, while not leaking it, by reattaching it as a child
-		 * context of the new rd_pdcxt.  Eventually it will get dropped by
-		 * either RelationClose or RelationClearRelation.
-		 */
-		if (rel->rd_pdcxt != NULL)
-			MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
-		rel->rd_pdcxt = new_pdcxt;
-
-		/* Store it into relcache */
 		rel->rd_partdesc = partdesc;
 	}
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 4d28ede5a6..f47925588a 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,10 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
-									   LOCKMODE lockmode, bool *detached_exist);
+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);
+
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 34e25eb597..6e5a5bf4a7 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -127,6 +127,9 @@ typedef struct RelationData
 
 	/* data managed by RelationGetPartitionDesc: */
 	PartitionDesc rd_partdesc;	/* partition descriptor, or NULL */
+	PartitionDesc rd_partdesc_nodetached;	/* partition descriptor */
+	TransactionId rd_partdesc_nodetached_xmin;	/* xmin of partdesc */
+
 	MemoryContext rd_pdcxt;		/* private context for rd_partdesc, if any */
 
 	/* data managed by RelationGetPartitionQual: */
diff --git a/src/test/isolation/expected/detach-partition-concurrently-3.out b/src/test/isolation/expected/detach-partition-concurrently-3.out
index 88e83638c7..dbb3825792 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-3.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-3.out
@@ -20,6 +20,7 @@ step s1describe: SELECT 'd3_listp' AS root, * FROM pg_partition_tree('d3_listp')
 root           relid          parentrelid    isleaf         level          
 
 d3_listp       d3_listp                      f              0              
+d3_listp       d3_listp2      d3_listp       t              1              
 d3_listp1      d3_listp1                     t              0              
 step s1alter: ALTER TABLE d3_listp1 ALTER a DROP NOT NULL;
 ERROR:  cannot alter partition "d3_listp1" with an incomplete detach
@@ -123,6 +124,61 @@ a
 
 1              
 
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detach2 s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+ERROR:  partition "d3_listp1" already pending detach in partitioned table "public.d3_listp"
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detachfinal s1c s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1droppart s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1droppart: DROP TABLE d3_listp1;
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
 starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2drop s1s s2commit
 step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
 step s1b: BEGIN;
@@ -279,4 +335,3 @@ step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
 step s1insertpart: INSERT INTO d3_listp1 VALUES (1); <waiting ...>
 step s2commit: COMMIT;
 step s1insertpart: <... completed>
-unused step name: s1droppart
diff --git a/src/test/isolation/specs/detach-partition-concurrently-3.spec b/src/test/isolation/specs/detach-partition-concurrently-3.spec
index 4b706430e1..4c563890eb 100644
--- a/src/test/isolation/specs/detach-partition-concurrently-3.spec
+++ b/src/test/isolation/specs/detach-partition-concurrently-3.spec
@@ -4,12 +4,13 @@ setup
 {
   CREATE TABLE d3_listp (a int) PARTITION BY LIST(a);
   CREATE TABLE d3_listp1 PARTITION OF d3_listp FOR VALUES IN (1);
+  CREATE TABLE d3_listp2 PARTITION OF d3_listp FOR VALUES IN (2);
   CREATE TABLE d3_pid (pid int);
   INSERT INTO d3_listp VALUES (1);
 }
 
 teardown {
-    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_pid;
+    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_listp2, d3_pid;
 }
 
 session "s1"
@@ -34,6 +35,7 @@ session "s2"
 step "s2begin"		{ BEGIN; }
 step "s2snitch"		{ INSERT INTO d3_pid SELECT pg_backend_pid(); }
 step "s2detach"		{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; }
+step "s2detach2"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; }
 step "s2detachfinal"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; }
 step "s2drop"		{ DROP TABLE d3_listp1; }
 step "s2commit"		{ COMMIT; }
@@ -49,6 +51,11 @@ permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1drop" "s1list"
 # "truncate" only does parent, not partition
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1trunc" "s1spart"
 
+# If a partition pending detach exists, we cannot drop another one
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detach2" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detachfinal" "s1c" "s2detach2"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1droppart" "s2detach2"
+
 # When a partition with incomplete detach is dropped, we grab lock on parent too.
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2drop" "s1s" "s2commit"
 
-- 
2.20.1

#71Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#70)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Sorry, I forgot to update some comments in that version. Fixed here.

--
�lvaro Herrera 39�49'30"S 73�17'W

Attachments:

v2-0001-Allow-a-partdesc-omitting-partitions-to-be-cached.patchtext/x-diff; charset=us-asciiDownload
From cb6d9e026624656e826ea880716ee552b15203a8 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 26 Apr 2021 14:53:04 -0400
Subject: [PATCH v2] Allow a partdesc-omitting-partitions to be cached

Makes partition descriptor acquisition faster during the transient
period in which a partition is in the process of being detached.

Per gripe from Amit Langote
Discussion: https://postgr.es/m/CA+HiwqFgpP1LxJZOBYGt9rpvTjXXkg5qG2+Xch2Z1Q7KrqZR1A@mail.gmail.com
---
 src/backend/catalog/pg_inherits.c             | 52 +++++++++-
 src/backend/commands/tablecmds.c              | 66 +++++++------
 src/backend/commands/trigger.c                |  3 +-
 src/backend/partitioning/partdesc.c           | 96 ++++++++++++-------
 src/include/catalog/pg_inherits.h             |  6 +-
 src/include/utils/rel.h                       |  3 +
 .../detach-partition-concurrently-3.out       | 57 ++++++++++-
 .../detach-partition-concurrently-3.spec      |  9 +-
 8 files changed, 219 insertions(+), 73 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 6447b52854..c373faf2d6 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,6 +52,22 @@ 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
+ * find_inheritance_children_extended for details.
+ */
+List *
+find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+{
+	return find_inheritance_children_extended(parentrelId, true, lockmode,
+											  NULL, NULL);
+}
+
+/*
+ * find_inheritance_children_extended
+ *
+ * As find_inheritance_children, with more options regarding detached
+ * partitions.
+ *
  * If a partition's pg_inherits row is marked "detach pending",
  * *detached_exist (if not null) is set true.
  *
@@ -60,11 +76,13 @@ typedef struct SeenRelsEntry
  * 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.
+ * committed to the active snapshot.  In addition, *detached_xmin (if not null)
+ * is set to the xmin of the row of the detached partition.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool omit_detached,
-						  LOCKMODE lockmode, bool *detached_exist)
+find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
+								   LOCKMODE lockmode, bool *detached_exist,
+								   TransactionId *detached_xmin)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -132,7 +150,32 @@ find_inheritance_children(Oid parentrelId, bool omit_detached,
 				snap = GetActiveSnapshot();
 
 				if (!XidInMVCCSnapshot(xmin, snap))
+				{
+					if (detached_xmin)
+					{
+						/*
+						 * Two detached partitions should not occur (see
+						 * checks in MarkInheritDetached), but if they do,
+						 * track the newer of the two.  Make sure to warn the
+						 * user, so that they can clean up.  Since this is
+						 * just a cross-check against potentially corrupt
+						 * catalogs, we don't make it a full-fledged error
+						 * message.
+						 */
+						if (*detached_xmin != InvalidTransactionId)
+						{
+							elog(WARNING, "more than one partition pending detach found for table with OID %u",
+								 parentrelId);
+							if (TransactionIdFollows(xmin, *detached_xmin))
+								*detached_xmin = xmin;
+						}
+						else
+							*detached_xmin = xmin;
+					}
+
+					/* Don't add the partition to the output list */
 					continue;
+				}
 			}
 		}
 
@@ -247,8 +290,7 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, true,
-													lockmode, NULL);
+		currentchildren = find_inheritance_children(currentrel, lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 7d00f4eb25..ab3c61898c 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3507,7 +3507,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+			find_inheritance_children(myrelid, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3706,7 +3706,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+				find_inheritance_children(myrelid, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6580,7 +6580,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+		find_inheritance_children(myrelid, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6826,7 +6826,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7689,7 +7689,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
+		find_inheritance_children(RelationGetRelid(rel), lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8297,7 +8297,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	if (children)
 	{
@@ -8785,7 +8785,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11318,8 +11318,7 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), true,
-											 lockmode, NULL);
+		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
 	else
 		children = NIL;
 
@@ -11703,8 +11702,7 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), true,
-									   NoLock, NULL) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14616,7 +14614,8 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * MarkInheritDetached
  *
  * Set inhdetachpending for a partition, for ATExecDetachPartition
- * in concurrent mode.
+ * in concurrent mode.  While at it, verify that no other partition is
+ * already pending detach.
  */
 static void
 MarkInheritDetached(Relation child_rel, Relation parent_rel)
@@ -14632,32 +14631,45 @@ MarkInheritDetached(Relation child_rel, Relation parent_rel)
 	Assert(parent_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
 	/*
-	 * Find pg_inherits entries by inhrelid.
+	 * Find pg_inherits entries by inhparent.  (We need to scan them all in
+	 * order to verify that no other partition is pending detach.)
 	 */
 	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
 	ScanKeyInit(&key,
-				Anum_pg_inherits_inhrelid,
+				Anum_pg_inherits_inhparent,
 				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(RelationGetRelid(child_rel)));
-	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+				ObjectIdGetDatum(RelationGetRelid(parent_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsParentIndexId,
 							  true, NULL, 1, &key);
 
 	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
 	{
-		HeapTuple	newtup;
+		Form_pg_inherits inhForm;
 
-		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
-			RelationGetRelid(parent_rel))
-			elog(ERROR, "bad parent tuple found for partition %u",
-				 RelationGetRelid(child_rel));
+		inhForm = (Form_pg_inherits) GETSTRUCT(inheritsTuple);
+		if (inhForm->inhdetachpending)
+			ereport(ERROR,
+					errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("partition \"%s\" already pending detach in partitioned table \"%s.%s\"",
+						   get_rel_name(inhForm->inhrelid),
+						   get_namespace_name(parent_rel->rd_rel->relnamespace),
+						   RelationGetRelationName(parent_rel)),
+					errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the detach operation."));
 
-		newtup = heap_copytuple(inheritsTuple);
-		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+		if (inhForm->inhrelid == RelationGetRelid(child_rel))
+		{
+			HeapTuple	newtup;
 
-		CatalogTupleUpdate(catalogRelation,
-						   &inheritsTuple->t_self,
-						   newtup);
-		found = true;
+			newtup = heap_copytuple(inheritsTuple);
+			((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+
+			CatalogTupleUpdate(catalogRelation,
+							   &inheritsTuple->t_self,
+							   newtup);
+			found = true;
+			heap_freetuple(newtup);
+			/* keep looking, to ensure we catch others pending detach */
+		}
 	}
 
 	/* Done */
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index d8393aa4de..f305f8bc0f 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, true,
-											 ShareRowExclusiveLock, NULL);
+			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 2305dff407..e113607367 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -54,6 +54,12 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
 /*
  * RelationGetPartitionDesc -- get partition descriptor, if relation is partitioned
  *
+ * We keep two partdescs in relcache: rd_partdesc_nodetached excludes
+ * partitions marked concurrently being detached, while rd_partdesc includes
+ * them.  We store the pg_inherits.xmin value for the former, to determine
+ * whether it can be validly reused in each case, since that depends on the
+ * active snapshot.
+ *
  * Note: we arrange for partition descriptors to not get freed until the
  * relcache entry's refcount goes to zero (see hacks in RelationClose,
  * RelationClearRelation, and RelationBuildPartitionDesc).  Therefore, even
@@ -61,13 +67,6 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
  * for callers to continue to use that pointer as long as (a) they hold the
  * relation open, and (b) they hold a relation lock strong enough to ensure
  * that the data doesn't become stale.
- *
- * The above applies to partition descriptors that are complete regarding
- * partitions concurrently being detached.  When a descriptor that omits
- * partitions being detached is requested (and such partitions are present),
- * said descriptor is not part of relcache and so it isn't freed by
- * invalidations either.  Caller must not use such a descriptor beyond the
- * current Portal.
  */
 PartitionDesc
 RelationGetPartitionDesc(Relation rel, bool omit_detached)
@@ -78,11 +77,36 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
 	 * If relcache has a partition descriptor, use that.  However, we can only
 	 * 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.
 	 */
 	if (likely(rel->rd_partdesc &&
-			   (!rel->rd_partdesc->detached_exist || !omit_detached)))
+			   (!rel->rd_partdesc->detached_exist || !omit_detached ||
+				!ActiveSnapshotSet())))
 		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 use it.  Otherwise build one
+	 * from scratch.
+	 */
+	if (omit_detached &&
+		rel->rd_partdesc_nodetached &&
+		TransactionIdIsValid(rel->rd_partdesc_nodetached_xmin) &&
+		ActiveSnapshotSet())
+	{
+		Snapshot	activesnap;
+
+		activesnap = GetActiveSnapshot();
+
+		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin, activesnap))
+			return rel->rd_partdesc_nodetached;
+	}
+
 	return RelationBuildPartitionDesc(rel, omit_detached);
 }
 
@@ -117,6 +141,7 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
 	bool		detached_exist;
+	TransactionId detached_xmin;
 	ListCell   *cell;
 	int			i,
 				nparts;
@@ -132,8 +157,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	 * some well-defined point in time.
 	 */
 	detached_exist = false;
-	inhoids = find_inheritance_children(RelationGetRelid(rel), omit_detached,
-										NoLock, &detached_exist);
+	detached_xmin = InvalidTransactionId;
+	inhoids = find_inheritance_children_extended(RelationGetRelid(rel),
+												 omit_detached, NoLock,
+												 &detached_exist,
+												 &detached_xmin);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -282,36 +310,34 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 
 	/*
 	 * We have a fully valid partdesc.  Reparent it so that it has the right
-	 * lifespan, and if appropriate put it into the relation's relcache entry.
+	 * lifespan.
 	 */
-	if (omit_detached && detached_exist)
+	MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
+
+	/*
+	 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
+	 * partition descriptor that may still be referenced somewhere.
+	 * Preserve it, while not leaking it, by reattaching it as a child
+	 * context of the new rd_pdcxt.  Eventually it will get dropped by
+	 * either RelationClose or RelationClearRelation.
+	 */
+	if (rel->rd_pdcxt != NULL)
+		MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
+	rel->rd_pdcxt = new_pdcxt;
+
+	/*
+	 * Store it into relcache.  For snapshots built excluding detached
+	 * partitions, which we save separately, we also record the
+	 * pg_inherits.xmin of the detached partition that was omitted; this
+	 * informs a future potential user of such a cached snapshot.
+	 */
+	if (omit_detached && detached_exist && ActiveSnapshotSet())
 	{
-		/*
-		 * A transient partition descriptor is only good for the current
-		 * statement, so make it a child of the current portal's context.
-		 */
-		MemoryContextSetParent(new_pdcxt, PortalContext);
+		rel->rd_partdesc_nodetached = partdesc;
+		rel->rd_partdesc_nodetached_xmin = detached_xmin;
 	}
 	else
 	{
-		/*
-		 * This partdesc goes into relcache.
-		 */
-
-		MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
-
-		/*
-		 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
-		 * partition descriptor that may still be referenced somewhere.
-		 * Preserve it, while not leaking it, by reattaching it as a child
-		 * context of the new rd_pdcxt.  Eventually it will get dropped by
-		 * either RelationClose or RelationClearRelation.
-		 */
-		if (rel->rd_pdcxt != NULL)
-			MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
-		rel->rd_pdcxt = new_pdcxt;
-
-		/* Store it into relcache */
 		rel->rd_partdesc = partdesc;
 	}
 
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 4d28ede5a6..f47925588a 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,10 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
-									   LOCKMODE lockmode, bool *detached_exist);
+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);
+
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 34e25eb597..0deb28bca2 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -127,6 +127,9 @@ typedef struct RelationData
 
 	/* data managed by RelationGetPartitionDesc: */
 	PartitionDesc rd_partdesc;	/* partition descriptor, or NULL */
+	PartitionDesc rd_partdesc_nodetached;	/* partdesc w/o detached parts */
+	TransactionId rd_partdesc_nodetached_xmin;	/* xmin for the above */
+
 	MemoryContext rd_pdcxt;		/* private context for rd_partdesc, if any */
 
 	/* data managed by RelationGetPartitionQual: */
diff --git a/src/test/isolation/expected/detach-partition-concurrently-3.out b/src/test/isolation/expected/detach-partition-concurrently-3.out
index 88e83638c7..dbb3825792 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-3.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-3.out
@@ -20,6 +20,7 @@ step s1describe: SELECT 'd3_listp' AS root, * FROM pg_partition_tree('d3_listp')
 root           relid          parentrelid    isleaf         level          
 
 d3_listp       d3_listp                      f              0              
+d3_listp       d3_listp2      d3_listp       t              1              
 d3_listp1      d3_listp1                     t              0              
 step s1alter: ALTER TABLE d3_listp1 ALTER a DROP NOT NULL;
 ERROR:  cannot alter partition "d3_listp1" with an incomplete detach
@@ -123,6 +124,61 @@ a
 
 1              
 
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detach2 s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+ERROR:  partition "d3_listp1" already pending detach in partitioned table "public.d3_listp"
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detachfinal s1c s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1droppart s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1droppart: DROP TABLE d3_listp1;
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
 starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2drop s1s s2commit
 step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
 step s1b: BEGIN;
@@ -279,4 +335,3 @@ step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
 step s1insertpart: INSERT INTO d3_listp1 VALUES (1); <waiting ...>
 step s2commit: COMMIT;
 step s1insertpart: <... completed>
-unused step name: s1droppart
diff --git a/src/test/isolation/specs/detach-partition-concurrently-3.spec b/src/test/isolation/specs/detach-partition-concurrently-3.spec
index 4b706430e1..4c563890eb 100644
--- a/src/test/isolation/specs/detach-partition-concurrently-3.spec
+++ b/src/test/isolation/specs/detach-partition-concurrently-3.spec
@@ -4,12 +4,13 @@ setup
 {
   CREATE TABLE d3_listp (a int) PARTITION BY LIST(a);
   CREATE TABLE d3_listp1 PARTITION OF d3_listp FOR VALUES IN (1);
+  CREATE TABLE d3_listp2 PARTITION OF d3_listp FOR VALUES IN (2);
   CREATE TABLE d3_pid (pid int);
   INSERT INTO d3_listp VALUES (1);
 }
 
 teardown {
-    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_pid;
+    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_listp2, d3_pid;
 }
 
 session "s1"
@@ -34,6 +35,7 @@ session "s2"
 step "s2begin"		{ BEGIN; }
 step "s2snitch"		{ INSERT INTO d3_pid SELECT pg_backend_pid(); }
 step "s2detach"		{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; }
+step "s2detach2"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; }
 step "s2detachfinal"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; }
 step "s2drop"		{ DROP TABLE d3_listp1; }
 step "s2commit"		{ COMMIT; }
@@ -49,6 +51,11 @@ permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1drop" "s1list"
 # "truncate" only does parent, not partition
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1trunc" "s1spart"
 
+# If a partition pending detach exists, we cannot drop another one
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detach2" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detachfinal" "s1c" "s2detach2"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1droppart" "s2detach2"
+
 # When a partition with incomplete detach is dropped, we grab lock on parent too.
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2drop" "s1s" "s2commit"
 
-- 
2.20.1

#72Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#71)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Thanks for the updated patch. I've been reading it, but I noticed a
bug in 8aba9322511f, which I thought you'd want to know to make a note
of when committing this one.

So we decided in 8aba9322511f that it is okay to make the memory
context in which a transient partdesc is allocated a child of
PortalContext so that it disappears when the portal does. But
PortalContext is apparently NULL when the planner runs, at least in
the "simple" query protocol, so any transient partdescs built by the
planner would effectively leak. Not good.

With this patch, partdesc_nodetached is no longer transient, so the
problem doesn't exist.

I will write more about the updated patch in a bit...

--
Amit Langote
EDB: http://www.enterprisedb.com

#73Amit Langote
amitlangote09@gmail.com
In reply to: Amit Langote (#72)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Tue, Apr 27, 2021 at 4:34 PM Amit Langote <amitlangote09@gmail.com> wrote:

Thanks for the updated patch. I've been reading it, but I noticed a
bug in 8aba9322511f, which I thought you'd want to know to make a note
of when committing this one.

So we decided in 8aba9322511f that it is okay to make the memory
context in which a transient partdesc is allocated a child of
PortalContext so that it disappears when the portal does. But
PortalContext is apparently NULL when the planner runs, at least in
the "simple" query protocol, so any transient partdescs built by the
planner would effectively leak. Not good.

With this patch, partdesc_nodetached is no longer transient, so the
problem doesn't exist.

I will write more about the updated patch in a bit...

The first thing that struck me about partdesc_nodetached is that it's
not handled in RelationClearRelation(), where we (re)set a regular
partdesc to NULL so that the next RelationGetPartitionDesc() has to
build it from scratch. I think partdesc_nodetached and the xmin
should likewise be reset. Also in load_relcache_init_file(), although
nothing serious there.

That said, I think I may have found a couple of practical problems
with partdesc_nodetached, or more precisely with having it
side-by-side with regular partdesc. Maybe they can be fixed, so the
problems are not as such deal breakers for the patch's main idea. The
problems can be seen when different queries in a serializable
transaction have to use both the regular partdesc and
partdesc_detached in a given relcache. For example, try the following
after first creating a situation where the table p has a
detach-pending partition p2 (for values in (2) and a live partition p1
(for values in (1)).

begin isolation level serializable;
insert into p values (1);
select * from p where a = 1;
insert into p values (1);

The 1st insert succeeds but the 2nd fails with:

ERROR: no partition of relation "p" found for row
DETAIL: Partition key of the failing row contains (a) = (1).

I haven't analyzed this error very closely but there is another
situation which causes a crash due to what appears to be a conflict
with rd_pdcxt's design:

-- run in a new session
begin isolation level serializable;
select * from p where a = 1;
insert into p values (1);
select * from p where a = 1;

The 2nd select crashes:

server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

The crash occurs because the planner gets handed a stale copy of
partdesc_nodetached for the 2nd select. It gets stale, because the
context it's allocated in gets made a child of rd_pdcxt, which is in
turn assigned the context of the regular partdesc when it is built for
the insert query. Any child contexts of rd_pdcxt are deleted as soon
as the Relation's refcount goes to zero, taking it with
partdesc_nodetached. Note it is this code in
RelationBuildPartitionDesc():

/*
* But first, a kluge: if there's an old rd_pdcxt, it contains an old
* partition descriptor that may still be referenced somewhere.
* Preserve it, while not leaking it, by reattaching it as a child
* context of the new rd_pdcxt. Eventually it will get dropped by
* either RelationClose or RelationClearRelation.
*/
if (rel->rd_pdcxt != NULL)
MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
rel->rd_pdcxt = new_pdcxt;

I think we may need a separate context for partdesc_nodetached, likely
with the same kludges as rd_pdcxt. Maybe the first problem will go
away with that as well.

Few other minor things I noticed:

+    * Store it into relcache.  For snapshots built excluding detached
+    * partitions, which we save separately, we also record the
+    * pg_inherits.xmin of the detached partition that was omitted; this
+    * informs a future potential user of such a cached snapshot.

The "snapshot" in the 1st and the last sentence should be "partdesc"?

+ * We keep two partdescs in relcache: rd_partdesc_nodetached excludes
+ * partitions marked concurrently being detached, while rd_partdesc includes
+ * them.

IMHO, describing rd_partdesc first in the sentence would be better.
Like: rd_partdesc includes all partitions including any that are being
concurrently detached, while rd_partdesc_nodetached excludes them.

--
Amit Langote
EDB: http://www.enterprisedb.com

#74Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#73)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-27, Amit Langote wrote:

On Tue, Apr 27, 2021 at 4:34 PM Amit Langote <amitlangote09@gmail.com> wrote:

I think we may need a separate context for partdesc_nodetached, likely
with the same kludges as rd_pdcxt. Maybe the first problem will go
away with that as well.

Ooh, seems I completely misunderstood what RelationClose was doing. I
thought it was deleting the whole rd_pdcxt, *including* the "current"
partdesc. But that's not at all what it does: it only deletes the
*children* memcontexts, so the partdesc that is currently valid remains
valid. I agree that your proposed fix appears to be promising, in that
a separate "context tree" rd_pddcxt (?) can be used for this. I'll try
it out now.

Few other minor things I noticed:

+    * Store it into relcache.  For snapshots built excluding detached
+    * partitions, which we save separately, we also record the
+    * pg_inherits.xmin of the detached partition that was omitted; this
+    * informs a future potential user of such a cached snapshot.

The "snapshot" in the 1st and the last sentence should be "partdesc"?

Doh, yeah.

+ * We keep two partdescs in relcache: rd_partdesc_nodetached excludes
+ * partitions marked concurrently being detached, while rd_partdesc includes
+ * them.

IMHO, describing rd_partdesc first in the sentence would be better.
Like: rd_partdesc includes all partitions including any that are being
concurrently detached, while rd_partdesc_nodetached excludes them.

Makes sense.

--
�lvaro Herrera Valdivia, Chile

#75Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#73)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

This v3 handles things as you suggested and works correctly AFAICT. I'm
going to add some more tests cases to verify the behavior in the
scenarios you showed, and get them to run under cache-clobber options to
make sure it's good.

Thanks!

--
�lvaro Herrera Valdivia, Chile

Attachments:

v3-0001-Allow-a-partdesc-omitting-partitions-to-be-cached.patchtext/x-diff; charset=us-asciiDownload
From 145ed63c4338d7c7804329e736019a00c97f1a7a Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Mon, 26 Apr 2021 14:53:04 -0400
Subject: [PATCH v3] Allow a partdesc-omitting-partitions to be cached

Makes partition descriptor acquisition faster during the transient
period in which a partition is in the process of being detached.

Per gripe from Amit Langote
Discussion: https://postgr.es/m/CA+HiwqFgpP1LxJZOBYGt9rpvTjXXkg5qG2+Xch2Z1Q7KrqZR1A@mail.gmail.com
---
 src/backend/catalog/pg_inherits.c             |  52 ++++++++-
 src/backend/commands/tablecmds.c              |  66 +++++++-----
 src/backend/commands/trigger.c                |   3 +-
 src/backend/partitioning/partdesc.c           | 101 ++++++++++++------
 src/backend/utils/cache/relcache.c            |  33 +++++-
 src/include/catalog/pg_inherits.h             |   6 +-
 src/include/utils/rel.h                       |   5 +
 .../detach-partition-concurrently-3.out       |  57 +++++++++-
 .../detach-partition-concurrently-3.spec      |   9 +-
 9 files changed, 254 insertions(+), 78 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 6447b52854..c373faf2d6 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,6 +52,22 @@ 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
+ * find_inheritance_children_extended for details.
+ */
+List *
+find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+{
+	return find_inheritance_children_extended(parentrelId, true, lockmode,
+											  NULL, NULL);
+}
+
+/*
+ * find_inheritance_children_extended
+ *
+ * As find_inheritance_children, with more options regarding detached
+ * partitions.
+ *
  * If a partition's pg_inherits row is marked "detach pending",
  * *detached_exist (if not null) is set true.
  *
@@ -60,11 +76,13 @@ typedef struct SeenRelsEntry
  * 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.
+ * committed to the active snapshot.  In addition, *detached_xmin (if not null)
+ * is set to the xmin of the row of the detached partition.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool omit_detached,
-						  LOCKMODE lockmode, bool *detached_exist)
+find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
+								   LOCKMODE lockmode, bool *detached_exist,
+								   TransactionId *detached_xmin)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -132,7 +150,32 @@ find_inheritance_children(Oid parentrelId, bool omit_detached,
 				snap = GetActiveSnapshot();
 
 				if (!XidInMVCCSnapshot(xmin, snap))
+				{
+					if (detached_xmin)
+					{
+						/*
+						 * Two detached partitions should not occur (see
+						 * checks in MarkInheritDetached), but if they do,
+						 * track the newer of the two.  Make sure to warn the
+						 * user, so that they can clean up.  Since this is
+						 * just a cross-check against potentially corrupt
+						 * catalogs, we don't make it a full-fledged error
+						 * message.
+						 */
+						if (*detached_xmin != InvalidTransactionId)
+						{
+							elog(WARNING, "more than one partition pending detach found for table with OID %u",
+								 parentrelId);
+							if (TransactionIdFollows(xmin, *detached_xmin))
+								*detached_xmin = xmin;
+						}
+						else
+							*detached_xmin = xmin;
+					}
+
+					/* Don't add the partition to the output list */
 					continue;
+				}
 			}
 		}
 
@@ -247,8 +290,7 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, true,
-													lockmode, NULL);
+		currentchildren = find_inheritance_children(currentrel, lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 8e717ada28..d9ba87a2a3 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3492,7 +3492,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+			find_inheritance_children(myrelid, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3691,7 +3691,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+				find_inheritance_children(myrelid, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6565,7 +6565,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+		find_inheritance_children(myrelid, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6811,7 +6811,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7674,7 +7674,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
+		find_inheritance_children(RelationGetRelid(rel), lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8282,7 +8282,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	if (children)
 	{
@@ -8770,7 +8770,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11303,8 +11303,7 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), true,
-											 lockmode, NULL);
+		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
 	else
 		children = NIL;
 
@@ -11688,8 +11687,7 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), true,
-									   NoLock, NULL) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14601,7 +14599,8 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * MarkInheritDetached
  *
  * Set inhdetachpending for a partition, for ATExecDetachPartition
- * in concurrent mode.
+ * in concurrent mode.  While at it, verify that no other partition is
+ * already pending detach.
  */
 static void
 MarkInheritDetached(Relation child_rel, Relation parent_rel)
@@ -14617,32 +14616,45 @@ MarkInheritDetached(Relation child_rel, Relation parent_rel)
 	Assert(parent_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
 	/*
-	 * Find pg_inherits entries by inhrelid.
+	 * Find pg_inherits entries by inhparent.  (We need to scan them all in
+	 * order to verify that no other partition is pending detach.)
 	 */
 	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
 	ScanKeyInit(&key,
-				Anum_pg_inherits_inhrelid,
+				Anum_pg_inherits_inhparent,
 				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(RelationGetRelid(child_rel)));
-	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+				ObjectIdGetDatum(RelationGetRelid(parent_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsParentIndexId,
 							  true, NULL, 1, &key);
 
 	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
 	{
-		HeapTuple	newtup;
+		Form_pg_inherits inhForm;
 
-		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
-			RelationGetRelid(parent_rel))
-			elog(ERROR, "bad parent tuple found for partition %u",
-				 RelationGetRelid(child_rel));
+		inhForm = (Form_pg_inherits) GETSTRUCT(inheritsTuple);
+		if (inhForm->inhdetachpending)
+			ereport(ERROR,
+					errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("partition \"%s\" already pending detach in partitioned table \"%s.%s\"",
+						   get_rel_name(inhForm->inhrelid),
+						   get_namespace_name(parent_rel->rd_rel->relnamespace),
+						   RelationGetRelationName(parent_rel)),
+					errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the detach operation."));
+
+		if (inhForm->inhrelid == RelationGetRelid(child_rel))
+		{
+			HeapTuple	newtup;
 
-		newtup = heap_copytuple(inheritsTuple);
-		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+			newtup = heap_copytuple(inheritsTuple);
+			((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
 
-		CatalogTupleUpdate(catalogRelation,
-						   &inheritsTuple->t_self,
-						   newtup);
-		found = true;
+			CatalogTupleUpdate(catalogRelation,
+							   &inheritsTuple->t_self,
+							   newtup);
+			found = true;
+			heap_freetuple(newtup);
+			/* keep looking, to ensure we catch others pending detach */
+		}
 	}
 
 	/* Done */
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index d8393aa4de..f305f8bc0f 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, true,
-											 ShareRowExclusiveLock, NULL);
+			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 2305dff407..290db0fa35 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -54,6 +54,12 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
 /*
  * RelationGetPartitionDesc -- get partition descriptor, if relation is partitioned
  *
+ * 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.
+ *
  * Note: we arrange for partition descriptors to not get freed until the
  * relcache entry's refcount goes to zero (see hacks in RelationClose,
  * RelationClearRelation, and RelationBuildPartitionDesc).  Therefore, even
@@ -61,13 +67,6 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
  * for callers to continue to use that pointer as long as (a) they hold the
  * relation open, and (b) they hold a relation lock strong enough to ensure
  * that the data doesn't become stale.
- *
- * The above applies to partition descriptors that are complete regarding
- * partitions concurrently being detached.  When a descriptor that omits
- * partitions being detached is requested (and such partitions are present),
- * said descriptor is not part of relcache and so it isn't freed by
- * invalidations either.  Caller must not use such a descriptor beyond the
- * current Portal.
  */
 PartitionDesc
 RelationGetPartitionDesc(Relation rel, bool omit_detached)
@@ -78,11 +77,36 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
 	 * If relcache has a partition descriptor, use that.  However, we can only
 	 * 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.
 	 */
 	if (likely(rel->rd_partdesc &&
-			   (!rel->rd_partdesc->detached_exist || !omit_detached)))
+			   (!rel->rd_partdesc->detached_exist || !omit_detached ||
+				!ActiveSnapshotSet())))
 		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 use it.  Otherwise build one
+	 * from scratch.
+	 */
+	if (omit_detached &&
+		rel->rd_partdesc_nodetached &&
+		TransactionIdIsValid(rel->rd_partdesc_nodetached_xmin) &&
+		ActiveSnapshotSet())
+	{
+		Snapshot	activesnap;
+
+		activesnap = GetActiveSnapshot();
+
+		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin, activesnap))
+			return rel->rd_partdesc_nodetached;
+	}
+
 	return RelationBuildPartitionDesc(rel, omit_detached);
 }
 
@@ -117,10 +141,13 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
 	bool		detached_exist;
+	bool		is_omit;
+	TransactionId detached_xmin;
 	ListCell   *cell;
 	int			i,
 				nparts;
 	PartitionKey key = RelationGetPartitionKey(rel);
+	MemoryContext *context;
 	MemoryContext new_pdcxt;
 	MemoryContext oldcxt;
 	int		   *mapping;
@@ -132,8 +159,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	 * some well-defined point in time.
 	 */
 	detached_exist = false;
-	inhoids = find_inheritance_children(RelationGetRelid(rel), omit_detached,
-										NoLock, &detached_exist);
+	detached_xmin = InvalidTransactionId;
+	inhoids = find_inheritance_children_extended(RelationGetRelid(rel),
+												 omit_detached, NoLock,
+												 &detached_exist,
+												 &detached_xmin);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -280,38 +310,39 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 		MemoryContextSwitchTo(oldcxt);
 	}
 
+	is_omit = omit_detached && detached_exist && ActiveSnapshotSet();
+
 	/*
 	 * We have a fully valid partdesc.  Reparent it so that it has the right
-	 * lifespan, and if appropriate put it into the relation's relcache entry.
+	 * lifespan.
+	 */
+	MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
+
+	/*
+	 * But first, a kluge: if there's an old context for this type of
+	 * descriptor, it contains an old partition descriptor that may still be
+	 * referenced somewhere.  Preserve it, while not leaking it, by
+	 * reattaching it as a child context of the new one.  Eventually it will
+	 * get dropped by either RelationClose or RelationClearRelation.
 	 */
-	if (omit_detached && detached_exist)
+	context = is_omit ? &rel->rd_pddcxt : &rel->rd_pdcxt;
+	if (*context != NULL)
+		MemoryContextSetParent(*context, new_pdcxt);
+	*context = new_pdcxt;
+
+	/*
+	 * Store it into relcache.  For partdescs built excluding detached
+	 * partitions, which we save separately, we also record the
+	 * pg_inherits.xmin of the detached partition that was omitted; this
+	 * informs a future potential user of such a cached partdescs.
+	 */
+	if (is_omit)
 	{
-		/*
-		 * A transient partition descriptor is only good for the current
-		 * statement, so make it a child of the current portal's context.
-		 */
-		MemoryContextSetParent(new_pdcxt, PortalContext);
+		rel->rd_partdesc_nodetached = partdesc;
+		rel->rd_partdesc_nodetached_xmin = detached_xmin;
 	}
 	else
 	{
-		/*
-		 * This partdesc goes into relcache.
-		 */
-
-		MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
-
-		/*
-		 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
-		 * partition descriptor that may still be referenced somewhere.
-		 * Preserve it, while not leaking it, by reattaching it as a child
-		 * context of the new rd_pdcxt.  Eventually it will get dropped by
-		 * either RelationClose or RelationClearRelation.
-		 */
-		if (rel->rd_pdcxt != NULL)
-			MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
-		rel->rd_pdcxt = new_pdcxt;
-
-		/* Store it into relcache */
 		rel->rd_partdesc = partdesc;
 	}
 
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 466c28d528..31c8e07f2a 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -1157,7 +1157,10 @@ RelationBuildDesc(Oid targetRelId, bool insertIt)
 	relation->rd_partkey = NULL;
 	relation->rd_partkeycxt = NULL;
 	relation->rd_partdesc = NULL;
+	relation->rd_partdesc_nodetached = NULL;
+	relation->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 	relation->rd_pdcxt = NULL;
+	relation->rd_pddcxt = NULL;
 	relation->rd_partcheck = NIL;
 	relation->rd_partcheckvalid = false;
 	relation->rd_partcheckcxt = NULL;
@@ -2103,10 +2106,16 @@ RelationClose(Relation relation)
 	 * stale partition descriptors it has.  This is unlikely, so check to see
 	 * if there are child contexts before expending a call to mcxt.c.
 	 */
-	if (RelationHasReferenceCountZero(relation) &&
-		relation->rd_pdcxt != NULL &&
-		relation->rd_pdcxt->firstchild != NULL)
-		MemoryContextDeleteChildren(relation->rd_pdcxt);
+	if (RelationHasReferenceCountZero(relation))
+	{
+		if (relation->rd_pdcxt != NULL &&
+			relation->rd_pdcxt->firstchild != NULL)
+			MemoryContextDeleteChildren(relation->rd_pdcxt);
+
+		if (relation->rd_pddcxt != NULL &&
+			relation->rd_pddcxt->firstchild != NULL)
+			MemoryContextDeleteChildren(relation->rd_pddcxt);
+	}
 
 #ifdef RELCACHE_FORCE_RELEASE
 	if (RelationHasReferenceCountZero(relation) &&
@@ -2390,6 +2399,8 @@ RelationDestroyRelation(Relation relation, bool remember_tupdesc)
 		MemoryContextDelete(relation->rd_partkeycxt);
 	if (relation->rd_pdcxt)
 		MemoryContextDelete(relation->rd_pdcxt);
+	if (relation->rd_pddcxt)
+		MemoryContextDelete(relation->rd_pddcxt);
 	if (relation->rd_partcheckcxt)
 		MemoryContextDelete(relation->rd_partcheckcxt);
 	pfree(relation);
@@ -2644,7 +2655,7 @@ RelationClearRelation(Relation relation, bool rebuild)
 			SWAPFIELD(PartitionKey, rd_partkey);
 			SWAPFIELD(MemoryContext, rd_partkeycxt);
 		}
-		if (newrel->rd_pdcxt != NULL)
+		if (newrel->rd_pdcxt != NULL || newrel->rd_pddcxt != NULL)
 		{
 			/*
 			 * We are rebuilding a partitioned relation with a non-zero
@@ -2672,13 +2683,22 @@ RelationClearRelation(Relation relation, bool rebuild)
 			 * newrel.
 			 */
 			relation->rd_partdesc = NULL;	/* ensure rd_partdesc is invalid */
+			relation->rd_partdesc_nodetached = NULL;
+			relation->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 			if (relation->rd_pdcxt != NULL) /* probably never happens */
 				MemoryContextSetParent(newrel->rd_pdcxt, relation->rd_pdcxt);
 			else
 				relation->rd_pdcxt = newrel->rd_pdcxt;
+			if (relation->rd_pddcxt != NULL)
+				MemoryContextSetParent(newrel->rd_pddcxt, relation->rd_pddcxt);
+			else
+				relation->rd_pddcxt = newrel->rd_pddcxt;
 			/* drop newrel's pointers so we don't destroy it below */
 			newrel->rd_partdesc = NULL;
+			newrel->rd_partdesc_nodetached = NULL;
+			newrel->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 			newrel->rd_pdcxt = NULL;
+			newrel->rd_pddcxt = NULL;
 		}
 
 #undef SWAPFIELD
@@ -6017,7 +6037,10 @@ load_relcache_init_file(bool shared)
 		rel->rd_partkey = NULL;
 		rel->rd_partkeycxt = NULL;
 		rel->rd_partdesc = NULL;
+		rel->rd_partdesc_nodetached = NULL;
+		rel->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 		rel->rd_pdcxt = NULL;
+		rel->rd_pddcxt = NULL;
 		rel->rd_partcheck = NIL;
 		rel->rd_partcheckvalid = false;
 		rel->rd_partcheckcxt = NULL;
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 4d28ede5a6..f47925588a 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,10 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
-									   LOCKMODE lockmode, bool *detached_exist);
+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);
+
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 34e25eb597..cdc2560f94 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -129,6 +129,11 @@ typedef struct RelationData
 	PartitionDesc rd_partdesc;	/* partition descriptor, or NULL */
 	MemoryContext rd_pdcxt;		/* private context for rd_partdesc, if any */
 
+	/* Same as above, for partdescs that omit detached partitions */
+	PartitionDesc rd_partdesc_nodetached;	/* partdesc w/o detached parts */
+	TransactionId rd_partdesc_nodetached_xmin;	/* xmin for the above */
+	MemoryContext rd_pddcxt;	/* for rd_partdesc_nodetached, if any */
+
 	/* data managed by RelationGetPartitionQual: */
 	List	   *rd_partcheck;	/* partition CHECK quals */
 	bool		rd_partcheckvalid;	/* true if list has been computed */
diff --git a/src/test/isolation/expected/detach-partition-concurrently-3.out b/src/test/isolation/expected/detach-partition-concurrently-3.out
index 88e83638c7..dbb3825792 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-3.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-3.out
@@ -20,6 +20,7 @@ step s1describe: SELECT 'd3_listp' AS root, * FROM pg_partition_tree('d3_listp')
 root           relid          parentrelid    isleaf         level          
 
 d3_listp       d3_listp                      f              0              
+d3_listp       d3_listp2      d3_listp       t              1              
 d3_listp1      d3_listp1                     t              0              
 step s1alter: ALTER TABLE d3_listp1 ALTER a DROP NOT NULL;
 ERROR:  cannot alter partition "d3_listp1" with an incomplete detach
@@ -123,6 +124,61 @@ a
 
 1              
 
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detach2 s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+ERROR:  partition "d3_listp1" already pending detach in partitioned table "public.d3_listp"
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detachfinal s1c s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1droppart s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1droppart: DROP TABLE d3_listp1;
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
 starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2drop s1s s2commit
 step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
 step s1b: BEGIN;
@@ -279,4 +335,3 @@ step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
 step s1insertpart: INSERT INTO d3_listp1 VALUES (1); <waiting ...>
 step s2commit: COMMIT;
 step s1insertpart: <... completed>
-unused step name: s1droppart
diff --git a/src/test/isolation/specs/detach-partition-concurrently-3.spec b/src/test/isolation/specs/detach-partition-concurrently-3.spec
index 4b706430e1..4c563890eb 100644
--- a/src/test/isolation/specs/detach-partition-concurrently-3.spec
+++ b/src/test/isolation/specs/detach-partition-concurrently-3.spec
@@ -4,12 +4,13 @@ setup
 {
   CREATE TABLE d3_listp (a int) PARTITION BY LIST(a);
   CREATE TABLE d3_listp1 PARTITION OF d3_listp FOR VALUES IN (1);
+  CREATE TABLE d3_listp2 PARTITION OF d3_listp FOR VALUES IN (2);
   CREATE TABLE d3_pid (pid int);
   INSERT INTO d3_listp VALUES (1);
 }
 
 teardown {
-    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_pid;
+    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_listp2, d3_pid;
 }
 
 session "s1"
@@ -34,6 +35,7 @@ session "s2"
 step "s2begin"		{ BEGIN; }
 step "s2snitch"		{ INSERT INTO d3_pid SELECT pg_backend_pid(); }
 step "s2detach"		{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; }
+step "s2detach2"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; }
 step "s2detachfinal"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; }
 step "s2drop"		{ DROP TABLE d3_listp1; }
 step "s2commit"		{ COMMIT; }
@@ -49,6 +51,11 @@ permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1drop" "s1list"
 # "truncate" only does parent, not partition
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1trunc" "s1spart"
 
+# If a partition pending detach exists, we cannot drop another one
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detach2" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detachfinal" "s1c" "s2detach2"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1droppart" "s2detach2"
+
 # When a partition with incomplete detach is dropped, we grab lock on parent too.
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2drop" "s1s" "s2commit"
 
-- 
2.20.1

#76Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#75)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-27, Alvaro Herrera wrote:

This v3 handles things as you suggested and works correctly AFAICT. I'm
going to add some more tests cases to verify the behavior in the
scenarios you showed, and get them to run under cache-clobber options to
make sure it's good.

Yep, it seems to work. Strangely, the new isolation case doesn't
actually crash before the fix -- it merely throws a memory allocation
error.

--
�lvaro Herrera Valdivia, Chile
"Linux transform� mi computadora, de una `m�quina para hacer cosas',
en un aparato realmente entretenido, sobre el cual cada d�a aprendo
algo nuevo" (Jaime Salinas)

Attachments:

v4-0001-Allow-a-partdesc-omitting-partitions-to-be-cached.patchtext/x-diff; charset=us-asciiDownload
From 8e41eb137b835f614ddeeb7c79e70d7e0740f522 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Tue, 27 Apr 2021 19:04:37 -0400
Subject: [PATCH v4] Allow a partdesc-omitting-partitions to be cached

Makes partition descriptor acquisition faster during the transient
period in which a partition is in the process of being detached.

(This incidentally fixes an alleged bug in 8aba9322511 whereby a
memory context holding a transient partdesc is reparented to NULL ...
leading to permanent leak of that memory.  Reported by Amit Langote.)

Per gripe from Amit Langote
Discussion: https://postgr.es/m/CA+HiwqFgpP1LxJZOBYGt9rpvTjXXkg5qG2+Xch2Z1Q7KrqZR1A@mail.gmail.com
---
 src/backend/catalog/pg_inherits.c             |  52 ++++++++-
 src/backend/commands/tablecmds.c              |  66 ++++++-----
 src/backend/commands/trigger.c                |   3 +-
 src/backend/partitioning/partdesc.c           | 110 ++++++++++++------
 src/backend/utils/cache/relcache.c            |  33 +++++-
 src/include/catalog/pg_inherits.h             |   6 +-
 src/include/utils/rel.h                       |   5 +
 .../detach-partition-concurrently-3.out       | 110 +++++++++++++++++-
 .../detach-partition-concurrently-3.spec      |  14 ++-
 9 files changed, 320 insertions(+), 79 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 6447b52854..c373faf2d6 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,6 +52,22 @@ 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
+ * find_inheritance_children_extended for details.
+ */
+List *
+find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+{
+	return find_inheritance_children_extended(parentrelId, true, lockmode,
+											  NULL, NULL);
+}
+
+/*
+ * find_inheritance_children_extended
+ *
+ * As find_inheritance_children, with more options regarding detached
+ * partitions.
+ *
  * If a partition's pg_inherits row is marked "detach pending",
  * *detached_exist (if not null) is set true.
  *
@@ -60,11 +76,13 @@ typedef struct SeenRelsEntry
  * 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.
+ * committed to the active snapshot.  In addition, *detached_xmin (if not null)
+ * is set to the xmin of the row of the detached partition.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool omit_detached,
-						  LOCKMODE lockmode, bool *detached_exist)
+find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
+								   LOCKMODE lockmode, bool *detached_exist,
+								   TransactionId *detached_xmin)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -132,7 +150,32 @@ find_inheritance_children(Oid parentrelId, bool omit_detached,
 				snap = GetActiveSnapshot();
 
 				if (!XidInMVCCSnapshot(xmin, snap))
+				{
+					if (detached_xmin)
+					{
+						/*
+						 * Two detached partitions should not occur (see
+						 * checks in MarkInheritDetached), but if they do,
+						 * track the newer of the two.  Make sure to warn the
+						 * user, so that they can clean up.  Since this is
+						 * just a cross-check against potentially corrupt
+						 * catalogs, we don't make it a full-fledged error
+						 * message.
+						 */
+						if (*detached_xmin != InvalidTransactionId)
+						{
+							elog(WARNING, "more than one partition pending detach found for table with OID %u",
+								 parentrelId);
+							if (TransactionIdFollows(xmin, *detached_xmin))
+								*detached_xmin = xmin;
+						}
+						else
+							*detached_xmin = xmin;
+					}
+
+					/* Don't add the partition to the output list */
 					continue;
+				}
 			}
 		}
 
@@ -247,8 +290,7 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, true,
-													lockmode, NULL);
+		currentchildren = find_inheritance_children(currentrel, lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 8e717ada28..d9ba87a2a3 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3492,7 +3492,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+			find_inheritance_children(myrelid, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3691,7 +3691,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+				find_inheritance_children(myrelid, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6565,7 +6565,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+		find_inheritance_children(myrelid, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6811,7 +6811,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7674,7 +7674,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
+		find_inheritance_children(RelationGetRelid(rel), lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8282,7 +8282,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	if (children)
 	{
@@ -8770,7 +8770,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11303,8 +11303,7 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), true,
-											 lockmode, NULL);
+		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
 	else
 		children = NIL;
 
@@ -11688,8 +11687,7 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), true,
-									   NoLock, NULL) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14601,7 +14599,8 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * MarkInheritDetached
  *
  * Set inhdetachpending for a partition, for ATExecDetachPartition
- * in concurrent mode.
+ * in concurrent mode.  While at it, verify that no other partition is
+ * already pending detach.
  */
 static void
 MarkInheritDetached(Relation child_rel, Relation parent_rel)
@@ -14617,32 +14616,45 @@ MarkInheritDetached(Relation child_rel, Relation parent_rel)
 	Assert(parent_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
 	/*
-	 * Find pg_inherits entries by inhrelid.
+	 * Find pg_inherits entries by inhparent.  (We need to scan them all in
+	 * order to verify that no other partition is pending detach.)
 	 */
 	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
 	ScanKeyInit(&key,
-				Anum_pg_inherits_inhrelid,
+				Anum_pg_inherits_inhparent,
 				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(RelationGetRelid(child_rel)));
-	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+				ObjectIdGetDatum(RelationGetRelid(parent_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsParentIndexId,
 							  true, NULL, 1, &key);
 
 	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
 	{
-		HeapTuple	newtup;
+		Form_pg_inherits inhForm;
 
-		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
-			RelationGetRelid(parent_rel))
-			elog(ERROR, "bad parent tuple found for partition %u",
-				 RelationGetRelid(child_rel));
+		inhForm = (Form_pg_inherits) GETSTRUCT(inheritsTuple);
+		if (inhForm->inhdetachpending)
+			ereport(ERROR,
+					errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("partition \"%s\" already pending detach in partitioned table \"%s.%s\"",
+						   get_rel_name(inhForm->inhrelid),
+						   get_namespace_name(parent_rel->rd_rel->relnamespace),
+						   RelationGetRelationName(parent_rel)),
+					errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the detach operation."));
 
-		newtup = heap_copytuple(inheritsTuple);
-		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+		if (inhForm->inhrelid == RelationGetRelid(child_rel))
+		{
+			HeapTuple	newtup;
 
-		CatalogTupleUpdate(catalogRelation,
-						   &inheritsTuple->t_self,
-						   newtup);
-		found = true;
+			newtup = heap_copytuple(inheritsTuple);
+			((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+
+			CatalogTupleUpdate(catalogRelation,
+							   &inheritsTuple->t_self,
+							   newtup);
+			found = true;
+			heap_freetuple(newtup);
+			/* keep looking, to ensure we catch others pending detach */
+		}
 	}
 
 	/* Done */
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index d8393aa4de..f305f8bc0f 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, true,
-											 ShareRowExclusiveLock, NULL);
+			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 2305dff407..1c780ffdb0 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -54,6 +54,12 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
 /*
  * RelationGetPartitionDesc -- get partition descriptor, if relation is partitioned
  *
+ * 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.
+ *
  * Note: we arrange for partition descriptors to not get freed until the
  * relcache entry's refcount goes to zero (see hacks in RelationClose,
  * RelationClearRelation, and RelationBuildPartitionDesc).  Therefore, even
@@ -61,13 +67,6 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
  * for callers to continue to use that pointer as long as (a) they hold the
  * relation open, and (b) they hold a relation lock strong enough to ensure
  * that the data doesn't become stale.
- *
- * The above applies to partition descriptors that are complete regarding
- * partitions concurrently being detached.  When a descriptor that omits
- * partitions being detached is requested (and such partitions are present),
- * said descriptor is not part of relcache and so it isn't freed by
- * invalidations either.  Caller must not use such a descriptor beyond the
- * current Portal.
  */
 PartitionDesc
 RelationGetPartitionDesc(Relation rel, bool omit_detached)
@@ -78,11 +77,36 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
 	 * If relcache has a partition descriptor, use that.  However, we can only
 	 * 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.
 	 */
 	if (likely(rel->rd_partdesc &&
-			   (!rel->rd_partdesc->detached_exist || !omit_detached)))
+			   (!rel->rd_partdesc->detached_exist || !omit_detached ||
+				!ActiveSnapshotSet())))
 		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 use it.  Otherwise build one
+	 * from scratch.
+	 */
+	if (omit_detached &&
+		rel->rd_partdesc_nodetached &&
+		TransactionIdIsValid(rel->rd_partdesc_nodetached_xmin) &&
+		ActiveSnapshotSet())
+	{
+		Snapshot	activesnap;
+
+		activesnap = GetActiveSnapshot();
+
+		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin, activesnap))
+			return rel->rd_partdesc_nodetached;
+	}
+
 	return RelationBuildPartitionDesc(rel, omit_detached);
 }
 
@@ -117,10 +141,13 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
 	bool		detached_exist;
+	bool		is_omit;
+	TransactionId detached_xmin;
 	ListCell   *cell;
 	int			i,
 				nparts;
 	PartitionKey key = RelationGetPartitionKey(rel);
+	MemoryContext *context;
 	MemoryContext new_pdcxt;
 	MemoryContext oldcxt;
 	int		   *mapping;
@@ -132,8 +159,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	 * some well-defined point in time.
 	 */
 	detached_exist = false;
-	inhoids = find_inheritance_children(RelationGetRelid(rel), omit_detached,
-										NoLock, &detached_exist);
+	detached_xmin = InvalidTransactionId;
+	inhoids = find_inheritance_children_extended(RelationGetRelid(rel),
+												 omit_detached, NoLock,
+												 &detached_exist,
+												 &detached_xmin);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -281,37 +311,45 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	}
 
 	/*
-	 * We have a fully valid partdesc.  Reparent it so that it has the right
-	 * lifespan, and if appropriate put it into the relation's relcache entry.
+	 * Are we working with the partition that omits detached partitions, or
+	 * the one that includes them?
 	 */
-	if (omit_detached && detached_exist)
+	is_omit = omit_detached && detached_exist && ActiveSnapshotSet();
+
+	/*
+	 * We have a fully valid partdesc.  Reparent it so that it has the right
+	 * lifespan.
+	 */
+	MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
+
+	/*
+	 * But first, a kluge: if there's an old context for this type of
+	 * descriptor, it contains an old partition descriptor that may still be
+	 * referenced somewhere.  Preserve it, while not leaking it, by
+	 * reattaching it as a child context of the new one.  Eventually it will
+	 * get dropped by either RelationClose or RelationClearRelation.
+	 *
+	 * We keep the regular partdesc in rd_pdcxt, and the partdesc-excluding-
+	 * detached-partitions in rd_pddcxt.
+	 */
+	context = is_omit ? &rel->rd_pddcxt : &rel->rd_pdcxt;
+	if (*context != NULL)
+		MemoryContextSetParent(*context, new_pdcxt);
+	*context = new_pdcxt;
+
+	/*
+	 * Store it into relcache.  For partdescs built excluding detached
+	 * partitions, which we save separately, we also record the
+	 * pg_inherits.xmin of the detached partition that was omitted; this
+	 * informs a future potential user of such a cached partdescs.
+	 */
+	if (is_omit)
 	{
-		/*
-		 * A transient partition descriptor is only good for the current
-		 * statement, so make it a child of the current portal's context.
-		 */
-		MemoryContextSetParent(new_pdcxt, PortalContext);
+		rel->rd_partdesc_nodetached = partdesc;
+		rel->rd_partdesc_nodetached_xmin = detached_xmin;
 	}
 	else
 	{
-		/*
-		 * This partdesc goes into relcache.
-		 */
-
-		MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
-
-		/*
-		 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
-		 * partition descriptor that may still be referenced somewhere.
-		 * Preserve it, while not leaking it, by reattaching it as a child
-		 * context of the new rd_pdcxt.  Eventually it will get dropped by
-		 * either RelationClose or RelationClearRelation.
-		 */
-		if (rel->rd_pdcxt != NULL)
-			MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
-		rel->rd_pdcxt = new_pdcxt;
-
-		/* Store it into relcache */
 		rel->rd_partdesc = partdesc;
 	}
 
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 466c28d528..31c8e07f2a 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -1157,7 +1157,10 @@ RelationBuildDesc(Oid targetRelId, bool insertIt)
 	relation->rd_partkey = NULL;
 	relation->rd_partkeycxt = NULL;
 	relation->rd_partdesc = NULL;
+	relation->rd_partdesc_nodetached = NULL;
+	relation->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 	relation->rd_pdcxt = NULL;
+	relation->rd_pddcxt = NULL;
 	relation->rd_partcheck = NIL;
 	relation->rd_partcheckvalid = false;
 	relation->rd_partcheckcxt = NULL;
@@ -2103,10 +2106,16 @@ RelationClose(Relation relation)
 	 * stale partition descriptors it has.  This is unlikely, so check to see
 	 * if there are child contexts before expending a call to mcxt.c.
 	 */
-	if (RelationHasReferenceCountZero(relation) &&
-		relation->rd_pdcxt != NULL &&
-		relation->rd_pdcxt->firstchild != NULL)
-		MemoryContextDeleteChildren(relation->rd_pdcxt);
+	if (RelationHasReferenceCountZero(relation))
+	{
+		if (relation->rd_pdcxt != NULL &&
+			relation->rd_pdcxt->firstchild != NULL)
+			MemoryContextDeleteChildren(relation->rd_pdcxt);
+
+		if (relation->rd_pddcxt != NULL &&
+			relation->rd_pddcxt->firstchild != NULL)
+			MemoryContextDeleteChildren(relation->rd_pddcxt);
+	}
 
 #ifdef RELCACHE_FORCE_RELEASE
 	if (RelationHasReferenceCountZero(relation) &&
@@ -2390,6 +2399,8 @@ RelationDestroyRelation(Relation relation, bool remember_tupdesc)
 		MemoryContextDelete(relation->rd_partkeycxt);
 	if (relation->rd_pdcxt)
 		MemoryContextDelete(relation->rd_pdcxt);
+	if (relation->rd_pddcxt)
+		MemoryContextDelete(relation->rd_pddcxt);
 	if (relation->rd_partcheckcxt)
 		MemoryContextDelete(relation->rd_partcheckcxt);
 	pfree(relation);
@@ -2644,7 +2655,7 @@ RelationClearRelation(Relation relation, bool rebuild)
 			SWAPFIELD(PartitionKey, rd_partkey);
 			SWAPFIELD(MemoryContext, rd_partkeycxt);
 		}
-		if (newrel->rd_pdcxt != NULL)
+		if (newrel->rd_pdcxt != NULL || newrel->rd_pddcxt != NULL)
 		{
 			/*
 			 * We are rebuilding a partitioned relation with a non-zero
@@ -2672,13 +2683,22 @@ RelationClearRelation(Relation relation, bool rebuild)
 			 * newrel.
 			 */
 			relation->rd_partdesc = NULL;	/* ensure rd_partdesc is invalid */
+			relation->rd_partdesc_nodetached = NULL;
+			relation->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 			if (relation->rd_pdcxt != NULL) /* probably never happens */
 				MemoryContextSetParent(newrel->rd_pdcxt, relation->rd_pdcxt);
 			else
 				relation->rd_pdcxt = newrel->rd_pdcxt;
+			if (relation->rd_pddcxt != NULL)
+				MemoryContextSetParent(newrel->rd_pddcxt, relation->rd_pddcxt);
+			else
+				relation->rd_pddcxt = newrel->rd_pddcxt;
 			/* drop newrel's pointers so we don't destroy it below */
 			newrel->rd_partdesc = NULL;
+			newrel->rd_partdesc_nodetached = NULL;
+			newrel->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 			newrel->rd_pdcxt = NULL;
+			newrel->rd_pddcxt = NULL;
 		}
 
 #undef SWAPFIELD
@@ -6017,7 +6037,10 @@ load_relcache_init_file(bool shared)
 		rel->rd_partkey = NULL;
 		rel->rd_partkeycxt = NULL;
 		rel->rd_partdesc = NULL;
+		rel->rd_partdesc_nodetached = NULL;
+		rel->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 		rel->rd_pdcxt = NULL;
+		rel->rd_pddcxt = NULL;
 		rel->rd_partcheck = NIL;
 		rel->rd_partcheckvalid = false;
 		rel->rd_partcheckcxt = NULL;
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 4d28ede5a6..f47c0a8cd8 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,10 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
-									   LOCKMODE lockmode, bool *detached_exist);
+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);
+
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 34e25eb597..cdc2560f94 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -129,6 +129,11 @@ typedef struct RelationData
 	PartitionDesc rd_partdesc;	/* partition descriptor, or NULL */
 	MemoryContext rd_pdcxt;		/* private context for rd_partdesc, if any */
 
+	/* Same as above, for partdescs that omit detached partitions */
+	PartitionDesc rd_partdesc_nodetached;	/* partdesc w/o detached parts */
+	TransactionId rd_partdesc_nodetached_xmin;	/* xmin for the above */
+	MemoryContext rd_pddcxt;	/* for rd_partdesc_nodetached, if any */
+
 	/* data managed by RelationGetPartitionQual: */
 	List	   *rd_partcheck;	/* partition CHECK quals */
 	bool		rd_partcheckvalid;	/* true if list has been computed */
diff --git a/src/test/isolation/expected/detach-partition-concurrently-3.out b/src/test/isolation/expected/detach-partition-concurrently-3.out
index 88e83638c7..bbb9d151fb 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-3.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-3.out
@@ -20,6 +20,7 @@ step s1describe: SELECT 'd3_listp' AS root, * FROM pg_partition_tree('d3_listp')
 root           relid          parentrelid    isleaf         level          
 
 d3_listp       d3_listp                      f              0              
+d3_listp       d3_listp2      d3_listp       t              1              
 d3_listp1      d3_listp1                     t              0              
 step s1alter: ALTER TABLE d3_listp1 ALTER a DROP NOT NULL;
 ERROR:  cannot alter partition "d3_listp1" with an incomplete detach
@@ -81,6 +82,59 @@ error in steps s1cancel s2detach: ERROR:  canceling statement due to user reques
 step s1c: COMMIT;
 step s1insertpart: INSERT INTO d3_listp1 VALUES (1);
 
+starting permutation: s2snitch s1b s1s s2detach2 s1cancel s1c s1brr s1insert s1s s1insert s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach2: <... completed>
+error in steps s1cancel s2detach2: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1insert: INSERT INTO d3_listp VALUES (1);
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+1              
+step s1insert: INSERT INTO d3_listp VALUES (1);
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach2 s1cancel s1c s1brr s1s s1insert s1s s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach2: <... completed>
+error in steps s1cancel s2detach2: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s1insert: INSERT INTO d3_listp VALUES (1);
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+
 starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1drop s1list
 step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
 step s1b: BEGIN;
@@ -123,6 +177,61 @@ a
 
 1              
 
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detach2 s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+ERROR:  partition "d3_listp1" already pending detach in partitioned table "public.d3_listp"
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detachfinal s1c s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1droppart s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1droppart: DROP TABLE d3_listp1;
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
 starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2drop s1s s2commit
 step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
 step s1b: BEGIN;
@@ -279,4 +388,3 @@ step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
 step s1insertpart: INSERT INTO d3_listp1 VALUES (1); <waiting ...>
 step s2commit: COMMIT;
 step s1insertpart: <... completed>
-unused step name: s1droppart
diff --git a/src/test/isolation/specs/detach-partition-concurrently-3.spec b/src/test/isolation/specs/detach-partition-concurrently-3.spec
index 4b706430e1..5a8351fc83 100644
--- a/src/test/isolation/specs/detach-partition-concurrently-3.spec
+++ b/src/test/isolation/specs/detach-partition-concurrently-3.spec
@@ -4,12 +4,13 @@ setup
 {
   CREATE TABLE d3_listp (a int) PARTITION BY LIST(a);
   CREATE TABLE d3_listp1 PARTITION OF d3_listp FOR VALUES IN (1);
+  CREATE TABLE d3_listp2 PARTITION OF d3_listp FOR VALUES IN (2);
   CREATE TABLE d3_pid (pid int);
   INSERT INTO d3_listp VALUES (1);
 }
 
 teardown {
-    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_pid;
+    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_listp2, d3_pid;
 }
 
 session "s1"
@@ -34,6 +35,7 @@ session "s2"
 step "s2begin"		{ BEGIN; }
 step "s2snitch"		{ INSERT INTO d3_pid SELECT pg_backend_pid(); }
 step "s2detach"		{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; }
+step "s2detach2"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; }
 step "s2detachfinal"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; }
 step "s2drop"		{ DROP TABLE d3_listp1; }
 step "s2commit"		{ COMMIT; }
@@ -44,11 +46,21 @@ permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1describe" "s1a
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1insert" "s1c"
 permutation "s2snitch" "s1brr" "s1s" "s2detach" "s1cancel" "s1insert" "s1c" "s1spart"
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1insertpart"
+
+# Test partition descriptor caching
+permutation "s2snitch" "s1b" "s1s" "s2detach2" "s1cancel" "s1c" "s1brr" "s1insert" "s1s" "s1insert" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach2" "s1cancel" "s1c" "s1brr" "s1s" "s1insert" "s1s" "s1c"
+
 # "drop" here does both tables
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1drop" "s1list"
 # "truncate" only does parent, not partition
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1trunc" "s1spart"
 
+# If a partition pending detach exists, we cannot drop another one
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detach2" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detachfinal" "s1c" "s2detach2"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1droppart" "s2detach2"
+
 # When a partition with incomplete detach is dropped, we grab lock on parent too.
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2drop" "s1s" "s2commit"
 
-- 
2.20.1

#77Amit Langote
amitlangote09@gmail.com
In reply to: Alvaro Herrera (#76)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Wed, Apr 28, 2021 at 8:32 AM Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2021-Apr-27, Alvaro Herrera wrote:

This v3 handles things as you suggested and works correctly AFAICT. I'm
going to add some more tests cases to verify the behavior in the
scenarios you showed, and get them to run under cache-clobber options to
make sure it's good.

Yep, it seems to work. Strangely, the new isolation case doesn't
actually crash before the fix -- it merely throws a memory allocation
error.

Thanks. Yeah, it does seem to work.

I noticed that rd_partdesc_nodetached_xmin can sometimes end up with
value 0. While you seem to be already aware of that, because otherwise
you wouldn't have added TransactionIdIsValid(...) in condition in
RelationGetPartitionDesc(), the comments nearby don't mention why such
a thing might happen. Also, I guess it can't be helped that the
partdesc_nodetached will have to be leaked when the xmin is 0, but
that shouldn't be as problematic as the case we discussed earlier.

+   /*
+    * But first, a kluge: if there's an old context for this type of
+    * descriptor, it contains an old partition descriptor that may still be
+    * referenced somewhere.  Preserve it, while not leaking it, by
+    * reattaching it as a child context of the new one.  Eventually it will
+    * get dropped by either RelationClose or RelationClearRelation.
+    *
+    * We keep the regular partdesc in rd_pdcxt, and the partdesc-excluding-
+    * detached-partitions in rd_pddcxt.
+    */
+   context = is_omit ? &rel->rd_pddcxt : &rel->rd_pdcxt;
+   if (*context != NULL)
+       MemoryContextSetParent(*context, new_pdcxt);
+   *context = new_pdcxt;

Would it be a bit more readable to just duplicate this stanza in the
blocks that assign to rd_partdesc_nodetached and rd_partdesc,
respectively? That's not much code to duplicate and it'd be easier to
see which context is for which partdesc.

+ TransactionId rd_partdesc_nodetached_xmin; /* xmin for the above */

Could you please expand this description a bit?

--
Amit Langote
EDB: http://www.enterprisedb.com

#78Álvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#77)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Thanks for re-reviewing! This one I hope is the last version.

On Wed, Apr 28, 2021, at 10:21 AM, Amit Langote wrote:

I noticed that rd_partdesc_nodetached_xmin can sometimes end up with
value 0. While you seem to be already aware of that, because otherwise
you wouldn't have added TransactionIdIsValid(...) in condition in
RelationGetPartitionDesc(), the comments nearby don't mention why such
a thing might happen. Also, I guess it can't be helped that the
partdesc_nodetached will have to be leaked when the xmin is 0, but
that shouldn't be as problematic as the case we discussed earlier.

The only case I am aware where that can happen is if the pg_inherits tuple is frozen. (That's exactly what the affected test case was testing, note the "VACUUM FREEZE pg_inherits" there). So that test case blew up immediately; but I think the real-world chances that people are going to be doing that are pretty low, so I'm not really concerned about the leak.

Would it be a bit more readable to just duplicate this stanza in the
blocks that assign to rd_partdesc_nodetached and rd_partdesc,
respectively? That's not much code to duplicate and it'd be easier to
see which context is for which partdesc.

Sure .. that's how I first wrote this code. We don't use that style much, so I'm OK with backing out of it.

+ TransactionId rd_partdesc_nodetached_xmin; /* xmin for the above */

Could you please expand this description a bit?

Done.

Attachments:

v5-0001-Allow-a-partdesc-omitting-partitions-to-be-cached.patchtext/x-patch; name=v5-0001-Allow-a-partdesc-omitting-partitions-to-be-cached.patchDownload
From 8b1489a17cb5cfcd30d4dfd18020c64e606c5042 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Tue, 27 Apr 2021 19:04:37 -0400
Subject: [PATCH v5] Allow a partdesc-omitting-partitions to be cached

Makes partition descriptor acquisition faster during the transient
period in which a partition is in the process of being detached.

(This incidentally fixes a bug in 8aba9322511 whereby a memory context
holding a transient partdesc is reparented to NULL, leading to permanent
leak of that memory.  Reported by Amit Langote.)

Per gripe from Amit Langote
Discussion: https://postgr.es/m/CA+HiwqFgpP1LxJZOBYGt9rpvTjXXkg5qG2+Xch2Z1Q7KrqZR1A@mail.gmail.com
---
 src/backend/catalog/pg_inherits.c             |  52 ++++++++-
 src/backend/commands/tablecmds.c              |  66 ++++++-----
 src/backend/commands/trigger.c                |   3 +-
 src/backend/partitioning/partdesc.c           | 101 +++++++++++-----
 src/backend/utils/cache/relcache.c            |  33 +++++-
 src/include/catalog/pg_inherits.h             |   6 +-
 src/include/utils/rel.h                       |  15 +++
 .../detach-partition-concurrently-3.out       | 110 +++++++++++++++++-
 .../detach-partition-concurrently-3.spec      |  14 ++-
 9 files changed, 327 insertions(+), 73 deletions(-)

diff --git a/src/backend/catalog/pg_inherits.c b/src/backend/catalog/pg_inherits.c
index 6447b52854..c373faf2d6 100644
--- a/src/backend/catalog/pg_inherits.c
+++ b/src/backend/catalog/pg_inherits.c
@@ -52,6 +52,22 @@ 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
+ * find_inheritance_children_extended for details.
+ */
+List *
+find_inheritance_children(Oid parentrelId, LOCKMODE lockmode)
+{
+	return find_inheritance_children_extended(parentrelId, true, lockmode,
+											  NULL, NULL);
+}
+
+/*
+ * find_inheritance_children_extended
+ *
+ * As find_inheritance_children, with more options regarding detached
+ * partitions.
+ *
  * If a partition's pg_inherits row is marked "detach pending",
  * *detached_exist (if not null) is set true.
  *
@@ -60,11 +76,13 @@ typedef struct SeenRelsEntry
  * 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.
+ * committed to the active snapshot.  In addition, *detached_xmin (if not null)
+ * is set to the xmin of the row of the detached partition.
  */
 List *
-find_inheritance_children(Oid parentrelId, bool omit_detached,
-						  LOCKMODE lockmode, bool *detached_exist)
+find_inheritance_children_extended(Oid parentrelId, bool omit_detached,
+								   LOCKMODE lockmode, bool *detached_exist,
+								   TransactionId *detached_xmin)
 {
 	List	   *list = NIL;
 	Relation	relation;
@@ -132,7 +150,32 @@ find_inheritance_children(Oid parentrelId, bool omit_detached,
 				snap = GetActiveSnapshot();
 
 				if (!XidInMVCCSnapshot(xmin, snap))
+				{
+					if (detached_xmin)
+					{
+						/*
+						 * Two detached partitions should not occur (see
+						 * checks in MarkInheritDetached), but if they do,
+						 * track the newer of the two.  Make sure to warn the
+						 * user, so that they can clean up.  Since this is
+						 * just a cross-check against potentially corrupt
+						 * catalogs, we don't make it a full-fledged error
+						 * message.
+						 */
+						if (*detached_xmin != InvalidTransactionId)
+						{
+							elog(WARNING, "more than one partition pending detach found for table with OID %u",
+								 parentrelId);
+							if (TransactionIdFollows(xmin, *detached_xmin))
+								*detached_xmin = xmin;
+						}
+						else
+							*detached_xmin = xmin;
+					}
+
+					/* Don't add the partition to the output list */
 					continue;
+				}
 			}
 		}
 
@@ -247,8 +290,7 @@ find_all_inheritors(Oid parentrelId, LOCKMODE lockmode, List **numparents)
 		ListCell   *lc;
 
 		/* Get the direct children of this rel */
-		currentchildren = find_inheritance_children(currentrel, true,
-													lockmode, NULL);
+		currentchildren = find_inheritance_children(currentrel, lockmode);
 
 		/*
 		 * Add to the queue only those children not already seen. This avoids
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 8e717ada28..d9ba87a2a3 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3492,7 +3492,7 @@ renameatt_internal(Oid myrelid,
 		 * expected_parents will only be 0 if we are not already recursing.
 		 */
 		if (expected_parents == 0 &&
-			find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+			find_inheritance_children(myrelid, NoLock) != NIL)
 			ereport(ERROR,
 					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 					 errmsg("inherited column \"%s\" must be renamed in child tables too",
@@ -3691,7 +3691,7 @@ rename_constraint_internal(Oid myrelid,
 		else
 		{
 			if (expected_parents == 0 &&
-				find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+				find_inheritance_children(myrelid, NoLock) != NIL)
 				ereport(ERROR,
 						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 						 errmsg("inherited constraint \"%s\" must be renamed in child tables too",
@@ -6565,7 +6565,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 */
 	if (colDef->identity &&
 		recurse &&
-		find_inheritance_children(myrelid, true, NoLock, NULL) != NIL)
+		find_inheritance_children(myrelid, NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("cannot recursively add identity column to table that has child tables")));
@@ -6811,7 +6811,7 @@ ATExecAddColumn(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * If we are told not to recurse, there had better not be any child
@@ -7674,7 +7674,7 @@ ATPrepDropExpression(Relation rel, AlterTableCmd *cmd, bool recurse, bool recurs
 	 * resulting state can be properly dumped and restored.
 	 */
 	if (!recurse &&
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL))
+		find_inheritance_children(RelationGetRelid(rel), lockmode))
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("ALTER TABLE / DROP EXPRESSION must be applied to child tables too")));
@@ -8282,7 +8282,7 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	if (children)
 	{
@@ -8770,7 +8770,7 @@ ATAddCheckConstraint(List **wqueue, AlteredTableInfo *tab, Relation rel,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	children =
-		find_inheritance_children(RelationGetRelid(rel), true, lockmode, NULL);
+		find_inheritance_children(RelationGetRelid(rel), lockmode);
 
 	/*
 	 * Check if ONLY was specified with ALTER TABLE.  If so, allow the
@@ -11303,8 +11303,7 @@ ATExecDropConstraint(Relation rel, const char *constrName,
 	 * use find_all_inheritors to do it in one pass.
 	 */
 	if (!is_no_inherit_constraint)
-		children = find_inheritance_children(RelationGetRelid(rel), true,
-											 lockmode, NULL);
+		children = find_inheritance_children(RelationGetRelid(rel), lockmode);
 	else
 		children = NIL;
 
@@ -11688,8 +11687,7 @@ ATPrepAlterColumnType(List **wqueue,
 		}
 	}
 	else if (!recursing &&
-			 find_inheritance_children(RelationGetRelid(rel), true,
-									   NoLock, NULL) != NIL)
+			 find_inheritance_children(RelationGetRelid(rel), NoLock) != NIL)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
 				 errmsg("type of inherited column \"%s\" must be changed in child tables too",
@@ -14601,7 +14599,8 @@ ATExecDropInherit(Relation rel, RangeVar *parent, LOCKMODE lockmode)
  * MarkInheritDetached
  *
  * Set inhdetachpending for a partition, for ATExecDetachPartition
- * in concurrent mode.
+ * in concurrent mode.  While at it, verify that no other partition is
+ * already pending detach.
  */
 static void
 MarkInheritDetached(Relation child_rel, Relation parent_rel)
@@ -14617,32 +14616,45 @@ MarkInheritDetached(Relation child_rel, Relation parent_rel)
 	Assert(parent_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
 	/*
-	 * Find pg_inherits entries by inhrelid.
+	 * Find pg_inherits entries by inhparent.  (We need to scan them all in
+	 * order to verify that no other partition is pending detach.)
 	 */
 	catalogRelation = table_open(InheritsRelationId, RowExclusiveLock);
 	ScanKeyInit(&key,
-				Anum_pg_inherits_inhrelid,
+				Anum_pg_inherits_inhparent,
 				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(RelationGetRelid(child_rel)));
-	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId,
+				ObjectIdGetDatum(RelationGetRelid(parent_rel)));
+	scan = systable_beginscan(catalogRelation, InheritsParentIndexId,
 							  true, NULL, 1, &key);
 
 	while (HeapTupleIsValid(inheritsTuple = systable_getnext(scan)))
 	{
-		HeapTuple	newtup;
+		Form_pg_inherits inhForm;
 
-		if (((Form_pg_inherits) GETSTRUCT(inheritsTuple))->inhparent !=
-			RelationGetRelid(parent_rel))
-			elog(ERROR, "bad parent tuple found for partition %u",
-				 RelationGetRelid(child_rel));
+		inhForm = (Form_pg_inherits) GETSTRUCT(inheritsTuple);
+		if (inhForm->inhdetachpending)
+			ereport(ERROR,
+					errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					errmsg("partition \"%s\" already pending detach in partitioned table \"%s.%s\"",
+						   get_rel_name(inhForm->inhrelid),
+						   get_namespace_name(parent_rel->rd_rel->relnamespace),
+						   RelationGetRelationName(parent_rel)),
+					errhint("Use ALTER TABLE ... DETACH PARTITION ... FINALIZE to complete the detach operation."));
+
+		if (inhForm->inhrelid == RelationGetRelid(child_rel))
+		{
+			HeapTuple	newtup;
 
-		newtup = heap_copytuple(inheritsTuple);
-		((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
+			newtup = heap_copytuple(inheritsTuple);
+			((Form_pg_inherits) GETSTRUCT(newtup))->inhdetachpending = true;
 
-		CatalogTupleUpdate(catalogRelation,
-						   &inheritsTuple->t_self,
-						   newtup);
-		found = true;
+			CatalogTupleUpdate(catalogRelation,
+							   &inheritsTuple->t_self,
+							   newtup);
+			found = true;
+			heap_freetuple(newtup);
+			/* keep looking, to ensure we catch others pending detach */
+		}
 	}
 
 	/* Done */
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index d8393aa4de..f305f8bc0f 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -1141,8 +1141,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 			ListCell   *l;
 			List	   *idxs = NIL;
 
-			idxs = find_inheritance_children(indexOid, true,
-											 ShareRowExclusiveLock, NULL);
+			idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
 			foreach(l, idxs)
 				childTbls = lappend_oid(childTbls,
 										IndexGetRelation(lfirst_oid(l),
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index 2305dff407..fa9729283a 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -54,6 +54,12 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
 /*
  * RelationGetPartitionDesc -- get partition descriptor, if relation is partitioned
  *
+ * 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.
+ *
  * Note: we arrange for partition descriptors to not get freed until the
  * relcache entry's refcount goes to zero (see hacks in RelationClose,
  * RelationClearRelation, and RelationBuildPartitionDesc).  Therefore, even
@@ -61,13 +67,6 @@ static PartitionDesc RelationBuildPartitionDesc(Relation rel,
  * for callers to continue to use that pointer as long as (a) they hold the
  * relation open, and (b) they hold a relation lock strong enough to ensure
  * that the data doesn't become stale.
- *
- * The above applies to partition descriptors that are complete regarding
- * partitions concurrently being detached.  When a descriptor that omits
- * partitions being detached is requested (and such partitions are present),
- * said descriptor is not part of relcache and so it isn't freed by
- * invalidations either.  Caller must not use such a descriptor beyond the
- * current Portal.
  */
 PartitionDesc
 RelationGetPartitionDesc(Relation rel, bool omit_detached)
@@ -78,11 +77,36 @@ RelationGetPartitionDesc(Relation rel, bool omit_detached)
 	 * If relcache has a partition descriptor, use that.  However, we can only
 	 * 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.
 	 */
 	if (likely(rel->rd_partdesc &&
-			   (!rel->rd_partdesc->detached_exist || !omit_detached)))
+			   (!rel->rd_partdesc->detached_exist || !omit_detached ||
+				!ActiveSnapshotSet())))
 		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 use it.  Otherwise build one
+	 * from scratch.
+	 */
+	if (omit_detached &&
+		rel->rd_partdesc_nodetached &&
+		TransactionIdIsValid(rel->rd_partdesc_nodetached_xmin) &&
+		ActiveSnapshotSet())
+	{
+		Snapshot	activesnap;
+
+		activesnap = GetActiveSnapshot();
+
+		if (!XidInMVCCSnapshot(rel->rd_partdesc_nodetached_xmin, activesnap))
+			return rel->rd_partdesc_nodetached;
+	}
+
 	return RelationBuildPartitionDesc(rel, omit_detached);
 }
 
@@ -117,6 +141,8 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	Oid		   *oids = NULL;
 	bool	   *is_leaf = NULL;
 	bool		detached_exist;
+	bool		is_omit;
+	TransactionId detached_xmin;
 	ListCell   *cell;
 	int			i,
 				nparts;
@@ -132,8 +158,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	 * some well-defined point in time.
 	 */
 	detached_exist = false;
-	inhoids = find_inheritance_children(RelationGetRelid(rel), omit_detached,
-										NoLock, &detached_exist);
+	detached_xmin = InvalidTransactionId;
+	inhoids = find_inheritance_children_extended(RelationGetRelid(rel),
+												 omit_detached, NoLock,
+												 &detached_exist,
+												 &detached_xmin);
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -280,38 +309,50 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 		MemoryContextSwitchTo(oldcxt);
 	}
 
+	/*
+	 * Are we working with the partition that omits detached partitions, or
+	 * the one that includes them?
+	 */
+	is_omit = omit_detached && detached_exist && ActiveSnapshotSet();
+
 	/*
 	 * We have a fully valid partdesc.  Reparent it so that it has the right
-	 * lifespan, and if appropriate put it into the relation's relcache entry.
+	 * lifespan.
 	 */
-	if (omit_detached && detached_exist)
+	MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
+
+	/*
+	 * Store it into relcache.
+	 *
+	 * But first, a kluge: if there's an old context for this type of
+	 * descriptor, it contains an old partition descriptor that may still be
+	 * referenced somewhere.  Preserve it, while not leaking it, by
+	 * reattaching it as a child context of the new one.  Eventually it will
+	 * get dropped by either RelationClose or RelationClearRelation.
+	 * (We keep the regular partdesc in rd_pdcxt, and the partdesc-excluding-
+	 * detached-partitions in rd_pddcxt.)
+	 */
+	if (is_omit)
 	{
+		if (rel->rd_pddcxt != NULL)
+			MemoryContextSetParent(rel->rd_pddcxt, new_pdcxt);
+		rel->rd_pddcxt = new_pdcxt;
+		rel->rd_partdesc_nodetached = partdesc;
+
 		/*
-		 * A transient partition descriptor is only good for the current
-		 * statement, so make it a child of the current portal's context.
+		 * For partdescs built excluding detached partitions, which we save
+		 * separately, we also record the pg_inherits.xmin of the detached
+		 * partition that was omitted; this informs a future potential user of
+		 * such a cached partdescs.  (This might be InvalidXid; see comments
+		 * in struct RelationData).
 		 */
-		MemoryContextSetParent(new_pdcxt, PortalContext);
+		rel->rd_partdesc_nodetached_xmin = detached_xmin;
 	}
 	else
 	{
-		/*
-		 * This partdesc goes into relcache.
-		 */
-
-		MemoryContextSetParent(new_pdcxt, CacheMemoryContext);
-
-		/*
-		 * But first, a kluge: if there's an old rd_pdcxt, it contains an old
-		 * partition descriptor that may still be referenced somewhere.
-		 * Preserve it, while not leaking it, by reattaching it as a child
-		 * context of the new rd_pdcxt.  Eventually it will get dropped by
-		 * either RelationClose or RelationClearRelation.
-		 */
 		if (rel->rd_pdcxt != NULL)
 			MemoryContextSetParent(rel->rd_pdcxt, new_pdcxt);
 		rel->rd_pdcxt = new_pdcxt;
-
-		/* Store it into relcache */
 		rel->rd_partdesc = partdesc;
 	}
 
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 466c28d528..31c8e07f2a 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -1157,7 +1157,10 @@ RelationBuildDesc(Oid targetRelId, bool insertIt)
 	relation->rd_partkey = NULL;
 	relation->rd_partkeycxt = NULL;
 	relation->rd_partdesc = NULL;
+	relation->rd_partdesc_nodetached = NULL;
+	relation->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 	relation->rd_pdcxt = NULL;
+	relation->rd_pddcxt = NULL;
 	relation->rd_partcheck = NIL;
 	relation->rd_partcheckvalid = false;
 	relation->rd_partcheckcxt = NULL;
@@ -2103,10 +2106,16 @@ RelationClose(Relation relation)
 	 * stale partition descriptors it has.  This is unlikely, so check to see
 	 * if there are child contexts before expending a call to mcxt.c.
 	 */
-	if (RelationHasReferenceCountZero(relation) &&
-		relation->rd_pdcxt != NULL &&
-		relation->rd_pdcxt->firstchild != NULL)
-		MemoryContextDeleteChildren(relation->rd_pdcxt);
+	if (RelationHasReferenceCountZero(relation))
+	{
+		if (relation->rd_pdcxt != NULL &&
+			relation->rd_pdcxt->firstchild != NULL)
+			MemoryContextDeleteChildren(relation->rd_pdcxt);
+
+		if (relation->rd_pddcxt != NULL &&
+			relation->rd_pddcxt->firstchild != NULL)
+			MemoryContextDeleteChildren(relation->rd_pddcxt);
+	}
 
 #ifdef RELCACHE_FORCE_RELEASE
 	if (RelationHasReferenceCountZero(relation) &&
@@ -2390,6 +2399,8 @@ RelationDestroyRelation(Relation relation, bool remember_tupdesc)
 		MemoryContextDelete(relation->rd_partkeycxt);
 	if (relation->rd_pdcxt)
 		MemoryContextDelete(relation->rd_pdcxt);
+	if (relation->rd_pddcxt)
+		MemoryContextDelete(relation->rd_pddcxt);
 	if (relation->rd_partcheckcxt)
 		MemoryContextDelete(relation->rd_partcheckcxt);
 	pfree(relation);
@@ -2644,7 +2655,7 @@ RelationClearRelation(Relation relation, bool rebuild)
 			SWAPFIELD(PartitionKey, rd_partkey);
 			SWAPFIELD(MemoryContext, rd_partkeycxt);
 		}
-		if (newrel->rd_pdcxt != NULL)
+		if (newrel->rd_pdcxt != NULL || newrel->rd_pddcxt != NULL)
 		{
 			/*
 			 * We are rebuilding a partitioned relation with a non-zero
@@ -2672,13 +2683,22 @@ RelationClearRelation(Relation relation, bool rebuild)
 			 * newrel.
 			 */
 			relation->rd_partdesc = NULL;	/* ensure rd_partdesc is invalid */
+			relation->rd_partdesc_nodetached = NULL;
+			relation->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 			if (relation->rd_pdcxt != NULL) /* probably never happens */
 				MemoryContextSetParent(newrel->rd_pdcxt, relation->rd_pdcxt);
 			else
 				relation->rd_pdcxt = newrel->rd_pdcxt;
+			if (relation->rd_pddcxt != NULL)
+				MemoryContextSetParent(newrel->rd_pddcxt, relation->rd_pddcxt);
+			else
+				relation->rd_pddcxt = newrel->rd_pddcxt;
 			/* drop newrel's pointers so we don't destroy it below */
 			newrel->rd_partdesc = NULL;
+			newrel->rd_partdesc_nodetached = NULL;
+			newrel->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 			newrel->rd_pdcxt = NULL;
+			newrel->rd_pddcxt = NULL;
 		}
 
 #undef SWAPFIELD
@@ -6017,7 +6037,10 @@ load_relcache_init_file(bool shared)
 		rel->rd_partkey = NULL;
 		rel->rd_partkeycxt = NULL;
 		rel->rd_partdesc = NULL;
+		rel->rd_partdesc_nodetached = NULL;
+		rel->rd_partdesc_nodetached_xmin = InvalidTransactionId;
 		rel->rd_pdcxt = NULL;
+		rel->rd_pddcxt = NULL;
 		rel->rd_partcheck = NIL;
 		rel->rd_partcheckvalid = false;
 		rel->rd_partcheckcxt = NULL;
diff --git a/src/include/catalog/pg_inherits.h b/src/include/catalog/pg_inherits.h
index 4d28ede5a6..f47c0a8cd8 100644
--- a/src/include/catalog/pg_inherits.h
+++ b/src/include/catalog/pg_inherits.h
@@ -50,8 +50,10 @@ DECLARE_INDEX(pg_inherits_parent_index, 2187, on pg_inherits using btree(inhpare
 #define InheritsParentIndexId  2187
 
 
-extern List *find_inheritance_children(Oid parentrelId, bool omit_detached,
-									   LOCKMODE lockmode, bool *detached_exist);
+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);
+
 extern List *find_all_inheritors(Oid parentrelId, LOCKMODE lockmode,
 								 List **parents);
 extern bool has_subclass(Oid relationId);
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 34e25eb597..07ae2c70a9 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -129,6 +129,21 @@ typedef struct RelationData
 	PartitionDesc rd_partdesc;	/* partition descriptor, or NULL */
 	MemoryContext rd_pdcxt;		/* private context for rd_partdesc, if any */
 
+	/* Same as above, for partdescs that omit detached partitions */
+	PartitionDesc rd_partdesc_nodetached;	/* partdesc w/o detached parts */
+	MemoryContext rd_pddcxt;	/* for rd_partdesc_nodetached, if any */
+
+	/*
+	 * pg_inherits.xmin of the partition that was excluded in
+	 * rd_partdesc_nodetached.  This informs a future user of that partdesc:
+	 * if this value is not in progress for the active snapshot, then the
+	 * partdesc can be used, otherwise they have to build a new one.  (This
+	 * matches what find_inheritance_children_extended would do).  In the rare
+	 * case where the pg_inherits tuple has been frozen, this will be
+	 * InvalidXid; behave as if the partdesc is unusable in that case.
+	 */
+	TransactionId rd_partdesc_nodetached_xmin;
+
 	/* data managed by RelationGetPartitionQual: */
 	List	   *rd_partcheck;	/* partition CHECK quals */
 	bool		rd_partcheckvalid;	/* true if list has been computed */
diff --git a/src/test/isolation/expected/detach-partition-concurrently-3.out b/src/test/isolation/expected/detach-partition-concurrently-3.out
index 88e83638c7..bbb9d151fb 100644
--- a/src/test/isolation/expected/detach-partition-concurrently-3.out
+++ b/src/test/isolation/expected/detach-partition-concurrently-3.out
@@ -20,6 +20,7 @@ step s1describe: SELECT 'd3_listp' AS root, * FROM pg_partition_tree('d3_listp')
 root           relid          parentrelid    isleaf         level          
 
 d3_listp       d3_listp                      f              0              
+d3_listp       d3_listp2      d3_listp       t              1              
 d3_listp1      d3_listp1                     t              0              
 step s1alter: ALTER TABLE d3_listp1 ALTER a DROP NOT NULL;
 ERROR:  cannot alter partition "d3_listp1" with an incomplete detach
@@ -81,6 +82,59 @@ error in steps s1cancel s2detach: ERROR:  canceling statement due to user reques
 step s1c: COMMIT;
 step s1insertpart: INSERT INTO d3_listp1 VALUES (1);
 
+starting permutation: s2snitch s1b s1s s2detach2 s1cancel s1c s1brr s1insert s1s s1insert s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach2: <... completed>
+error in steps s1cancel s2detach2: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1insert: INSERT INTO d3_listp VALUES (1);
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+1              
+step s1insert: INSERT INTO d3_listp VALUES (1);
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach2 s1cancel s1c s1brr s1s s1insert s1s s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach2: <... completed>
+error in steps s1cancel s2detach2: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s1insert: INSERT INTO d3_listp VALUES (1);
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+1              
+step s1c: COMMIT;
+
 starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1drop s1list
 step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
 step s1b: BEGIN;
@@ -123,6 +177,61 @@ a
 
 1              
 
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detach2 s1c
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+ERROR:  partition "d3_listp1" already pending detach in partitioned table "public.d3_listp"
+step s1c: COMMIT;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s2detachfinal s1c s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; <waiting ...>
+step s1c: COMMIT;
+step s2detachfinal: <... completed>
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
+starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s1droppart s2detach2
+step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
+step s1b: BEGIN;
+step s1s: SELECT * FROM d3_listp;
+a              
+
+1              
+step s2detach: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; <waiting ...>
+step s1cancel: SELECT pg_cancel_backend(pid) FROM d3_pid;
+pg_cancel_backend
+
+t              
+step s2detach: <... completed>
+error in steps s1cancel s2detach: ERROR:  canceling statement due to user request
+step s1c: COMMIT;
+step s1droppart: DROP TABLE d3_listp1;
+step s2detach2: ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY;
+
 starting permutation: s2snitch s1b s1s s2detach s1cancel s1c s2begin s2drop s1s s2commit
 step s2snitch: INSERT INTO d3_pid SELECT pg_backend_pid();
 step s1b: BEGIN;
@@ -279,4 +388,3 @@ step s2detachfinal: ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE;
 step s1insertpart: INSERT INTO d3_listp1 VALUES (1); <waiting ...>
 step s2commit: COMMIT;
 step s1insertpart: <... completed>
-unused step name: s1droppart
diff --git a/src/test/isolation/specs/detach-partition-concurrently-3.spec b/src/test/isolation/specs/detach-partition-concurrently-3.spec
index 4b706430e1..5a8351fc83 100644
--- a/src/test/isolation/specs/detach-partition-concurrently-3.spec
+++ b/src/test/isolation/specs/detach-partition-concurrently-3.spec
@@ -4,12 +4,13 @@ setup
 {
   CREATE TABLE d3_listp (a int) PARTITION BY LIST(a);
   CREATE TABLE d3_listp1 PARTITION OF d3_listp FOR VALUES IN (1);
+  CREATE TABLE d3_listp2 PARTITION OF d3_listp FOR VALUES IN (2);
   CREATE TABLE d3_pid (pid int);
   INSERT INTO d3_listp VALUES (1);
 }
 
 teardown {
-    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_pid;
+    DROP TABLE IF EXISTS d3_listp, d3_listp1, d3_listp2, d3_pid;
 }
 
 session "s1"
@@ -34,6 +35,7 @@ session "s2"
 step "s2begin"		{ BEGIN; }
 step "s2snitch"		{ INSERT INTO d3_pid SELECT pg_backend_pid(); }
 step "s2detach"		{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 CONCURRENTLY; }
+step "s2detach2"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp2 CONCURRENTLY; }
 step "s2detachfinal"	{ ALTER TABLE d3_listp DETACH PARTITION d3_listp1 FINALIZE; }
 step "s2drop"		{ DROP TABLE d3_listp1; }
 step "s2commit"		{ COMMIT; }
@@ -44,11 +46,21 @@ permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1describe" "s1a
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1insert" "s1c"
 permutation "s2snitch" "s1brr" "s1s" "s2detach" "s1cancel" "s1insert" "s1c" "s1spart"
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1insertpart"
+
+# Test partition descriptor caching
+permutation "s2snitch" "s1b" "s1s" "s2detach2" "s1cancel" "s1c" "s1brr" "s1insert" "s1s" "s1insert" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach2" "s1cancel" "s1c" "s1brr" "s1s" "s1insert" "s1s" "s1c"
+
 # "drop" here does both tables
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1drop" "s1list"
 # "truncate" only does parent, not partition
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1trunc" "s1spart"
 
+# If a partition pending detach exists, we cannot drop another one
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detach2" "s1c"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s2detachfinal" "s1c" "s2detach2"
+permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s1droppart" "s2detach2"
+
 # When a partition with incomplete detach is dropped, we grab lock on parent too.
 permutation "s2snitch" "s1b" "s1s" "s2detach" "s1cancel" "s1c" "s2begin" "s2drop" "s1s" "s2commit"
 
-- 
2.20.1

#79Álvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Álvaro Herrera (#78)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Pushed that now, with a one-line addition to the docs that only one
partition can be marked detached.

--
�lvaro Herrera 39�49'30"S 73�17'W
"That sort of implies that there are Emacs keystrokes which aren't obscure.
I've been using it daily for 2 years now and have yet to discover any key
sequence which makes any sense." (Paul Thomas)

#80Amit Langote
amitlangote09@gmail.com
In reply to: Álvaro Herrera (#78)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

(Thanks for committing the fix.)

On Thu, Apr 29, 2021 at 1:11 AM Álvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On Wed, Apr 28, 2021, at 10:21 AM, Amit Langote wrote:
I noticed that rd_partdesc_nodetached_xmin can sometimes end up with
value 0. While you seem to be already aware of that, because otherwise
you wouldn't have added TransactionIdIsValid(...) in condition in
RelationGetPartitionDesc(), the comments nearby don't mention why such
a thing might happen. Also, I guess it can't be helped that the
partdesc_nodetached will have to be leaked when the xmin is 0, but
that shouldn't be as problematic as the case we discussed earlier.

The only case I am aware where that can happen is if the pg_inherits tuple is frozen. (That's exactly what the affected test case was testing, note the "VACUUM FREEZE pg_inherits" there). So that test case blew up immediately; but I think the real-world chances that people are going to be doing that are pretty low, so I'm not really concerned about the leak.

The case I was looking at is when a partition detach appears as
in-progress to a serializable transaction. If the caller wants to
omit detached partitions, such a partition ends up in
rd_partdesc_nodetached, with the corresponding xmin being set to 0 due
to the way find_inheritance_children_extended() sets *detached_xmin.
The next query in the transaction that wants to omit detached
partitions, seeing rd_partdesc_nodetached_xmin being invalid, rebuilds
the partdesc, again including that partition because the snapshot
wouldn't have changed, and so on until the transaction ends. Now,
this can perhaps be "fixed" by making
find_inheritance_children_extended() set the xmin outside the
snapshot-checking block, but maybe there's no need to address this on
priority.

Rather, a point that bothers me a bit is that we're including a
detached partition in the partdesc labeled "nodetached" in this
particular case. Maybe we should avoid that by considering in this
scenario that no detached partitions exist for this transactions and
so initialize rd_partdesc, instead of rd_partdesc_nodetached. That
will let us avoid the situations where the xmin is left in invalid
state. Maybe like the attached (it also fixes a couple of
typos/thinkos in the previous commit).

Note that we still end up in the same situation as before where each
query in the serializable transaction that sees the detach as
in-progress to have to rebuild the partition descriptor omitting the
detached partitions, even when it's clear that the detach-in-progress
partition will be included every time.

--
Amit Langote
EDB: http://www.enterprisedb.com

Attachments:

partdesc_nodetached-only-if-detached-visible.patchapplication/octet-stream; name=partdesc_nodetached-only-if-detached-visible.patchDownload
diff --git a/src/backend/partitioning/partdesc.c b/src/backend/partitioning/partdesc.c
index fa9729283a..ca3ebd9e0b 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -163,6 +163,7 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 												 omit_detached, NoLock,
 												 &detached_exist,
 												 &detached_xmin);
+
 	nparts = list_length(inhoids);
 
 	/* Allocate working arrays for OIDs, leaf flags, and boundspecs. */
@@ -310,10 +311,17 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 	}
 
 	/*
-	 * Are we working with the partition that omits detached partitions, or
-	 * the one that includes them?
+	 * Are we working with the partdesc that omits the detached partition, or
+	 * the one that includes it?
+	 *
+	 * 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 would 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 && ActiveSnapshotSet() &&
+				TransactionIdIsValid(detached_xmin);
 
 	/*
 	 * We have a fully valid partdesc.  Reparent it so that it has the right
@@ -343,9 +351,11 @@ RelationBuildPartitionDesc(Relation rel, bool omit_detached)
 		 * For partdescs built excluding detached partitions, which we save
 		 * separately, we also record the pg_inherits.xmin of the detached
 		 * partition that was omitted; this informs a future potential user of
-		 * such a cached partdescs.  (This might be InvalidXid; see comments
-		 * in struct RelationData).
+		 * such a cached partdesc to only use it after cross-checking that the
+		 * xmin is indeed visible to the snapshot it is going to be working
+		 * with.
 		 */
+		Assert(TransactionIdIsValid(detached_xmin));
 		rel->rd_partdesc_nodetached_xmin = detached_xmin;
 	}
 	else
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 07ae2c70a9..5bd44be9a7 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -138,9 +138,7 @@ typedef struct RelationData
 	 * rd_partdesc_nodetached.  This informs a future user of that partdesc:
 	 * if this value is not in progress for the active snapshot, then the
 	 * partdesc can be used, otherwise they have to build a new one.  (This
-	 * matches what find_inheritance_children_extended would do).  In the rare
-	 * case where the pg_inherits tuple has been frozen, this will be
-	 * InvalidXid; behave as if the partdesc is unusable in that case.
+	 * matches what find_inheritance_children_extended would do).
 	 */
 	TransactionId rd_partdesc_nodetached_xmin;
 
#81Pavel Luzanov
p.luzanov@postgrespro.ru
In reply to: Amit Langote (#80)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

Hello,

I found this in the documentation, section '5.11.3. Partitioning Using
Inheritance'[1]:
"Some operations require a stronger lock when using declarative
partitioning than when using table inheritance. For example, removing a
partition from a partitioned table requires taking an ACCESS EXCLUSIVE
lock on the parent table, whereas a SHARE UPDATE EXCLUSIVE lock is
enough in the case of regular inheritance."

This point is no longer valid with some restrictions. If the table has a
default partition, then removing a partition still requires taking an
ACCESS EXCLUSIVE lock.

May be make sense to add some details about DETACH CONCURRENTLY to the
section '5.11.2.2. Partition Maintenance' and completely remove this point?

1.
https://www.postgresql.org/docs/devel/ddl-partitioning.html#DDL-PARTITIONING-USING-INHERITANCE

Pavel Luzanov
Postgres Professional: https://postgrespro.com
The Russian Postgres Company

#82Amit Langote
amitlangote09@gmail.com
In reply to: Pavel Luzanov (#81)
1 attachment(s)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Wed, May 5, 2021 at 7:59 PM Pavel Luzanov <p.luzanov@postgrespro.ru> wrote:

I found this in the documentation, section '5.11.3. Partitioning Using Inheritance'[1]:
"Some operations require a stronger lock when using declarative partitioning than when using table inheritance. For example, removing a partition from a partitioned table requires taking an ACCESS EXCLUSIVE lock on the parent table, whereas a SHARE UPDATE EXCLUSIVE lock is enough in the case of regular inheritance."

This point is no longer valid with some restrictions. If the table has a default partition, then removing a partition still requires taking an ACCESS EXCLUSIVE lock.

May be make sense to add some details about DETACH CONCURRENTLY to the section '5.11.2.2. Partition Maintenance' and completely remove this point?

1. https://www.postgresql.org/docs/devel/ddl-partitioning.html#DDL-PARTITIONING-USING-INHERITANCE

That makes sense, thanks for noticing.

How about the attached?

--
Amit Langote
EDB: http://www.enterprisedb.com

Attachments:

ddl-partitioning-note-detach-concurrenltly.patchapplication/octet-stream; name=ddl-partitioning-note-detach-concurrenltly.patchDownload
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 513112a216..bb494b3f89 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -3926,7 +3926,10 @@ ALTER TABLE measurement DETACH PARTITION measurement_y2006m02;
      the data using <command>COPY</command>, <application>pg_dump</application>, or
      similar tools. It might also be a useful time to aggregate data
      into smaller formats, perform other data manipulations, or run
-     reports.
+     reports.  Moroever, specifying <literal>CONCURRENTLY</literal> allows
+     the detach operation to finish while taking only
+     <literal>SHARE UPDATE EXCLUSIVE</literal> lock on the parent table; see
+     <xref linkend="sql-altertable-detach-partition"/> for more information.
    </para>
 
    <para>
@@ -4163,17 +4166,6 @@ ALTER INDEX measurement_city_id_logdate_key
         might be poor.)
        </para>
       </listitem>
-
-      <listitem>
-       <para>
-        Some operations require a stronger lock when using declarative
-        partitioning than when using table inheritance.  For example,
-        removing a partition from a partitioned table requires taking
-        an <literal>ACCESS EXCLUSIVE</literal> lock on the parent table,
-        whereas a <literal>SHARE UPDATE EXCLUSIVE</literal> lock is enough
-        in the case of regular inheritance.
-       </para>
-      </listitem>
      </itemizedlist>
     </para>
 
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 6166b26334..3cce2fdbde 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -953,7 +953,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
     </listitem>
    </varlistentry>
 
-   <varlistentry>
+   <varlistentry id="sql-altertable-detach-partition">
     <term><literal>DETACH PARTITION <replaceable class="parameter">partition_name</replaceable> [ CONCURRENTLY | FINALIZE ]</literal></term>
 
     <listitem>
#83Pavel Luzanov
p.luzanov@postgrespro.ru
In reply to: Amit Langote (#82)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 06.05.2021 08:35, Amit Langote wrote:

On Wed, May 5, 2021 at 7:59 PM Pavel Luzanov
<p.luzanov@postgrespro.ru> wrote:

I found this in the documentation, section '5.11.3. Partitioning
Using Inheritance'[1]: "Some operations require a stronger lock when
using declarative partitioning than when using table inheritance. For
example, removing a partition from a partitioned table requires
taking an ACCESS EXCLUSIVE lock on the parent table, whereas a SHARE
UPDATE EXCLUSIVE lock is enough in the case of regular inheritance."
This point is no longer valid with some restrictions. If the table
has a default partition, then removing a partition still requires
taking an ACCESS EXCLUSIVE lock. May be make sense to add some
details about DETACH CONCURRENTLY to the section '5.11.2.2. Partition
Maintenance' and completely remove this point? 1.
https://www.postgresql.org/docs/devel/ddl-partitioning.html#DDL-PARTITIONING-USING-INHERITANCE

That makes sense, thanks for noticing. How about the attached?

I like it.
Especially the link to the ALTER TABLE, this avoids duplication of all
the nuances of the the DETACH .. CONCURRENTLY.

--
Pavel Luzanov
Postgres Professional: https://postgrespro.com
The Russian Postgres Company

#84Álvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#80)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-Apr-30, Amit Langote wrote:

The case I was looking at is when a partition detach appears as
in-progress to a serializable transaction.

Yeah, I was exceedingly sloppy on my reasoning about this, and you're
right that that's what actually happens rather than what I said.

If the caller wants to omit detached partitions, such a partition ends
up in rd_partdesc_nodetached, with the corresponding xmin being set to
0 due to the way find_inheritance_children_extended() sets
*detached_xmin. The next query in the transaction that wants to omit
detached partitions, seeing rd_partdesc_nodetached_xmin being invalid,
rebuilds the partdesc, again including that partition because the
snapshot wouldn't have changed, and so on until the transaction ends.
Now, this can perhaps be "fixed" by making
find_inheritance_children_extended() set the xmin outside the
snapshot-checking block, but maybe there's no need to address this on
priority.

Hmm. See below.

Rather, a point that bothers me a bit is that we're including a
detached partition in the partdesc labeled "nodetached" in this
particular case. Maybe we should avoid that by considering in this
scenario that no detached partitions exist for this transactions and
so initialize rd_partdesc, instead of rd_partdesc_nodetached. That
will let us avoid the situations where the xmin is left in invalid
state. Maybe like the attached (it also fixes a couple of
typos/thinkos in the previous commit).

Makes sense -- applied, thanks.

Note that we still end up in the same situation as before where each
query in the serializable transaction that sees the detach as
in-progress to have to rebuild the partition descriptor omitting the
detached partitions, even when it's clear that the detach-in-progress
partition will be included every time.

Yeah, you're right that there is a performance hole in the case where a
partition pending detach exists and you're using repeatable read
transactions. I didn't see it as terribly critical since it's supposed
to be very transient, but I may be wrong.

--
�lvaro Herrera Valdivia, Chile
"Hay quien adquiere la mala costumbre de ser infeliz" (M. A. Evans)

#85Álvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Pavel Luzanov (#81)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-May-05, Pavel Luzanov wrote:

Hello,

I found this in the documentation, section '5.11.3. Partitioning Using
Inheritance'[1]:
"Some operations require a stronger lock when using declarative partitioning
than when using table inheritance. For example, removing a partition from a
partitioned table requires taking an ACCESS EXCLUSIVE lock on the parent
table, whereas a SHARE UPDATE EXCLUSIVE lock is enough in the case of
regular inheritance."

This point is no longer valid with some restrictions. If the table has a
default partition, then removing a partition still requires taking an ACCESS
EXCLUSIVE lock.

Hmm, are there any other operations for which the partitioning command
takes a strong lock than the legacy inheritance corresponding command?
If there aren't any, then it's okay to delete this paragraph as in the
proposed patch. But if there are any, then I think we should change the
example to mention that other operation. I'm not sure what's a good way
to verify that, though.

Also, it remains true that without CONCURRENTLY the DETACH operation
takes AEL. I'm not sure it's worth pointing this out in this paragraph.

May be make sense to add some details about DETACH CONCURRENTLY to the
section '5.11.2.2. Partition Maintenance' and completely remove this point?

Maybe you're right, though.

--
�lvaro Herrera 39�49'30"S 73�17'W
"Las mujeres son como hondas: mientras m�s resistencia tienen,
m�s lejos puedes llegar con ellas" (Jonas Nightingale, Leap of Faith)

#86Álvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#82)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-May-06, Amit Langote wrote:

That makes sense, thanks for noticing.

How about the attached?

I tweaked the linkage; as submitted, the text in the link contained what
is in the <term> tag, so literally it had:

... see DETACH PARTITION partition_name [ CONCURRENTLY | FINALIZE ] for
details ...

which didn't look very nice. So I made it use <link> instead of xref
and wrote the "ALTER TABLE .. DETACH PARTITION" text. I first tried to
fix it by adding an "xreflabel" attrib, but I didn't like it because the
text was not set in fixed width font.

I also tweaked the wording to match the surrounding text a bit better,
at least IMO. Feel free to suggest improvements.

Thanks!

--
�lvaro Herrera 39�49'30"S 73�17'W
"That sort of implies that there are Emacs keystrokes which aren't obscure.
I've been using it daily for 2 years now and have yet to discover any key
sequence which makes any sense." (Paul Thomas)

#87Amit Langote
amitlangote09@gmail.com
In reply to: Álvaro Herrera (#84)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Fri, May 7, 2021 at 2:13 AM Álvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2021-Apr-30, Amit Langote wrote:

The case I was looking at is when a partition detach appears as
in-progress to a serializable transaction.

Yeah, I was exceedingly sloppy on my reasoning about this, and you're
right that that's what actually happens rather than what I said.

If the caller wants to omit detached partitions, such a partition ends
up in rd_partdesc_nodetached, with the corresponding xmin being set to
0 due to the way find_inheritance_children_extended() sets
*detached_xmin. The next query in the transaction that wants to omit
detached partitions, seeing rd_partdesc_nodetached_xmin being invalid,
rebuilds the partdesc, again including that partition because the
snapshot wouldn't have changed, and so on until the transaction ends.
Now, this can perhaps be "fixed" by making
find_inheritance_children_extended() set the xmin outside the
snapshot-checking block, but maybe there's no need to address this on
priority.

Hmm. See below.

Rather, a point that bothers me a bit is that we're including a
detached partition in the partdesc labeled "nodetached" in this
particular case. Maybe we should avoid that by considering in this
scenario that no detached partitions exist for this transactions and
so initialize rd_partdesc, instead of rd_partdesc_nodetached. That
will let us avoid the situations where the xmin is left in invalid
state. Maybe like the attached (it also fixes a couple of
typos/thinkos in the previous commit).

Makes sense -- applied, thanks.

Thank you.

Note that we still end up in the same situation as before where each
query in the serializable transaction that sees the detach as
in-progress to have to rebuild the partition descriptor omitting the
detached partitions, even when it's clear that the detach-in-progress
partition will be included every time.

Yeah, you're right that there is a performance hole in the case where a
partition pending detach exists and you're using repeatable read
transactions. I didn't see it as terribly critical since it's supposed
to be very transient, but I may be wrong.

Yeah, I'd hope so too. I think RR transactions would have to be
concurrent with an interrupted DETACH CONCURRENTLY to suffer the
performance hit and that does kind of make this a rarely occurring
case.

--
Amit Langote
EDB: http://www.enterprisedb.com

#88Noah Misch
noah@leadboat.com
In reply to: Alvaro Herrera (#61)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Wed, Apr 21, 2021 at 04:38:55PM -0400, Alvaro Herrera wrote:

[fix to let CLOBBER_CACHE_ALWAYS pass]

Barring objections, I will get this pushed early tomorrow.

prairiedog and wrasse failed a $SUBJECT test after this (commit 8aba932).
Also, some non-CLOBBER_CACHE_ALWAYS animals failed a test before the fix.
These IsolationCheck failures match detach-partition-concurrently[^\n]*FAILED:

sysname │ snapshot │ branch │ bfurl
────────────┼─────────────────────┼────────┼────────────────────────────────────────────────────────────────────────────────────────────────
hyrax │ 2021-03-27 07:29:34 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-03-27%2007%3A29%3A34
topminnow │ 2021-03-28 20:37:38 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=topminnow&amp;dt=2021-03-28%2020%3A37%3A38
trilobite │ 2021-03-29 18:14:24 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=trilobite&amp;dt=2021-03-29%2018%3A14%3A24
hyrax │ 2021-04-01 07:21:10 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-01%2007%3A21%3A10
dragonet │ 2021-04-01 19:48:03 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=dragonet&amp;dt=2021-04-01%2019%3A48%3A03
avocet │ 2021-04-05 15:45:56 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=avocet&amp;dt=2021-04-05%2015%3A45%3A56
hyrax │ 2021-04-06 07:15:16 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-06%2007%3A15%3A16
hyrax │ 2021-04-11 07:25:50 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-11%2007%3A25%3A50
hyrax │ 2021-04-20 18:25:37 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-20%2018%3A25%3A37
wrasse │ 2021-04-21 10:38:32 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=wrasse&amp;dt=2021-04-21%2010%3A38%3A32
prairiedog │ 2021-04-25 22:05:48 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=prairiedog&amp;dt=2021-04-25%2022%3A05%3A48
wrasse │ 2021-05-11 03:43:40 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=wrasse&amp;dt=2021-05-11%2003%3A43%3A40
(12 rows)

#89Amit Langote
amitlangote09@gmail.com
In reply to: Noah Misch (#88)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On Mon, May 24, 2021 at 6:07 PM Noah Misch <noah@leadboat.com> wrote:

On Wed, Apr 21, 2021 at 04:38:55PM -0400, Alvaro Herrera wrote:

[fix to let CLOBBER_CACHE_ALWAYS pass]

Barring objections, I will get this pushed early tomorrow.

prairiedog and wrasse failed a $SUBJECT test after this (commit 8aba932).
Also, some non-CLOBBER_CACHE_ALWAYS animals failed a test before the fix.
These IsolationCheck failures match detach-partition-concurrently[^\n]*FAILED:

FWIW, all 4 detach-partition-concurrently suites passed for me on a
build of the latest HEAD, with CPPFLAGS = -DRELCACHE_FORCE_RELEASE
-DCATCACHE_FORCE_RELEASE -DCLOBBER_CACHE_ALWAYS -D_GNU_SOURCE

--
Amit Langote
EDB: http://www.enterprisedb.com

#90Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Noah Misch (#88)
Re: ALTER TABLE .. DETACH PARTITION CONCURRENTLY

On 2021-May-24, Noah Misch wrote:

prairiedog and wrasse failed a $SUBJECT test after this (commit 8aba932).
Also, some non-CLOBBER_CACHE_ALWAYS animals failed a test before the fix.
These IsolationCheck failures match detach-partition-concurrently[^\n]*FAILED:

sysname │ snapshot │ branch │ bfurl
────────────┼─────────────────────┼────────┼────────────────────────────────────────────────────────────────────────────────────────────────

Checking this list, these three failures can be explained by the
detach-partition-concurrently-3 that was just patched.

wrasse │ 2021-04-21 10:38:32 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=wrasse&amp;dt=2021-04-21%2010%3A38%3A32
prairiedog │ 2021-04-25 22:05:48 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=prairiedog&amp;dt=2021-04-25%2022%3A05%3A48
wrasse │ 2021-05-11 03:43:40 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=wrasse&amp;dt=2021-05-11%2003%3A43%3A40

Next there's a bunch whose error message is the same that we had seen
earlier in this thread; these animals are all CLOBBER_CACHE_ALWAYS:

step s1insert: insert into d4_fk values (1);
+ERROR: insert or update on table "d4_fk" violates foreign key constraint "d4_fk_a_fkey"

hyrax │ 2021-03-27 07:29:34 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-03-27%2007%3A29%3A34
trilobite │ 2021-03-29 18:14:24 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=trilobite&amp;dt=2021-03-29%2018%3A14%3A24
hyrax │ 2021-04-01 07:21:10 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-01%2007%3A21%3A10
avocet │ 2021-04-05 15:45:56 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=avocet&amp;dt=2021-04-05%2015%3A45%3A56
hyrax │ 2021-04-06 07:15:16 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-06%2007%3A15%3A16
hyrax │ 2021-04-11 07:25:50 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-11%2007%3A25%3A50
hyrax │ 2021-04-20 18:25:37 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hyrax&amp;dt=2021-04-20%2018%3A25%3A37

This is fine, because the fix commit 8aba932 is dated April 22 and these
failures all predate that.

And finally there's these two:

topminnow │ 2021-03-28 20:37:38 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=topminnow&amp;dt=2021-03-28%2020%3A37%3A38
dragonet │ 2021-04-01 19:48:03 │ HEAD │ http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=dragonet&amp;dt=2021-04-01%2019%3A48%3A03

(animals not CCA) which are exposing the same problem in
detach-partition-concurrently-4 that we just fixed in
detach-partition-concurrently-3, so we should apply the same fix: add a
no-op step right after the cancel to prevent the error report from
changing. I'll go do that after grabbing some coffee.

Thanks for digging into the reports!

--
Álvaro Herrera 39°49'30"S 73°17'W
"Cada quien es cada cual y baja las escaleras como quiere" (JMSerrat)