On Thu, Sep 24, 2020 at 05:11:03PM +0900, Michael Paquier wrote:
> start.  So, the goal of the patch, as I would define it, is to give a
> way to CLUSTER to work on a partitioned table using a given
> partitioned index.  Hence, we would perform CLUSTER automatically from
> the top-most parent for any partitions that have an index on the same
> partition tree as the partitioned index defined in USING, switching
> indisclustered automatically depending on the index used.

I think that's right, except there's no need to look for a compatible
partitioned index: we just use the child index.

Also, if a partitioned index is clustered, when we clear indisclustered for
other indexes, should we also propogate that to their parent indexes, if any ?

> From what I can see, attempting to use a CLUSTER on a top-most
> partitioned table fails to work on child tables,

Oops - it looks like this patch never worked right, due to the RECHECK on
indisclustered.  I think maybe I returned to the CIC patch and never finishing
with cluster.

> It would be good also to check if
> we have a partition index tree that maps partially with a partition
> table tree (aka no all table partitions have a partition index), where
> these don't get clustered because there is no index to work on.

This should not happen, since a incomplete partitioned index is "invalid".

> Isn't NoLock unsafe here, even knowing that an exclusive lock is taken on
> the parent?  It seems to me that at least schema changes should be
> prevented with a ShareLock in the first transaction building the list
> of elements to cluster.

Thanks for noticing.  I chose ShareUpdateExclusiveLock since it's
set-exclusive.

-- 
Justin
>From e4fe1e422dc2c0ee21ce9df15d5baf24e825bf9d Mon Sep 17 00:00:00 2001
From: Justin Pryzby <pryz...@telsasoft.com>
Date: Sat, 6 Jun 2020 17:42:23 -0500
Subject: [PATCH v8 1/3] Allow CREATE INDEX CONCURRENTLY on partitioned table

Note, this effectively reverts 050098b14, so take care to not reintroduce the
bug it fixed.

XXX: does pgstat_progress_update_param() break other commands progress ?
---
 doc/src/sgml/ref/create_index.sgml     |   9 --
 src/backend/commands/indexcmds.c       | 135 +++++++++++++++++--------
 src/test/regress/expected/indexing.out |  60 ++++++++++-
 src/test/regress/sql/indexing.sql      |  18 +++-
 4 files changed, 166 insertions(+), 56 deletions(-)

diff --git a/doc/src/sgml/ref/create_index.sgml b/doc/src/sgml/ref/create_index.sgml
index 33aa64e81d..c780dc9547 100644
--- a/doc/src/sgml/ref/create_index.sgml
+++ b/doc/src/sgml/ref/create_index.sgml
@@ -657,15 +657,6 @@ Indexes:
     cannot.
    </para>
 
-   <para>
-    Concurrent builds for indexes on partitioned tables are currently not
-    supported.  However, you may concurrently build the index on each
-    partition individually and then finally create the partitioned index
-    non-concurrently in order to reduce the time where writes to the
-    partitioned table will be locked out.  In this case, building the
-    partitioned index is a metadata only operation.
-   </para>
-
   </refsect2>
  </refsect1>
 
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index f1b5f87e6a..d417404211 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -91,6 +91,7 @@ static bool ReindexRelationConcurrently(Oid relationOid, int options);
 
 static void ReindexPartitions(Oid relid, int options, bool isTopLevel);
 static void ReindexMultipleInternal(List *relids, int options);
+static void reindex_invalid_child_indexes(Oid indexRelationId);
 static void reindex_error_callback(void *args);
 static void update_relispartition(Oid relationId, bool newval);
 static bool CompareOpclassOptions(Datum *opts1, Datum *opts2, int natts);
@@ -665,17 +666,6 @@ DefineIndex(Oid relationId,
 	partitioned = rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE;
 	if (partitioned)
 	{
-		/*
-		 * Note: we check 'stmt->concurrent' rather than 'concurrent', so that
-		 * the error is thrown also for temporary tables.  Seems better to be
-		 * consistent, even though we could do it on temporary table because
-		 * we're not actually doing it concurrently.
-		 */
-		if (stmt->concurrent)
-			ereport(ERROR,
-					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-					 errmsg("cannot create index on partitioned table \"%s\" concurrently",
-							RelationGetRelationName(rel))));
 		if (stmt->excludeOpNames)
 			ereport(ERROR,
 					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
@@ -1110,6 +1100,11 @@ DefineIndex(Oid relationId,
 		if (pd->nparts != 0)
 			flags |= INDEX_CREATE_INVALID;
 	}
+	else if (concurrent && OidIsValid(parentIndexId))
+	{
+		/* If concurrent, initially build index partitions as "invalid" */
+		flags |= INDEX_CREATE_INVALID;
+	}
 
 	if (stmt->deferrable)
 		constr_flags |= INDEX_CONSTR_CREATE_DEFERRABLE;
@@ -1162,6 +1157,14 @@ DefineIndex(Oid relationId,
 		 */
 		if (!stmt->relation || stmt->relation->inh)
 		{
+			/*
+			 * Need to close the relation before recursing into children, so
+			 * copy needed data into a longlived context.
+			 */
+
+			MemoryContext	ind_context = AllocSetContextCreate(PortalContext, "CREATE INDEX",
+					ALLOCSET_DEFAULT_SIZES);
+			MemoryContext	oldcontext = MemoryContextSwitchTo(ind_context);
 			PartitionDesc partdesc = RelationGetPartitionDesc(rel);
 			int			nparts = partdesc->nparts;
 			Oid		   *part_oids = palloc(sizeof(Oid) * nparts);
@@ -1169,12 +1172,15 @@ DefineIndex(Oid relationId,
 			TupleDesc	parentDesc;
 			Oid		   *opfamOids;
 
+			// If concurrent, maybe this should be done after excluding indexes which already exist ?
 			pgstat_progress_update_param(PROGRESS_CREATEIDX_PARTITIONS_TOTAL,
 										 nparts);
 
 			memcpy(part_oids, partdesc->oids, sizeof(Oid) * nparts);
+			parentDesc = CreateTupleDescCopy(RelationGetDescr(rel));
+			table_close(rel, NoLock);
+			MemoryContextSwitchTo(oldcontext);
 
-			parentDesc = RelationGetDescr(rel);
 			opfamOids = palloc(sizeof(Oid) * numberOfKeyAttributes);
 			for (i = 0; i < numberOfKeyAttributes; i++)
 				opfamOids[i] = get_opclass_family(classObjectId[i]);
@@ -1217,10 +1223,12 @@ DefineIndex(Oid relationId,
 					continue;
 				}
 
+				oldcontext = MemoryContextSwitchTo(ind_context);
 				childidxs = RelationGetIndexList(childrel);
 				attmap =
 					build_attrmap_by_name(RelationGetDescr(childrel),
 										  parentDesc);
+				MemoryContextSwitchTo(oldcontext);
 
 				foreach(cell, childidxs)
 				{
@@ -1291,10 +1299,14 @@ DefineIndex(Oid relationId,
 				 */
 				if (!found)
 				{
-					IndexStmt  *childStmt = copyObject(stmt);
+					IndexStmt  *childStmt;
 					bool		found_whole_row;
 					ListCell   *lc;
 
+					oldcontext = MemoryContextSwitchTo(ind_context);
+					childStmt = copyObject(stmt);
+					MemoryContextSwitchTo(oldcontext);
+
 					/*
 					 * We can't use the same index name for the child index,
 					 * so clear idxname to let the recursive invocation choose
@@ -1346,10 +1358,18 @@ DefineIndex(Oid relationId,
 								createdConstraintId,
 								is_alter_table, check_rights, check_not_in_use,
 								skip_build, quiet);
+					if (concurrent)
+					{
+						PopActiveSnapshot();
+						PushActiveSnapshot(GetTransactionSnapshot());
+						invalidate_parent = true;
+					}
 				}
 
-				pgstat_progress_update_param(PROGRESS_CREATEIDX_PARTITIONS_DONE,
-											 i + 1);
+				/* For concurrent build, this is a catalog-only stage */
+				if (!concurrent)
+					pgstat_progress_update_param(PROGRESS_CREATEIDX_PARTITIONS_DONE,
+												 i + 1);
 				free_attrmap(attmap);
 			}
 
@@ -1359,51 +1379,42 @@ DefineIndex(Oid relationId,
 			 * invalid, this is incorrect, so update our row to invalid too.
 			 */
 			if (invalidate_parent)
-			{
-				Relation	pg_index = table_open(IndexRelationId, RowExclusiveLock);
-				HeapTuple	tup,
-							newtup;
-
-				tup = SearchSysCache1(INDEXRELID,
-									  ObjectIdGetDatum(indexRelationId));
-				if (!HeapTupleIsValid(tup))
-					elog(ERROR, "cache lookup failed for index %u",
-						 indexRelationId);
-				newtup = heap_copytuple(tup);
-				((Form_pg_index) GETSTRUCT(newtup))->indisvalid = false;
-				CatalogTupleUpdate(pg_index, &tup->t_self, newtup);
-				ReleaseSysCache(tup);
-				table_close(pg_index, RowExclusiveLock);
-				heap_freetuple(newtup);
-			}
-		}
+				index_set_state_flags(indexRelationId, INDEX_DROP_CLEAR_VALID);
+		} else
+			table_close(rel, NoLock);
 
 		/*
 		 * Indexes on partitioned tables are not themselves built, so we're
 		 * done here.
 		 */
-		table_close(rel, NoLock);
 		if (!OidIsValid(parentIndexId))
+		{
+			if (concurrent)
+				reindex_invalid_child_indexes(indexRelationId);
+
 			pgstat_progress_end_command();
+		}
+
 		return address;
 	}
 
-	if (!concurrent)
+	table_close(rel, NoLock);
+	if (!concurrent || OidIsValid(parentIndexId))
 	{
-		/* Close the heap and we're done, in the non-concurrent case */
-		table_close(rel, NoLock);
+		/*
+		 * We're done if this is the top-level index,
+		 * or the catalog-only phase of a partition built concurrently
+		 */
 
-		/* If this is the top-level index, we're done. */
 		if (!OidIsValid(parentIndexId))
 			pgstat_progress_end_command();
 
 		return address;
 	}
 
-	/* save lockrelid and locktag for below, then close rel */
+	/* save lockrelid and locktag for below */
 	heaprelid = rel->rd_lockInfo.lockRelId;
 	SET_LOCKTAG_RELATION(heaplocktag, heaprelid.dbId, heaprelid.relId);
-	table_close(rel, NoLock);
 
 	/*
 	 * For a concurrent build, it's important to make the catalog entries
@@ -1585,6 +1596,50 @@ DefineIndex(Oid relationId,
 	return address;
 }
 
+/* Reindex invalid child indexes created earlier */
+static void
+reindex_invalid_child_indexes(Oid indexRelationId)
+{
+	ListCell *lc;
+	int		npart = 0;
+
+	MemoryContext	ind_context = AllocSetContextCreate(PortalContext, "CREATE INDEX",
+			ALLOCSET_DEFAULT_SIZES);
+	MemoryContext	oldcontext = MemoryContextSwitchTo(ind_context);
+	List		*childs = find_inheritance_children(indexRelationId, NoLock);
+	MemoryContextSwitchTo(oldcontext);
+
+	PopActiveSnapshot();
+
+	foreach (lc, childs)
+	{
+		Oid			indexrelid = lfirst_oid(lc);
+
+		pgstat_progress_update_param(PROGRESS_CREATEIDX_PARTITIONS_DONE,
+									 npart++);
+
+		CommandCounterIncrement();
+		PushActiveSnapshot(GetTransactionSnapshot());
+
+		if (get_index_isvalid(indexrelid) ||
+				get_rel_relkind(indexrelid) == RELKIND_PARTITIONED_INDEX)
+		{
+			PopActiveSnapshot();
+			continue;
+		}
+
+		ReindexRelationConcurrently(indexrelid, 0);
+	}
+
+	/*
+	 * CIC needs to mark a partitioned index as VALID, which itself
+	 * requires setting READY, which is unset for CIC (even though
+	 * it's meaningless for an index without storage).
+	 */
+	index_set_state_flags(indexRelationId, INDEX_CREATE_SET_READY);
+	CommandCounterIncrement();
+	index_set_state_flags(indexRelationId, INDEX_CREATE_SET_VALID);
+}
 
 /*
  * CheckMutability
diff --git a/src/test/regress/expected/indexing.out b/src/test/regress/expected/indexing.out
index 7e78a07af8..c6d7cfd446 100644
--- a/src/test/regress/expected/indexing.out
+++ b/src/test/regress/expected/indexing.out
@@ -50,11 +50,63 @@ select relname, relkind, relhassubclass, inhparent::regclass
 (8 rows)
 
 drop table idxpart;
--- Some unsupported features
+-- CIC on partitioned table
 create table idxpart (a int, b int, c text) partition by range (a);
-create table idxpart1 partition of idxpart for values from (0) to (10);
-create index concurrently on idxpart (a);
-ERROR:  cannot create index on partitioned table "idxpart" concurrently
+create table idxpart1 partition of idxpart for values from (0) to (10) partition by range(a);
+create table idxpart11 partition of idxpart1 for values from (0) to (10) partition by range(a);
+create table idxpart111 partition of idxpart11 default partition by range(a);
+create table idxpart1111 partition of idxpart111 default partition by range(a);
+create table idxpart2 partition of idxpart for values from (10) to (20);
+insert into idxpart2 values(10),(10); -- not unique
+create index concurrently on idxpart (a); -- partitioned
+create index concurrently on idxpart1 (a); -- partitioned and partition
+create index concurrently on idxpart11 (a); -- partitioned and partition, with no leaves
+create index concurrently on idxpart2 (a); -- leaf
+create unique index concurrently on idxpart (a); -- partitioned, unique failure
+ERROR:  could not create unique index "idxpart2_a_idx2_ccnew"
+DETAIL:  Key (a)=(10) is duplicated.
+\d idxpart
+        Partitioned table "public.idxpart"
+ Column |  Type   | Collation | Nullable | Default 
+--------+---------+-----------+----------+---------
+ a      | integer |           |          | 
+ b      | integer |           |          | 
+ c      | text    |           |          | 
+Partition key: RANGE (a)
+Indexes:
+    "idxpart_a_idx" btree (a)
+    "idxpart_a_idx1" UNIQUE, btree (a) INVALID
+Number of partitions: 2 (Use \d+ to list them.)
+
+\d idxpart1
+        Partitioned table "public.idxpart1"
+ Column |  Type   | Collation | Nullable | Default 
+--------+---------+-----------+----------+---------
+ a      | integer |           |          | 
+ b      | integer |           |          | 
+ c      | text    |           |          | 
+Partition of: idxpart FOR VALUES FROM (0) TO (10)
+Partition key: RANGE (a)
+Indexes:
+    "idxpart1_a_idx" btree (a) INVALID
+    "idxpart1_a_idx1" btree (a)
+    "idxpart1_a_idx2" UNIQUE, btree (a) INVALID
+Number of partitions: 1 (Use \d+ to list them.)
+
+\d idxpart2
+              Table "public.idxpart2"
+ Column |  Type   | Collation | Nullable | Default 
+--------+---------+-----------+----------+---------
+ a      | integer |           |          | 
+ b      | integer |           |          | 
+ c      | text    |           |          | 
+Partition of: idxpart FOR VALUES FROM (10) TO (20)
+Indexes:
+    "idxpart2_a_idx" btree (a)
+    "idxpart2_a_idx1" btree (a)
+    "idxpart2_a_idx2" UNIQUE, btree (a) INVALID
+    "idxpart2_a_idx2_ccnew" UNIQUE, btree (a) INVALID
+
 drop table idxpart;
 -- Verify bugfix with query on indexed partitioned table with no partitions
 -- https://postgr.es/m/20180124162006.pmapfiznhgngwtjf@alvherre.pgsql
diff --git a/src/test/regress/sql/indexing.sql b/src/test/regress/sql/indexing.sql
index 42f398b67c..3d4b6e9bc9 100644
--- a/src/test/regress/sql/indexing.sql
+++ b/src/test/regress/sql/indexing.sql
@@ -29,10 +29,22 @@ select relname, relkind, relhassubclass, inhparent::regclass
 	where relname like 'idxpart%' order by relname;
 drop table idxpart;
 
--- Some unsupported features
+-- CIC on partitioned table
 create table idxpart (a int, b int, c text) partition by range (a);
-create table idxpart1 partition of idxpart for values from (0) to (10);
-create index concurrently on idxpart (a);
+create table idxpart1 partition of idxpart for values from (0) to (10) partition by range(a);
+create table idxpart11 partition of idxpart1 for values from (0) to (10) partition by range(a);
+create table idxpart111 partition of idxpart11 default partition by range(a);
+create table idxpart1111 partition of idxpart111 default partition by range(a);
+create table idxpart2 partition of idxpart for values from (10) to (20);
+insert into idxpart2 values(10),(10); -- not unique
+create index concurrently on idxpart (a); -- partitioned
+create index concurrently on idxpart1 (a); -- partitioned and partition
+create index concurrently on idxpart11 (a); -- partitioned and partition, with no leaves
+create index concurrently on idxpart2 (a); -- leaf
+create unique index concurrently on idxpart (a); -- partitioned, unique failure
+\d idxpart
+\d idxpart1
+\d idxpart2
 drop table idxpart;
 
 -- Verify bugfix with query on indexed partitioned table with no partitions
-- 
2.17.0

>From 2648911e8314025ba1b4f6822f76d42ae7445825 Mon Sep 17 00:00:00 2001
From: Justin Pryzby <pryz...@telsasoft.com>
Date: Sun, 7 Jun 2020 16:58:42 -0500
Subject: [PATCH v8 2/3] Implement CLUSTER of partitioned table..

This requires either specification of a partitioned index on which to cluster,
or that an partitioned index was previously set clustered.
---
 doc/src/sgml/ref/cluster.sgml         |   6 +
 src/backend/commands/cluster.c        | 230 +++++++++++++++++++-------
 src/bin/psql/tab-complete.c           |   1 +
 src/include/nodes/parsenodes.h        |   5 +-
 src/test/regress/expected/cluster.out |  70 +++++++-
 src/test/regress/sql/cluster.sql      |  27 ++-
 6 files changed, 268 insertions(+), 71 deletions(-)

diff --git a/doc/src/sgml/ref/cluster.sgml b/doc/src/sgml/ref/cluster.sgml
index 4da60d8d56..2df8aaabc4 100644
--- a/doc/src/sgml/ref/cluster.sgml
+++ b/doc/src/sgml/ref/cluster.sgml
@@ -172,6 +172,12 @@ CLUSTER [VERBOSE]
     are periodically reclustered.
    </para>
 
+   <para>
+    Clustering a partitioned table clusters each of its partitions using the
+    index partition of the given partitioned index or (if not specified) the
+    partitioned index marked as clustered.
+   </para>
+
  </refsect1>
 
  <refsect1>
diff --git a/src/backend/commands/cluster.c b/src/backend/commands/cluster.c
index 0d647e912c..d2e3ebc8af 100644
--- a/src/backend/commands/cluster.c
+++ b/src/backend/commands/cluster.c
@@ -32,7 +32,9 @@
 #include "catalog/index.h"
 #include "catalog/namespace.h"
 #include "catalog/objectaccess.h"
+#include "catalog/partition.h"
 #include "catalog/pg_am.h"
+#include "catalog/pg_inherits.h"
 #include "catalog/toasting.h"
 #include "commands/cluster.h"
 #include "commands/progress.h"
@@ -74,7 +76,11 @@ static void copy_table_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex,
 							bool *pSwapToastByContent,
 							TransactionId *pFreezeXid,
 							MultiXactId *pCutoffMulti);
+static void set_indisclustered(Oid indexOid, bool isclustered, Relation pg_index);
 static List *get_tables_to_cluster(MemoryContext cluster_context);
+static List *get_tables_to_cluster_partitioned(MemoryContext cluster_context,
+		Oid indexOid);
+static void cluster_multiple_rels(List *rvs, bool isTopLevel, int options);
 
 
 /*---------------------------------------------------------------------------
@@ -116,7 +122,7 @@ cluster(ClusterStmt *stmt, bool isTopLevel)
 											AccessExclusiveLock,
 											0,
 											RangeVarCallbackOwnsTable, NULL);
-		rel = table_open(tableOid, NoLock);
+		rel = table_open(tableOid, ShareUpdateExclusiveLock);
 
 		/*
 		 * Reject clustering a remote temp table ... their local buffer
@@ -127,14 +133,6 @@ cluster(ClusterStmt *stmt, bool isTopLevel)
 					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 					 errmsg("cannot cluster temporary tables of other sessions")));
 
-		/*
-		 * Reject clustering a partitioned table.
-		 */
-		if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
-			ereport(ERROR,
-					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-					 errmsg("cannot cluster a partitioned table")));
-
 		if (stmt->indexname == NULL)
 		{
 			ListCell   *index;
@@ -172,8 +170,32 @@ cluster(ClusterStmt *stmt, bool isTopLevel)
 		/* close relation, keep lock till commit */
 		table_close(rel, NoLock);
 
-		/* Do the job. */
-		cluster_rel(tableOid, indexOid, stmt->options, isTopLevel);
+		if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		{
+			/* Do the job. */
+			cluster_rel(tableOid, indexOid, stmt->options, isTopLevel);
+		}
+		else
+		{
+			List	   *rvs;
+			MemoryContext cluster_context;
+
+			/* Refuse to hold strong locks in a user transaction */
+			PreventInTransactionBlock(isTopLevel, "CLUSTER");
+
+			cluster_context = AllocSetContextCreate(PortalContext,
+												"Cluster",
+												ALLOCSET_DEFAULT_SIZES);
+
+			rvs = get_tables_to_cluster_partitioned(cluster_context, indexOid);
+			cluster_multiple_rels(rvs, isTopLevel, stmt->options);
+
+			/* Start a new transaction for the cleanup work. */
+			StartTransactionCommand();
+
+			/* Clean up working storage */
+			MemoryContextDelete(cluster_context);
+		}
 	}
 	else
 	{
@@ -183,7 +205,6 @@ cluster(ClusterStmt *stmt, bool isTopLevel)
 		 */
 		MemoryContext cluster_context;
 		List	   *rvs;
-		ListCell   *rv;
 
 		/*
 		 * We cannot run this form of CLUSTER inside a user transaction block;
@@ -207,26 +228,7 @@ cluster(ClusterStmt *stmt, bool isTopLevel)
 		 */
 		rvs = get_tables_to_cluster(cluster_context);
 
-		/* Commit to get out of starting transaction */
-		PopActiveSnapshot();
-		CommitTransactionCommand();
-
-		/* Ok, now that we've got them all, cluster them one by one */
-		foreach(rv, rvs)
-		{
-			RelToCluster *rvtc = (RelToCluster *) lfirst(rv);
-
-			/* Start a new transaction for each relation. */
-			StartTransactionCommand();
-			/* functions in indexes may want a snapshot set */
-			PushActiveSnapshot(GetTransactionSnapshot());
-			/* Do the job. */
-			cluster_rel(rvtc->tableOid, rvtc->indexOid,
-						stmt->options | CLUOPT_RECHECK,
-						isTopLevel);
-			PopActiveSnapshot();
-			CommitTransactionCommand();
-		}
+		cluster_multiple_rels(rvs, isTopLevel, stmt->options | CLUOPT_RECHECK_ISCLUSTERED);
 
 		/* Start a new transaction for the cleanup work. */
 		StartTransactionCommand();
@@ -332,9 +334,10 @@ cluster_rel(Oid tableOid, Oid indexOid, int options, bool isTopLevel)
 			}
 
 			/*
-			 * Check that the index is still the one with indisclustered set.
+			 * Check that the index is still the one with indisclustered set, if needed.
 			 */
-			if (!get_index_isclustered(indexOid))
+			if ((options & CLUOPT_RECHECK_ISCLUSTERED) != 0 &&
+					!get_index_isclustered(indexOid))
 			{
 				relation_close(OldHeap, AccessExclusiveLock);
 				pgstat_progress_end_command();
@@ -378,8 +381,13 @@ cluster_rel(Oid tableOid, Oid indexOid, int options, bool isTopLevel)
 
 	/* Check heap and index are valid to cluster on */
 	if (OidIsValid(indexOid))
+	{
 		check_index_is_clusterable(OldHeap, indexOid, recheck, AccessExclusiveLock);
 
+		/* Mark the index as clustered */
+		mark_index_clustered(OldHeap, indexOid, true);
+	}
+
 	/*
 	 * Quietly ignore the request if this is a materialized view which has not
 	 * been populated from its query. No harm is done because there is no data
@@ -395,6 +403,14 @@ cluster_rel(Oid tableOid, Oid indexOid, int options, bool isTopLevel)
 		return;
 	}
 
+	/* For a partitioned rel, we're done. */
+	if (!RELKIND_HAS_STORAGE(get_rel_relkind(tableOid)))
+	{
+		relation_close(OldHeap, AccessExclusiveLock);
+		pgstat_progress_end_command();
+		return;
+	}
+
 	/*
 	 * All predicate locks on the tuples or pages are about to be made
 	 * invalid, because we move tuples around.  Promote them to relation
@@ -463,6 +479,9 @@ check_index_is_clusterable(Relation OldHeap, Oid indexOid, bool recheck, LOCKMOD
 	 * the worst consequence of following broken HOT chains would be that we
 	 * might put recently-dead tuples out-of-order in the new table, and there
 	 * is little harm in that.)
+	 *
+	 * This also refuses to cluster on an "incomplete" partitioned index
+	 * created with "ON ONLY".
 	 */
 	if (!OldIndex->rd_index->indisvalid)
 		ereport(ERROR,
@@ -474,6 +493,32 @@ check_index_is_clusterable(Relation OldHeap, Oid indexOid, bool recheck, LOCKMOD
 	index_close(OldIndex, NoLock);
 }
 
+/*
+ * Helper for mark_index_clustered
+ * Mark a single index as clustered or not.
+ * pg_index is passed by caller to avoid repeatedly re-opening it.
+ */
+static void
+set_indisclustered(Oid indexOid, bool isclustered, Relation pg_index)
+{
+	HeapTuple	indexTuple;
+	Form_pg_index indexForm;
+
+	indexTuple = SearchSysCacheCopy1(INDEXRELID,
+									ObjectIdGetDatum(indexOid));
+	if (!HeapTupleIsValid(indexTuple))
+		elog(ERROR, "cache lookup failed for index %u", indexOid);
+	indexForm = (Form_pg_index) GETSTRUCT(indexTuple);
+
+	/* this was checked earlier, but let's be real sure */
+	if (isclustered && !indexForm->indisvalid)
+		elog(ERROR, "cannot cluster on invalid index %u", indexOid);
+
+	indexForm->indisclustered = isclustered;
+	CatalogTupleUpdate(pg_index, &indexTuple->t_self, indexTuple);
+	heap_freetuple(indexTuple);
+}
+
 /*
  * mark_index_clustered: mark the specified index as the one clustered on
  *
@@ -482,17 +527,9 @@ check_index_is_clusterable(Relation OldHeap, Oid indexOid, bool recheck, LOCKMOD
 void
 mark_index_clustered(Relation rel, Oid indexOid, bool is_internal)
 {
-	HeapTuple	indexTuple;
-	Form_pg_index indexForm;
 	Relation	pg_index;
 	ListCell   *index;
 
-	/* Disallow applying to a partitioned table */
-	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
-		ereport(ERROR,
-				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-				 errmsg("cannot mark index clustered in partitioned table")));
-
 	/*
 	 * If the index is already marked clustered, no need to do anything.
 	 */
@@ -511,34 +548,32 @@ mark_index_clustered(Relation rel, Oid indexOid, bool is_internal)
 	{
 		Oid			thisIndexOid = lfirst_oid(index);
 
-		indexTuple = SearchSysCacheCopy1(INDEXRELID,
-										 ObjectIdGetDatum(thisIndexOid));
-		if (!HeapTupleIsValid(indexTuple))
-			elog(ERROR, "cache lookup failed for index %u", thisIndexOid);
-		indexForm = (Form_pg_index) GETSTRUCT(indexTuple);
-
 		/*
 		 * Unset the bit if set.  We know it's wrong because we checked this
 		 * earlier.
 		 */
-		if (indexForm->indisclustered)
+		if (thisIndexOid != indexOid)
 		{
-			indexForm->indisclustered = false;
-			CatalogTupleUpdate(pg_index, &indexTuple->t_self, indexTuple);
+			Oid parentind = thisIndexOid;
+			set_indisclustered(thisIndexOid, false, pg_index);
+
+			/*
+			 * When setting a given index as clustered, also remove
+			 * indisclustered from all parents of other partitioned indexes
+			 */
+			while (get_rel_relispartition(parentind))
+			{
+				parentind = get_partition_parent(parentind);
+				set_indisclustered(parentind, false, pg_index);
+			}
 		}
-		else if (thisIndexOid == indexOid)
+		else
 		{
-			/* this was checked earlier, but let's be real sure */
-			if (!indexForm->indisvalid)
-				elog(ERROR, "cannot cluster on invalid index %u", indexOid);
-			indexForm->indisclustered = true;
-			CatalogTupleUpdate(pg_index, &indexTuple->t_self, indexTuple);
+			set_indisclustered(thisIndexOid, true, pg_index);
 		}
 
 		InvokeObjectPostAlterHookArg(IndexRelationId, thisIndexOid, 0,
 									 InvalidOid, is_internal);
-
-		heap_freetuple(indexTuple);
 	}
 
 	table_close(pg_index, RowExclusiveLock);
@@ -565,10 +600,6 @@ rebuild_relation(Relation OldHeap, Oid indexOid, bool isTopLevel, bool verbose)
 	TransactionId frozenXid;
 	MultiXactId cutoffMulti;
 
-	/* Mark the correct index as clustered */
-	if (OidIsValid(indexOid))
-		mark_index_clustered(OldHeap, indexOid, true);
-
 	/* Remember info about rel before closing OldHeap */
 	relpersistence = OldHeap->rd_rel->relpersistence;
 	is_system_catalog = IsSystemRelation(OldHeap);
@@ -1563,3 +1594,76 @@ get_tables_to_cluster(MemoryContext cluster_context)
 
 	return rvs;
 }
+
+/*
+ * Return a List of tables and associated index, where each index is a
+ * partition of the given index
+ */
+static List *
+get_tables_to_cluster_partitioned(MemoryContext cluster_context, Oid indexOid)
+{
+	List		*inhoids;
+	ListCell	*lc;
+	List		*rvs = NIL;
+	MemoryContext	old_context;
+
+	inhoids = find_all_inheritors(indexOid, NoLock, NULL);
+
+	foreach(lc, inhoids)
+	{
+		Oid		indexrelid = lfirst_oid(lc);
+		Oid		relid = IndexGetRelation(indexrelid, false);
+		RelToCluster	*rvtc;
+
+		/*
+		 * Partitioned rels are also processed by cluster_rel, to
+		 * call check_index_is_clusterable() and mark_index_clustered().
+		 */
+
+		/*
+		 * We have to build the list in a different memory context so it will
+		 * survive the cross-transaction processing
+		 */
+		old_context = MemoryContextSwitchTo(cluster_context);
+
+		rvtc = (RelToCluster *) palloc(sizeof(RelToCluster));
+		rvtc->tableOid = relid;
+		rvtc->indexOid = indexrelid;
+		rvs = lappend(rvs, rvtc);
+
+		MemoryContextSwitchTo(old_context);
+	}
+
+	return rvs;
+}
+
+/* Cluster each relation in a separate transaction */
+static void
+cluster_multiple_rels(List *rvs, bool isTopLevel, int options)
+{
+	ListCell *lc;
+
+	/* Commit to get out of starting transaction */
+	PopActiveSnapshot();
+	CommitTransactionCommand();
+
+	/* Ok, now that we've got them all, cluster them one by one */
+	foreach(lc, rvs)
+	{
+		RelToCluster *rvtc = (RelToCluster *) lfirst(lc);
+
+		/* Start a new transaction for each relation. */
+		StartTransactionCommand();
+
+		/* functions in indexes may want a snapshot set */
+		PushActiveSnapshot(GetTransactionSnapshot());
+
+		/* Do the job. */
+		cluster_rel(rvtc->tableOid, rvtc->indexOid,
+					options | CLUOPT_RECHECK,
+					isTopLevel);
+
+		PopActiveSnapshot();
+		CommitTransactionCommand();
+	}
+}
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9c6f5ecb6a..ad0cd2ec92 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -587,6 +587,7 @@ static const SchemaQuery Query_for_list_of_clusterables = {
 	.catname = "pg_catalog.pg_class c",
 	.selcondition =
 	"c.relkind IN (" CppAsString2(RELKIND_RELATION) ", "
+	CppAsString2(RELKIND_PARTITIONED_TABLE) ", "
 	CppAsString2(RELKIND_MATVIEW) ")",
 	.viscondition = "pg_catalog.pg_table_is_visible(c.oid)",
 	.namespace = "c.relnamespace",
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 60c2f45466..d428a94454 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -3202,8 +3202,9 @@ typedef struct AlterSystemStmt
  */
 typedef enum ClusterOption
 {
-	CLUOPT_RECHECK = 1 << 0,	/* recheck relation state */
-	CLUOPT_VERBOSE = 1 << 1		/* print progress info */
+	CLUOPT_VERBOSE = 1 << 0,	/* print progress info */
+	CLUOPT_RECHECK = 1 << 1,	/* recheck relation state */
+	CLUOPT_RECHECK_ISCLUSTERED = 1 << 2,	/* recheck relation state for indisclustered */
 } ClusterOption;
 
 typedef struct ClusterStmt
diff --git a/src/test/regress/expected/cluster.out b/src/test/regress/expected/cluster.out
index bdae8fe00c..60d2a36255 100644
--- a/src/test/regress/expected/cluster.out
+++ b/src/test/regress/expected/cluster.out
@@ -439,13 +439,75 @@ select * from clstr_temp;
 
 drop table clstr_temp;
 RESET SESSION AUTHORIZATION;
--- Check that partitioned tables cannot be clustered
+-- Check that partitioned tables can be clustered
 CREATE TABLE clstrpart (a int) PARTITION BY RANGE (a);
+CREATE TABLE clstrpart1 PARTITION OF clstrpart FOR VALUES FROM (1)TO(10) PARTITION BY RANGE (a);
+CREATE TABLE clstrpart11 PARTITION OF clstrpart1 FOR VALUES FROM (1)TO(10);
+CREATE TABLE clstrpart12 PARTITION OF clstrpart1 FOR VALUES FROM (10)TO(20) PARTITION BY RANGE(a);
+CREATE TABLE clstrpart2 PARTITION OF clstrpart FOR VALUES FROM (20)TO(30);
+CREATE TABLE clstrpart3 PARTITION OF clstrpart DEFAULT PARTITION BY RANGE(a);
+CREATE TABLE clstrpart33 PARTITION OF clstrpart3 DEFAULT;
+ALTER TABLE clstrpart SET WITHOUT CLUSTER;
+CREATE INDEX clstrpart_only_idx ON ONLY clstrpart (a);
+CLUSTER clstrpart USING clstrpart_only_idx; -- fails
+ERROR:  cannot cluster on invalid index "clstrpart_only_idx"
+DROP INDEX clstrpart_only_idx;
 CREATE INDEX clstrpart_idx ON clstrpart (a);
-ALTER TABLE clstrpart CLUSTER ON clstrpart_idx;
-ERROR:  cannot mark index clustered in partitioned table
+-- Check that clustering sets new relfilenodes:
+CREATE TEMP TABLE old_cluster_info AS SELECT relname, relfilenode, relkind FROM pg_partition_tree('clstrpart'::regclass) AS tree JOIN pg_class c ON c.oid=tree.relid ;
 CLUSTER clstrpart USING clstrpart_idx;
-ERROR:  cannot cluster a partitioned table
+CREATE TEMP TABLE new_cluster_info AS SELECT relname, relfilenode, relkind FROM pg_partition_tree('clstrpart'::regclass) AS tree JOIN pg_class c ON c.oid=tree.relid ;
+SELECT relname, old.relkind, old.relfilenode = new.relfilenode FROM old_cluster_info AS old JOIN new_cluster_info AS new USING (relname) ORDER BY 1;
+   relname   | relkind | ?column? 
+-------------+---------+----------
+ clstrpart   | p       | t
+ clstrpart1  | p       | t
+ clstrpart11 | r       | f
+ clstrpart12 | p       | t
+ clstrpart2  | r       | f
+ clstrpart3  | p       | t
+ clstrpart33 | r       | f
+(7 rows)
+
+-- Check that clustering sets new indisclustered:
+SELECT i.indexrelid::regclass::text, relkind, indisclustered FROM pg_partition_tree('clstrpart_idx'::regclass) AS tree JOIN pg_index i ON i.indexrelid=tree.relid JOIN pg_class c ON c.oid=indexrelid ORDER BY 1;
+    indexrelid     | relkind | indisclustered 
+-------------------+---------+----------------
+ clstrpart11_a_idx | i       | t
+ clstrpart12_a_idx | I       | t
+ clstrpart1_a_idx  | I       | t
+ clstrpart2_a_idx  | i       | t
+ clstrpart33_a_idx | i       | t
+ clstrpart3_a_idx  | I       | t
+ clstrpart_idx     | I       | t
+(7 rows)
+
+CLUSTER clstrpart1 USING clstrpart1_a_idx; -- partition which is itself partitioned
+CLUSTER clstrpart12 USING clstrpart12_a_idx; -- partition which is itself partitioned, no childs
+CLUSTER clstrpart2 USING clstrpart2_a_idx; -- leaf
+\d clstrpart
+       Partitioned table "public.clstrpart"
+ Column |  Type   | Collation | Nullable | Default 
+--------+---------+-----------+----------+---------
+ a      | integer |           |          | 
+Partition key: RANGE (a)
+Indexes:
+    "clstrpart_idx" btree (a) CLUSTER
+Number of partitions: 3 (Use \d+ to list them.)
+
+-- Check that the parent index is marked not clustered after clustering a partition on a different index:
+CREATE INDEX clstrpart1_idx_2 ON clstrpart1(a);
+CLUSTER clstrpart1 USING clstrpart1_idx_2;
+\d clstrpart
+       Partitioned table "public.clstrpart"
+ Column |  Type   | Collation | Nullable | Default 
+--------+---------+-----------+----------+---------
+ a      | integer |           |          | 
+Partition key: RANGE (a)
+Indexes:
+    "clstrpart_idx" btree (a)
+Number of partitions: 3 (Use \d+ to list them.)
+
 DROP TABLE clstrpart;
 -- Test CLUSTER with external tuplesorting
 create table clstr_4 as select * from tenk1;
diff --git a/src/test/regress/sql/cluster.sql b/src/test/regress/sql/cluster.sql
index 188183647c..93e9c8c6ee 100644
--- a/src/test/regress/sql/cluster.sql
+++ b/src/test/regress/sql/cluster.sql
@@ -196,11 +196,34 @@ drop table clstr_temp;
 
 RESET SESSION AUTHORIZATION;
 
--- Check that partitioned tables cannot be clustered
+-- Check that partitioned tables can be clustered
 CREATE TABLE clstrpart (a int) PARTITION BY RANGE (a);
+CREATE TABLE clstrpart1 PARTITION OF clstrpart FOR VALUES FROM (1)TO(10) PARTITION BY RANGE (a);
+CREATE TABLE clstrpart11 PARTITION OF clstrpart1 FOR VALUES FROM (1)TO(10);
+CREATE TABLE clstrpart12 PARTITION OF clstrpart1 FOR VALUES FROM (10)TO(20) PARTITION BY RANGE(a);
+CREATE TABLE clstrpart2 PARTITION OF clstrpart FOR VALUES FROM (20)TO(30);
+CREATE TABLE clstrpart3 PARTITION OF clstrpart DEFAULT PARTITION BY RANGE(a);
+CREATE TABLE clstrpart33 PARTITION OF clstrpart3 DEFAULT;
+ALTER TABLE clstrpart SET WITHOUT CLUSTER;
+CREATE INDEX clstrpart_only_idx ON ONLY clstrpart (a);
+CLUSTER clstrpart USING clstrpart_only_idx; -- fails
+DROP INDEX clstrpart_only_idx;
 CREATE INDEX clstrpart_idx ON clstrpart (a);
-ALTER TABLE clstrpart CLUSTER ON clstrpart_idx;
+-- Check that clustering sets new relfilenodes:
+CREATE TEMP TABLE old_cluster_info AS SELECT relname, relfilenode, relkind FROM pg_partition_tree('clstrpart'::regclass) AS tree JOIN pg_class c ON c.oid=tree.relid ;
 CLUSTER clstrpart USING clstrpart_idx;
+CREATE TEMP TABLE new_cluster_info AS SELECT relname, relfilenode, relkind FROM pg_partition_tree('clstrpart'::regclass) AS tree JOIN pg_class c ON c.oid=tree.relid ;
+SELECT relname, old.relkind, old.relfilenode = new.relfilenode FROM old_cluster_info AS old JOIN new_cluster_info AS new USING (relname) ORDER BY 1;
+-- Check that clustering sets new indisclustered:
+SELECT i.indexrelid::regclass::text, relkind, indisclustered FROM pg_partition_tree('clstrpart_idx'::regclass) AS tree JOIN pg_index i ON i.indexrelid=tree.relid JOIN pg_class c ON c.oid=indexrelid ORDER BY 1;
+CLUSTER clstrpart1 USING clstrpart1_a_idx; -- partition which is itself partitioned
+CLUSTER clstrpart12 USING clstrpart12_a_idx; -- partition which is itself partitioned, no childs
+CLUSTER clstrpart2 USING clstrpart2_a_idx; -- leaf
+\d clstrpart
+-- Check that the parent index is marked not clustered after clustering a partition on a different index:
+CREATE INDEX clstrpart1_idx_2 ON clstrpart1(a);
+CLUSTER clstrpart1 USING clstrpart1_idx_2;
+\d clstrpart
 DROP TABLE clstrpart;
 
 -- Test CLUSTER with external tuplesorting
-- 
2.17.0

>From 98ec192a4de85a90c595be47a03834d1a36abb16 Mon Sep 17 00:00:00 2001
From: Justin Pryzby <pryz...@telsasoft.com>
Date: Tue, 8 Sep 2020 12:12:44 -0500
Subject: [PATCH v8 3/3] WIP: implement DROP INDEX CONCURRENTLY on partitioned
 index

See also
https://postgr.es/m/16594-d2956ca909585...@postgresql.org
---
 doc/src/sgml/ref/drop_index.sgml       |  2 --
 src/backend/catalog/dependency.c       | 24 ++++++++++++++++++++++++
 src/backend/catalog/index.c            | 19 +------------------
 src/backend/commands/tablecmds.c       | 11 -----------
 src/test/regress/expected/indexing.out |  4 +---
 src/test/regress/sql/indexing.sql      |  3 +--
 6 files changed, 27 insertions(+), 36 deletions(-)

diff --git a/doc/src/sgml/ref/drop_index.sgml b/doc/src/sgml/ref/drop_index.sgml
index 85cf23bca2..0aedd71bd6 100644
--- a/doc/src/sgml/ref/drop_index.sgml
+++ b/doc/src/sgml/ref/drop_index.sgml
@@ -57,8 +57,6 @@ DROP INDEX [ CONCURRENTLY ] [ IF EXISTS ] <replaceable class="parameter">name</r
       Also, regular <command>DROP INDEX</command> commands can be
       performed within a transaction block, but
       <command>DROP INDEX CONCURRENTLY</command> cannot.
-      Lastly, indexes on partitioned tables cannot be dropped using this
-      option.
      </para>
      <para>
       For temporary tables, <command>DROP INDEX</command> is always
diff --git a/src/backend/catalog/dependency.c b/src/backend/catalog/dependency.c
index f515e2c308..25ad74da2d 100644
--- a/src/backend/catalog/dependency.c
+++ b/src/backend/catalog/dependency.c
@@ -395,6 +395,30 @@ performMultipleDeletions(const ObjectAddresses *objects,
 	 */
 	targetObjects = new_object_addresses();
 
+	/*
+	 * We must commit our transaction in order to make the first pg_index
+	 * state update visible to other sessions.  If the DROP machinery has
+	 * already performed any other actions (removal of other objects,
+	 * pg_depend entries, etc), the commit would make those actions
+	 * permanent, which would leave us with inconsistent catalog state if
+	 * we fail partway through the following sequence.  Since DROP INDEX
+	 * CONCURRENTLY is restricted to dropping just one index that has no
+	 * dependencies, we should get here before anything's been done ---
+	 * but let's check that to be sure.  We can verify that the current
+	 * transaction has not executed any transactional updates by checking
+	 * that no XID has been assigned.
+	 */
+	if ((flags & PERFORM_DELETION_CONCURRENTLY) != 0 &&
+		objects->numrefs == 1 &&
+		objects->refs[0].classId == RelationRelationId &&
+		get_rel_relkind(objects->refs[0].objectId) == RELKIND_PARTITIONED_INDEX)
+	{
+		if (GetTopTransactionIdIfAny() != InvalidTransactionId)
+			ereport(ERROR,
+					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+					 errmsg("DROP INDEX CONCURRENTLY must be first action in transaction")));
+	}
+
 	for (i = 0; i < objects->numrefs; i++)
 	{
 		const ObjectAddress *thisobj = objects->refs + i;
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 0974f3e23a..5f250deeda 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2095,24 +2095,6 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 	 */
 	if (concurrent)
 	{
-		/*
-		 * We must commit our transaction in order to make the first pg_index
-		 * state update visible to other sessions.  If the DROP machinery has
-		 * already performed any other actions (removal of other objects,
-		 * pg_depend entries, etc), the commit would make those actions
-		 * permanent, which would leave us with inconsistent catalog state if
-		 * we fail partway through the following sequence.  Since DROP INDEX
-		 * CONCURRENTLY is restricted to dropping just one index that has no
-		 * dependencies, we should get here before anything's been done ---
-		 * but let's check that to be sure.  We can verify that the current
-		 * transaction has not executed any transactional updates by checking
-		 * that no XID has been assigned.
-		 */
-		if (GetTopTransactionIdIfAny() != InvalidTransactionId)
-			ereport(ERROR,
-					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-					 errmsg("DROP INDEX CONCURRENTLY must be first action in transaction")));
-
 		/*
 		 * Mark index invalid by updating its pg_index entry
 		 */
@@ -2180,6 +2162,7 @@ index_drop(Oid indexId, bool concurrent, bool concurrent_lock_mode)
 		 */
 		CommitTransactionCommand();
 		StartTransactionCommand();
+		PushActiveSnapshot(GetTransactionSnapshot());
 
 		/*
 		 * Wait till every transaction that saw the old index state has
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 16285ad09f..83092f8f6b 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -1374,17 +1374,6 @@ RemoveRelations(DropStmt *drop)
 			flags |= PERFORM_DELETION_CONCURRENTLY;
 		}
 
-		/*
-		 * Concurrent index drop cannot be used with partitioned indexes,
-		 * either.
-		 */
-		if ((flags & PERFORM_DELETION_CONCURRENTLY) != 0 &&
-			get_rel_relkind(relOid) == RELKIND_PARTITIONED_INDEX)
-			ereport(ERROR,
-					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-					 errmsg("cannot drop partitioned index \"%s\" concurrently",
-							rel->relname)));
-
 		/* OK, we're ready to delete this one */
 		obj.classId = RelationRelationId;
 		obj.objectId = relOid;
diff --git a/src/test/regress/expected/indexing.out b/src/test/regress/expected/indexing.out
index c6d7cfd446..18645dd9c6 100644
--- a/src/test/regress/expected/indexing.out
+++ b/src/test/regress/expected/indexing.out
@@ -226,9 +226,7 @@ create table idxpart1 partition of idxpart for values from (0) to (10);
 drop index idxpart1_a_idx;	-- no way
 ERROR:  cannot drop index idxpart1_a_idx because index idxpart_a_idx requires it
 HINT:  You can drop index idxpart_a_idx instead.
-drop index concurrently idxpart_a_idx;	-- unsupported
-ERROR:  cannot drop partitioned index "idxpart_a_idx" concurrently
-drop index idxpart_a_idx;	-- both indexes go away
+drop index concurrently idxpart_a_idx;
 select relname, relkind from pg_class
   where relname like 'idxpart%' order by relname;
  relname  | relkind 
diff --git a/src/test/regress/sql/indexing.sql b/src/test/regress/sql/indexing.sql
index 3d4b6e9bc9..610c12de9e 100644
--- a/src/test/regress/sql/indexing.sql
+++ b/src/test/regress/sql/indexing.sql
@@ -104,8 +104,7 @@ create table idxpart (a int) partition by range (a);
 create index on idxpart (a);
 create table idxpart1 partition of idxpart for values from (0) to (10);
 drop index idxpart1_a_idx;	-- no way
-drop index concurrently idxpart_a_idx;	-- unsupported
-drop index idxpart_a_idx;	-- both indexes go away
+drop index concurrently idxpart_a_idx;
 select relname, relkind from pg_class
   where relname like 'idxpart%' order by relname;
 create index on idxpart (a);
-- 
2.17.0

Reply via email to