Allow CLUSTER on partitioned tables
authorAlvaro Herrera <alvherre@alvh.no-ip.org>
Sat, 2 Apr 2022 17:08:34 +0000 (19:08 +0200)
committerAlvaro Herrera <alvherre@alvh.no-ip.org>
Sat, 2 Apr 2022 17:08:34 +0000 (19:08 +0200)
This is essentially the same as applying VACUUM FULL to a partitioned
table, which has been supported since commit 3c3bb99330aa (March 2017).
While there's no great use case in applying CLUSTER to partitioned
tables, we don't have any strong reason not to allow it either.

For now, partitioned indexes cannot be marked clustered, so an index
must always be specified.

While at it, rename some variables that were RangeVars during the
development that led to 8bc717cb8878 but never made it that way to the
source tree; there's no need to perpetuate names that have always been
more confusing than helpful.

Author: Justin Pryzby <pryzby@telsasoft.com>
Reviewed-by: Matthias van de Meent <boekewurm+postgres@gmail.com>
Discussion: https://postgr.es/m/20201028003312.GU9241@telsasoft.com
Discussion: https://postgr.es/m/20200611153502.GT14879@telsasoft.com

doc/src/sgml/ref/cluster.sgml
src/backend/commands/cluster.c
src/backend/commands/tablecmds.c
src/bin/psql/tab-complete.c
src/include/commands/cluster.h
src/test/regress/expected/cluster.out
src/test/regress/sql/cluster.sql

index 86f5fdc469be470d57913e3d41a64b2eb5940ef0..c37f4236f17f12d2a2bdf351887eaa514cf05424 100644 (file)
@@ -196,6 +196,13 @@ CLUSTER [VERBOSE]
     in the <structname>pg_stat_progress_cluster</structname> view. See
     <xref linkend="cluster-progress-reporting"/> for details.
   </para>
+
+   <para>
+    Clustering a partitioned table clusters each of its partitions using the
+    partition of the specified partitioned index.  When clustering a partitioned
+    table, the index may not be omitted.
+   </para>
+
  </refsect1>
 
  <refsect1>
index 02a7e94bf9b1c9d76136b76ed447145343452c29..cd19e35319e49ba33485a2858e88c9c66ed2a9eb 100644 (file)
@@ -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/defrem.h"
@@ -68,11 +70,14 @@ typedef struct
 } RelToCluster;
 
 
+static void cluster_multiple_rels(List *rtcs, ClusterParams *params);
 static void rebuild_relation(Relation OldHeap, Oid indexOid, bool verbose);
 static void copy_table_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex,
                            bool verbose, bool *pSwapToastByContent,
                            TransactionId *pFreezeXid, MultiXactId *pCutoffMulti);
 static List *get_tables_to_cluster(MemoryContext cluster_context);
+static List *get_tables_to_cluster_partitioned(MemoryContext cluster_context,
+                                              Oid indexOid);
 
 
 /*---------------------------------------------------------------------------
@@ -105,6 +110,10 @@ cluster(ParseState *pstate, ClusterStmt *stmt, bool isTopLevel)
    ListCell   *lc;
    ClusterParams params = {0};
    bool        verbose = false;
+   Relation    rel = NULL;
+   Oid         indexOid = InvalidOid;
+   MemoryContext cluster_context;
+   List       *rtcs;
 
    /* Parse option list */
    foreach(lc, stmt->params)
@@ -126,11 +135,13 @@ cluster(ParseState *pstate, ClusterStmt *stmt, bool isTopLevel)
    if (stmt->relation != NULL)
    {
        /* This is the single-relation case. */
-       Oid         tableOid,
-                   indexOid = InvalidOid;
-       Relation    rel;
+       Oid         tableOid;
 
-       /* Find, lock, and check permissions on the table */
+       /*
+        * Find, lock, and check permissions on the table.  We obtain
+        * AccessExclusiveLock right away to avoid lock-upgrade hazard in the
+        * single-transaction case.
+        */
        tableOid = RangeVarGetRelidExtended(stmt->relation,
                                            AccessExclusiveLock,
                                            0,
@@ -146,14 +157,6 @@ cluster(ParseState *pstate, 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;
@@ -188,71 +191,101 @@ cluster(ParseState *pstate, ClusterStmt *stmt, bool isTopLevel)
                                stmt->indexname, stmt->relation->relname)));
        }
 
-       /* close relation, keep lock till commit */
-       table_close(rel, NoLock);
+       if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+       {
+           /* close relation, keep lock till commit */
+           table_close(rel, NoLock);
 
-       /* Do the job. */
-       cluster_rel(tableOid, indexOid, &params);
+           /* Do the job. */
+           cluster_rel(tableOid, indexOid, &params);
+
+           return;
+       }
+   }
+
+   /*
+    * By here, we know we are in a multi-table situation.  In order to avoid
+    * holding locks for too long, we want to process each table in its own
+    * transaction.  This forces us to disallow running inside a user
+    * transaction block.
+    */
+   PreventInTransactionBlock(isTopLevel, "CLUSTER");
+
+   /* Also, we need a memory context to hold our list of relations */
+   cluster_context = AllocSetContextCreate(PortalContext,
+                                           "Cluster",
+                                           ALLOCSET_DEFAULT_SIZES);
+
+   /*
+    * Either we're processing a partitioned table, or we were not given any
+    * table name at all.  In either case, obtain a list of relations to
+    * process.
+    *
+    * In the former case, an index name must have been given, so we don't
+    * need to recheck its "indisclustered" bit, but we have to check that it
+    * is an index that we can cluster on.  In the latter case, we set the
+    * option bit to have indisclustered verified.
+    *
+    * Rechecking the relation itself is necessary here in all cases.
+    */
+   params.options |= CLUOPT_RECHECK;
+   if (rel != NULL)
+   {
+       Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
+       check_index_is_clusterable(rel, indexOid, true, AccessShareLock);
+       rtcs = get_tables_to_cluster_partitioned(cluster_context, indexOid);
+
+       /* close relation, releasing lock on parent table */
+       table_close(rel, AccessExclusiveLock);
    }
    else
    {
-       /*
-        * This is the "multi relation" case. We need to cluster all tables
-        * that have some index with indisclustered set.
-        */
-       MemoryContext cluster_context;
-       List       *rvs;
-       ListCell   *rv;
+       rtcs = get_tables_to_cluster(cluster_context);
+       params.options |= CLUOPT_RECHECK_ISCLUSTERED;
+   }
 
-       /*
-        * We cannot run this form of CLUSTER inside a user transaction block;
-        * we'd be holding locks way too long.
-        */
-       PreventInTransactionBlock(isTopLevel, "CLUSTER");
+   /* Do the job. */
+   cluster_multiple_rels(rtcs, &params);
 
-       /*
-        * Create special memory context for cross-transaction storage.
-        *
-        * Since it is a child of PortalContext, it will go away even in case
-        * of error.
-        */
-       cluster_context = AllocSetContextCreate(PortalContext,
-                                               "Cluster",
-                                               ALLOCSET_DEFAULT_SIZES);
+   /* Start a new transaction for the cleanup work. */
+   StartTransactionCommand();
 
-       /*
-        * Build the list of relations to cluster.  Note that this lives in
-        * cluster_context.
-        */
-       rvs = get_tables_to_cluster(cluster_context);
+   /* Clean up working storage */
+   MemoryContextDelete(cluster_context);
+}
 
-       /* Commit to get out of starting transaction */
-       PopActiveSnapshot();
-       CommitTransactionCommand();
+/*
+ * Given a list of relations to cluster, process each of them in a separate
+ * transaction.
+ *
+ * We expect to be in a transaction at start, but there isn't one when we
+ * return.
+ */
+static void
+cluster_multiple_rels(List *rtcs, ClusterParams *params)
+{
+   ListCell   *lc;
 
-       /* Ok, now that we've got them all, cluster them one by one */
-       foreach(rv, rvs)
-       {
-           RelToCluster *rvtc = (RelToCluster *) lfirst(rv);
-           ClusterParams cluster_params = params;
+   /* Commit to get out of starting transaction */
+   PopActiveSnapshot();
+   CommitTransactionCommand();
 
-           /* Start a new transaction for each relation. */
-           StartTransactionCommand();
-           /* functions in indexes may want a snapshot set */
-           PushActiveSnapshot(GetTransactionSnapshot());
-           /* Do the job. */
-           cluster_params.options |= CLUOPT_RECHECK;
-           cluster_rel(rvtc->tableOid, rvtc->indexOid,
-                       &cluster_params);
-           PopActiveSnapshot();
-           CommitTransactionCommand();
-       }
+   /* Cluster the tables, each in a separate transaction */
+   foreach(lc, rtcs)
+   {
+       RelToCluster *rtc = (RelToCluster *) lfirst(lc);
 
-       /* Start a new transaction for the cleanup work. */
+       /* Start a new transaction for each relation. */
        StartTransactionCommand();
 
-       /* Clean up working storage */
-       MemoryContextDelete(cluster_context);
+       /* functions in indexes may want a snapshot set */
+       PushActiveSnapshot(GetTransactionSnapshot());
+
+       /* Do the job. */
+       cluster_rel(rtc->tableOid, rtc->indexOid, params);
+
+       PopActiveSnapshot();
+       CommitTransactionCommand();
    }
 }
 
@@ -327,10 +360,11 @@ cluster_rel(Oid tableOid, Oid indexOid, ClusterParams *params)
        /*
         * Silently skip a temp table for a remote session.  Only doing this
         * check in the "recheck" case is appropriate (which currently means
-        * somebody is executing a database-wide CLUSTER), because there is
-        * another check in cluster() which will stop any attempt to cluster
-        * remote temp tables by name.  There is another check in cluster_rel
-        * which is redundant, but we leave it for extra safety.
+        * somebody is executing a database-wide CLUSTER or on a partitioned
+        * table), because there is another check in cluster() which will stop
+        * any attempt to cluster remote temp tables by name.  There is
+        * another check in cluster_rel which is redundant, but we leave it
+        * for extra safety.
         */
        if (RELATION_IS_OTHER_TEMP(OldHeap))
        {
@@ -352,9 +386,11 @@ cluster_rel(Oid tableOid, Oid indexOid, ClusterParams *params)
            }
 
            /*
-            * 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 ((params->options & CLUOPT_RECHECK_ISCLUSTERED) != 0 &&
+               !get_index_isclustered(indexOid))
            {
                relation_close(OldHeap, AccessExclusiveLock);
                pgstat_progress_end_command();
@@ -415,6 +451,10 @@ cluster_rel(Oid tableOid, Oid indexOid, ClusterParams *params)
        return;
    }
 
+   Assert(OldHeap->rd_rel->relkind == RELKIND_RELATION ||
+          OldHeap->rd_rel->relkind == RELKIND_MATVIEW ||
+          OldHeap->rd_rel->relkind == RELKIND_TOASTVALUE);
+
    /*
     * All predicate locks on the tuples or pages are about to be made
     * invalid, because we move tuples around.  Promote them to relation
@@ -585,8 +625,8 @@ rebuild_relation(Relation OldHeap, Oid indexOid, bool verbose)
    TransactionId frozenXid;
    MultiXactId cutoffMulti;
 
-   /* Mark the correct index as clustered */
    if (OidIsValid(indexOid))
+       /* Mark the correct index as clustered */
        mark_index_clustered(OldHeap, indexOid, true);
 
    /* Remember info about rel before closing OldHeap */
@@ -1528,8 +1568,8 @@ finish_heap_swap(Oid OIDOldHeap, Oid OIDNewHeap,
 
            /*
             * Reset the relrewrite for the toast. The command-counter
-            * increment is required here as we are about to update
-            * the tuple that is updated as part of RenameRelationInternal.
+            * increment is required here as we are about to update the tuple
+            * that is updated as part of RenameRelationInternal.
             */
            CommandCounterIncrement();
            ResetRelRewrite(newrel->rd_rel->reltoastrelid);
@@ -1564,8 +1604,7 @@ get_tables_to_cluster(MemoryContext cluster_context)
    HeapTuple   indexTuple;
    Form_pg_index index;
    MemoryContext old_context;
-   RelToCluster *rvtc;
-   List       *rvs = NIL;
+   List       *rtcs = NIL;
 
    /*
     * Get all indexes that have indisclustered set and are owned by
@@ -1579,21 +1618,20 @@ get_tables_to_cluster(MemoryContext cluster_context)
    scan = table_beginscan_catalog(indRelation, 1, &entry);
    while ((indexTuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
    {
+       RelToCluster *rtc;
+
        index = (Form_pg_index) GETSTRUCT(indexTuple);
 
        if (!pg_class_ownercheck(index->indrelid, GetUserId()))
            continue;
 
-       /*
-        * We have to build the list in a different memory context so it will
-        * survive the cross-transaction processing
-        */
+       /* Use a permanent memory context for the result list */
        old_context = MemoryContextSwitchTo(cluster_context);
 
-       rvtc = (RelToCluster *) palloc(sizeof(RelToCluster));
-       rvtc->tableOid = index->indrelid;
-       rvtc->indexOid = index->indexrelid;
-       rvs = lappend(rvs, rvtc);
+       rtc = (RelToCluster *) palloc(sizeof(RelToCluster));
+       rtc->tableOid = index->indrelid;
+       rtc->indexOid = index->indexrelid;
+       rtcs = lappend(rtcs, rtc);
 
        MemoryContextSwitchTo(old_context);
    }
@@ -1601,5 +1639,48 @@ get_tables_to_cluster(MemoryContext cluster_context)
 
    relation_close(indRelation, AccessShareLock);
 
-   return rvs;
+   return rtcs;
+}
+
+/*
+ * Given an index on a partitioned table, return a list of RelToCluster for
+ * all the children leaves tables/indexes.
+ *
+ * Caller must hold lock on the table containing the index.
+ *
+ * Like expand_vacuum_rel, but here caller must hold AccessExclusiveLock
+ * on the table already.
+ */
+static List *
+get_tables_to_cluster_partitioned(MemoryContext cluster_context, Oid indexOid)
+{
+   List       *inhoids;
+   ListCell   *lc;
+   List       *rtcs = NIL;
+   MemoryContext old_context;
+
+   /* Do not lock the children until they're processed */
+   inhoids = find_all_inheritors(indexOid, NoLock, NULL);
+
+   /* Use a permanent memory context for the result list */
+   old_context = MemoryContextSwitchTo(cluster_context);
+
+   foreach(lc, inhoids)
+   {
+       Oid         indexrelid = lfirst_oid(lc);
+       Oid         relid = IndexGetRelation(indexrelid, false);
+       RelToCluster *rtc;
+
+       /* consider only leaf indexes */
+       if (get_rel_relkind(indexrelid) != RELKIND_INDEX)
+           continue;
+
+       rtc = (RelToCluster *) palloc(sizeof(RelToCluster));
+       rtc->tableOid = relid;
+       rtc->indexOid = indexrelid;
+       rtcs = lappend(rtcs, rtc);
+   }
+
+   MemoryContextSwitchTo(old_context);
+   return rtcs;
 }
index 51b4a00d50d52caef864dfa31dd828cbc3f4c975..7febb5018fc1e47dce3d742d1c8edf7492fe4d4c 100644 (file)
@@ -16911,11 +16911,11 @@ AtEOSubXact_on_commit_actions(bool isCommit, SubTransactionId mySubid,
 
 /*
  * This is intended as a callback for RangeVarGetRelidExtended().  It allows
- * the relation to be locked only if (1) it's a plain table, materialized
- * view, or TOAST table and (2) the current user is the owner (or the
- * superuser).  This meets the permission-checking needs of CLUSTER, REINDEX
- * TABLE, and REFRESH MATERIALIZED VIEW; we expose it here so that it can be
- * used by all.
+ * the relation to be locked only if (1) it's a plain or partitioned table,
+ * materialized view, or TOAST table and (2) the current user is the owner (or
+ * the superuser).  This meets the permission-checking needs of CLUSTER,
+ * REINDEX TABLE, and REFRESH MATERIALIZED VIEW; we expose it here so that it
+ * can be used by all.
  */
 void
 RangeVarCallbackOwnsTable(const RangeVar *relation,
index 3f9dfffd57fa18c0072f4229a961846559f897cd..6f040674a256d7af5b59ecbadc8145ba564f9321 100644 (file)
@@ -897,6 +897,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",
index 3db375d7cc76bd5a89830842e5e0c239ac3336c9..3c279f6210ae1dc6011155fdce72e8a6da98dacf 100644 (file)
 #include "utils/relcache.h"
 
 
-/* flag bits for ClusterParams->flags */
-#define CLUOPT_RECHECK 0x01        /* recheck relation state */
-#define CLUOPT_VERBOSE 0x02        /* print progress info */
+/* flag bits for ClusterParams->options */
+#define CLUOPT_VERBOSE 0x01        /* print progress info */
+#define CLUOPT_RECHECK 0x02        /* recheck relation state */
+#define CLUOPT_RECHECK_ISCLUSTERED 0x04 /* recheck relation state for
+                                        * indisclustered */
 
 /* options for CLUSTER */
 typedef struct ClusterParams
index e46a66952f0c408c98c67f4096df499653b92b1d..953818c74e183518982aa868f49fcf83f3d9a2e8 100644 (file)
@@ -305,6 +305,8 @@ WHERE pg_class.oid=indexrelid
 ---------
 (0 rows)
 
+-- Verify that toast tables are clusterable
+CLUSTER pg_toast.pg_toast_826 USING pg_toast_826_index;
 -- Verify that clustering all tables does in fact cluster the right ones
 CREATE USER regress_clstr_user;
 CREATE TABLE clstr_1 (a INT PRIMARY KEY);
@@ -444,13 +446,52 @@ DROP TABLE clustertest;
 CREATE TABLE clustertest (f1 int PRIMARY KEY);
 CLUSTER clustertest USING clustertest_pkey;
 CLUSTER clustertest;
--- 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 (5);
+CREATE TABLE clstrpart12 PARTITION OF clstrpart1 FOR VALUES FROM (5) TO (10) PARTITION BY RANGE (a);
+CREATE TABLE clstrpart2 PARTITION OF clstrpart FOR VALUES FROM (10) TO (20);
+CREATE TABLE clstrpart3 PARTITION OF clstrpart DEFAULT PARTITION BY RANGE (a);
+CREATE TABLE clstrpart33 PARTITION OF clstrpart3 DEFAULT;
+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);
+-- Check that clustering sets new relfilenodes:
+CREATE TEMP TABLE old_cluster_info AS SELECT relname, level, 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, level, relfilenode, relkind FROM pg_partition_tree('clstrpart'::regclass) AS tree JOIN pg_class c ON c.oid=tree.relid ;
+SELECT relname, old.level, old.relkind, old.relfilenode = new.relfilenode FROM old_cluster_info AS old JOIN new_cluster_info AS new USING (relname) ORDER BY relname COLLATE "C";
+   relname   | level | relkind | ?column? 
+-------------+-------+---------+----------
+ clstrpart   |     0 | p       | t
+ clstrpart1  |     1 | p       | t
+ clstrpart11 |     2 | r       | f
+ clstrpart12 |     2 | p       | t
+ clstrpart2  |     1 | r       | f
+ clstrpart3  |     1 | p       | t
+ clstrpart33 |     2 | r       | f
+(7 rows)
+
+-- Partitioned indexes aren't and can't be marked un/clustered:
+\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.)
+
+CLUSTER clstrpart;
+ERROR:  there is no previously clustered index for table "clstrpart"
+ALTER TABLE clstrpart SET WITHOUT CLUSTER;
+ERROR:  cannot mark index clustered in partitioned table
 ALTER TABLE clstrpart CLUSTER ON clstrpart_idx;
 ERROR:  cannot mark index clustered in partitioned table
-CLUSTER clstrpart USING clstrpart_idx;
-ERROR:  cannot cluster a partitioned table
 DROP TABLE clstrpart;
 -- Test CLUSTER with external tuplesorting
 create table clstr_4 as select * from tenk1;
index aee9cf83e04c8433d2d300081baa481a12d06fa7..5601684ee3fc0c94cbdff66980938bcd23412a1c 100644 (file)
@@ -104,6 +104,9 @@ WHERE pg_class.oid=indexrelid
    AND pg_class_2.relname = 'clstr_tst'
    AND indisclustered;
 
+-- Verify that toast tables are clusterable
+CLUSTER pg_toast.pg_toast_826 USING pg_toast_826_index;
+
 -- Verify that clustering all tables does in fact cluster the right ones
 CREATE USER regress_clstr_user;
 CREATE TABLE clstr_1 (a INT PRIMARY KEY);
@@ -202,11 +205,28 @@ CREATE TABLE clustertest (f1 int PRIMARY KEY);
 CLUSTER clustertest USING clustertest_pkey;
 CLUSTER clustertest;
 
--- 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 (5);
+CREATE TABLE clstrpart12 PARTITION OF clstrpart1 FOR VALUES FROM (5) TO (10) PARTITION BY RANGE (a);
+CREATE TABLE clstrpart2 PARTITION OF clstrpart FOR VALUES FROM (10) TO (20);
+CREATE TABLE clstrpart3 PARTITION OF clstrpart DEFAULT PARTITION BY RANGE (a);
+CREATE TABLE clstrpart33 PARTITION OF clstrpart3 DEFAULT;
+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, level, 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, level, relfilenode, relkind FROM pg_partition_tree('clstrpart'::regclass) AS tree JOIN pg_class c ON c.oid=tree.relid ;
+SELECT relname, old.level, old.relkind, old.relfilenode = new.relfilenode FROM old_cluster_info AS old JOIN new_cluster_info AS new USING (relname) ORDER BY relname COLLATE "C";
+-- Partitioned indexes aren't and can't be marked un/clustered:
+\d clstrpart
+CLUSTER clstrpart;
+ALTER TABLE clstrpart SET WITHOUT CLUSTER;
+ALTER TABLE clstrpart CLUSTER ON clstrpart_idx;
 DROP TABLE clstrpart;
 
 -- Test CLUSTER with external tuplesorting