Implement ALTER TABLE ... MERGE PARTITIONS ... command
authorAlexander Korotkov <[email protected]>
Sat, 6 Apr 2024 21:57:22 +0000 (00:57 +0300)
committerAlexander Korotkov <[email protected]>
Sat, 6 Apr 2024 22:18:43 +0000 (01:18 +0300)
This new DDL command merges several partitions into the one partition of the
target table.  The target partition is created using new
createPartitionTable() function with parent partition as the template.

This commit comprises quite naive implementation which works in single process
and holds the ACCESS EXCLUSIVE LOCK on the parent table during all the
operations including the tuple routing.  This is why this new DDL command
can't be recommended for large partitioned tables under a high load.  However,
this implementation come in handy in certain cases even as is.
Also, it could be used as a foundation for future implementations with lesser
locking and possibly parallel.

Discussion: https://postgr.es/m/c73a1746-0cd0-6bdd-6b23-3ae0b7c0c582%40postgrespro.ru
Author: Dmitry Koval
Reviewed-by: Matthias van de Meent, Laurenz Albe, Zhihong Yu, Justin Pryzby
Reviewed-by: Alvaro Herrera, Robert Haas, Stephane Tachoires
17 files changed:
doc/src/sgml/ddl.sgml
doc/src/sgml/ref/alter_table.sgml
src/backend/commands/tablecmds.c
src/backend/parser/gram.y
src/backend/parser/parse_utilcmd.c
src/backend/partitioning/partbounds.c
src/include/nodes/parsenodes.h
src/include/parser/kwlist.h
src/include/partitioning/partbounds.h
src/test/isolation/expected/partition-merge.out [new file with mode: 0644]
src/test/isolation/isolation_schedule
src/test/isolation/specs/partition-merge.spec [new file with mode: 0644]
src/test/modules/test_ddl_deparse/test_ddl_deparse.c
src/test/regress/expected/partition_merge.out [new file with mode: 0644]
src/test/regress/parallel_schedule
src/test/regress/sql/partition_merge.sql [new file with mode: 0644]
src/tools/pgindent/typedefs.list

index 6bc43ad10cac22fabe21823dd46174ae7fcf3767..f9f2778b3d79462722601a00230bb08473f760c6 100644 (file)
@@ -4379,6 +4379,25 @@ ALTER TABLE measurement_y2006m02 ADD UNIQUE (city_id, logdate);
 ALTER INDEX measurement_city_id_logdate_key
     ATTACH PARTITION measurement_y2006m02_city_id_logdate_key;
 ...
+</programlisting>
+    </para>
+
+    <para>
+     There is also an option for merging multiple table partitions into
+     a single partition using the
+     <link linkend="sql-altertable-merge-partitions"><command>ALTER TABLE ... MERGE PARTITIONS</command></link>.
+     This feature simplifies the management of partitioned tables by allowing
+     administrators to combine partitions that are no longer needed as
+     separate entities.  It's important to note that this operation is not
+     supported for hash-partitioned tables and acquires an
+     <literal>ACCESS EXCLUSIVE</literal> lock, which could impact high-load
+     systems due to the lock's restrictive nature.  For example, we can
+     merge three monthly partitions into one quarter partition:
+<programlisting>
+ALTER TABLE measurement
+    MERGE PARTITIONS (measurement_y2006m01,
+                      measurement_y2006m02,
+                      measurement_y2006m03) INTO measurement_y2006q1;
 </programlisting>
     </para>
    </sect3>
index 6cdcd779ef83a4e35a929cf95c5f70ae1ae769a7..e0a98bc538527f517846204c516fdd047eb1936c 100644 (file)
@@ -37,6 +37,9 @@ 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> [ CONCURRENTLY | FINALIZE ]
+ALTER TABLE [ IF EXISTS ] <replaceable class="parameter">name</replaceable>
+    MERGE PARTITIONS (<replaceable class="parameter">partition_name1</replaceable>, <replaceable class="parameter">partition_name2</replaceable> [, ...])
+        INTO <replaceable class="parameter">partition_name</replaceable>
 
 <phrase>where <replaceable class="parameter">action</replaceable> is one of:</phrase>
 
@@ -1118,14 +1121,74 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
     </listitem>
    </varlistentry>
 
+   <varlistentry id="sql-altertable-merge-partitions">
+    <term><literal>MERGE PARTITIONS (<replaceable class="parameter">partition_name1</replaceable>, <replaceable class="parameter">partition_name2</replaceable> [, ...]) INTO <replaceable class="parameter">partition_name</replaceable></literal></term>
+
+    <listitem>
+     <para>
+      This form merges several partitions into the one partition of the target table.
+      Hash-partitioning is not supported.  If DEFAULT partition is not in the
+      list of partitions <replaceable class="parameter">partition_name1</replaceable>,
+      <replaceable class="parameter">partition_name2</replaceable> [, ...]:
+      <itemizedlist>
+       <listitem>
+        <para>
+         For range-partitioned tables is necessary that the ranges
+         of the partitions <replaceable class="parameter">partition_name1</replaceable>,
+         <replaceable class="parameter">partition_name2</replaceable> [, ...] can
+         be merged into one range without spaces and overlaps (otherwise an error
+         will be generated).  The combined range will be the range for the partition
+         <replaceable class="parameter">partition_name</replaceable>.
+        </para>
+       </listitem>
+       <listitem>
+        <para>
+         For list-partitioned tables the values lists of all partitions
+         <replaceable class="parameter">partition_name1</replaceable>,
+         <replaceable class="parameter">partition_name2</replaceable> [, ...] are
+         combined and form a list of values of partition
+         <replaceable class="parameter">partition_name</replaceable>.
+        </para>
+       </listitem>
+      </itemizedlist>
+      If DEFAULT partition is in the list of partitions <replaceable class="parameter">partition_name1</replaceable>,
+      <replaceable class="parameter">partition_name2</replaceable> [, ...]:
+      <itemizedlist>
+       <listitem>
+        <para>
+         The partition <replaceable class="parameter">partition_name</replaceable>
+         will be the DEFAULT partition.
+        </para>
+       </listitem>
+       <listitem>
+        <para>
+         For range- and list-partitioned tables the ranges and lists of values
+         of the merged partitions can be any.
+        </para>
+       </listitem>
+      </itemizedlist>
+      The new partition <replaceable class="parameter">partition_name</replaceable>
+      can have the same name as one of the merged partitions.  Only simple,
+      non-partitioned partitions can be merged.
+     </para>
+     <note>
+      <para>
+       This command acquires an <literal>ACCESS EXCLUSIVE</literal> lock.
+       This is a significant limitation, which limits the usage of this
+       command with large partitioned tables under a high load.
+      </para>
+     </note>
+    </listitem>
+   </varlistentry>
+
   </variablelist>
   </para>
 
   <para>
    All the forms of ALTER TABLE that act on a single table, except
    <literal>RENAME</literal>, <literal>SET SCHEMA</literal>,
-   <literal>ATTACH PARTITION</literal>, and
-   <literal>DETACH PARTITION</literal> can be combined into
+   <literal>ATTACH PARTITION</literal>, <literal>DETACH PARTITION</literal>,
+   and <literal>MERGE PARTITIONS</literal> can be combined into
    a list of multiple alterations to be applied together.  For example, it
    is possible to add several columns and/or alter the type of several
    columns in a single command.  This is particularly useful with large
@@ -1368,7 +1431,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
       <term><replaceable class="parameter">partition_name</replaceable></term>
       <listitem>
        <para>
-        The name of the table to attach as a new partition or to detach from this table.
+        The name of the table to attach as a new partition or to detach from this table,
+        or the name of the new merged partition.
        </para>
       </listitem>
      </varlistentry>
@@ -1784,6 +1848,13 @@ ALTER TABLE measurement
     DETACH PARTITION measurement_y2015m12;
 </programlisting></para>
 
+  <para>
+   To merge several partitions into one partition of the target table:
+<programlisting>
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_central)
+    INTO sales_all;
+</programlisting></para>
+
  </refsect1>
 
  <refsect1>
index 317b89f67c3bc19a0d6e9990430d77bdb1f5bbda..05e86de8ebc25f5ce3b3712eec82b9512cb1b4cc 100644 (file)
@@ -670,6 +670,8 @@ static void ATDetachCheckNoForeignKeyRefs(Relation partition);
 static char GetAttributeCompression(Oid atttypid, const char *compression);
 static char GetAttributeStorage(Oid atttypid, const char *storagemode);
 
+static void ATExecMergePartitions(List **wqueue, AlteredTableInfo *tab, Relation rel,
+                                 PartitionCmd *cmd, AlterTableUtilityContext *context);
 
 /* ----------------------------------------------------------------
  *     DefineRelation
@@ -4738,6 +4740,10 @@ AlterTableGetLockLevel(List *cmds)
                cmd_lockmode = ShareUpdateExclusiveLock;
                break;
 
+           case AT_MergePartitions:
+               cmd_lockmode = AccessExclusiveLock;
+               break;
+
            default:            /* oops */
                elog(ERROR, "unrecognized alter table type: %d",
                     (int) cmd->subtype);
@@ -5157,6 +5163,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
            /* No command-specific prep needed */
            pass = AT_PASS_MISC;
            break;
+       case AT_MergePartitions:
+           ATSimplePermissions(cmd->subtype, 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);
@@ -5554,6 +5565,14 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
        case AT_DetachPartitionFinalize:
            address = ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
            break;
+       case AT_MergePartitions:
+           cmd = ATParseTransformCmd(wqueue, tab, rel, cmd, false, lockmode,
+                                     cur_pass, context);
+           Assert(cmd != NULL);
+           Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
+           ATExecMergePartitions(wqueue, tab, rel, (PartitionCmd *) cmd->def,
+                                 context);
+           break;
        default:                /* oops */
            elog(ERROR, "unrecognized alter table type: %d",
                 (int) cmd->subtype);
@@ -6548,6 +6567,8 @@ alter_table_type_to_string(AlterTableType cmdtype)
            return "DETACH PARTITION";
        case AT_DetachPartitionFinalize:
            return "DETACH PARTITION ... FINALIZE";
+       case AT_MergePartitions:
+           return "MERGE PARTITIONS";
        case AT_AddIdentity:
            return "ALTER COLUMN ... ADD IDENTITY";
        case AT_SetIdentity:
@@ -19042,6 +19063,37 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
    }
 }
 
+/*
+ * attachPartitionTable: attach new partition to partitioned table
+ *
+ * wqueue: the ALTER TABLE work queue; can be NULL when not running as part
+ *   of an ALTER TABLE sequence.
+ * rel: partitioned relation;
+ * attachrel: relation of attached partition;
+ * bound: bounds of attached relation.
+ */
+static void
+attachPartitionTable(List **wqueue, Relation rel, Relation attachrel, PartitionBoundSpec *bound)
+{
+   /* OK to create inheritance.  Rest of the checks performed there */
+   CreateInheritance(attachrel, rel, true);
+
+   /* Update the pg_class entry. */
+   StorePartitionBound(attachrel, rel, bound);
+
+   /* Ensure there exists a correct set of indexes in the partition. */
+   AttachPartitionEnsureIndexes(wqueue, rel, attachrel);
+
+   /* and triggers */
+   CloneRowTriggersToPartition(rel, attachrel);
+
+   /*
+    * Clone foreign key constraints.  Callee is responsible for setting up
+    * for phase 3 constraint verification.
+    */
+   CloneForeignKeyConstraints(wqueue, rel, attachrel);
+}
+
 /*
  * ALTER TABLE <name> ATTACH PARTITION <partition-name> FOR VALUES
  *
@@ -19244,23 +19296,8 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
    check_new_partition_bound(RelationGetRelationName(attachrel), rel,
                              cmd->bound, pstate);
 
-   /* OK to create inheritance.  Rest of the checks performed there */
-   CreateInheritance(attachrel, rel, true);
-
-   /* Update the pg_class entry. */
-   StorePartitionBound(attachrel, rel, cmd->bound);
-
-   /* Ensure there exists a correct set of indexes in the partition. */
-   AttachPartitionEnsureIndexes(wqueue, rel, attachrel);
-
-   /* and triggers */
-   CloneRowTriggersToPartition(rel, attachrel);
-
-   /*
-    * Clone foreign key constraints.  Callee is responsible for setting up
-    * for phase 3 constraint verification.
-    */
-   CloneForeignKeyConstraints(wqueue, rel, attachrel);
+   /* Attach a new partition to the partitioned table. */
+   attachPartitionTable(wqueue, rel, attachrel, cmd->bound);
 
    /*
     * Generate partition constraint from the partition bound specification.
@@ -20814,3 +20851,286 @@ GetAttributeStorage(Oid atttypid, const char *storagemode)
 
    return cstorage;
 }
+
+/*
+ * createPartitionTable: create table for new partition with given name
+ * (newPartName) like table (modelRelName)
+ *
+ * Emulates command: CREATE TABLE <newPartName> (LIKE <modelRelName>
+ * INCLUDING ALL EXCLUDING INDEXES EXCLUDING IDENTITY)
+ */
+static void
+createPartitionTable(RangeVar *newPartName, RangeVar *modelRelName,
+                    AlterTableUtilityContext *context)
+{
+   CreateStmt *createStmt;
+   TableLikeClause *tlc;
+   PlannedStmt *wrapper;
+
+   createStmt = makeNode(CreateStmt);
+   createStmt->relation = newPartName;
+   createStmt->tableElts = NIL;
+   createStmt->inhRelations = NIL;
+   createStmt->constraints = NIL;
+   createStmt->options = NIL;
+   createStmt->oncommit = ONCOMMIT_NOOP;
+   createStmt->tablespacename = NULL;
+   createStmt->if_not_exists = false;
+
+   tlc = makeNode(TableLikeClause);
+   tlc->relation = modelRelName;
+
+   /*
+    * Indexes will be inherited on "attach new partitions" stage, after data
+    * moving.
+    */
+   tlc->options = CREATE_TABLE_LIKE_ALL & ~(CREATE_TABLE_LIKE_INDEXES | CREATE_TABLE_LIKE_IDENTITY);
+   tlc->relationOid = InvalidOid;
+   createStmt->tableElts = lappend(createStmt->tableElts, tlc);
+
+   /* Need to make a wrapper PlannedStmt. */
+   wrapper = makeNode(PlannedStmt);
+   wrapper->commandType = CMD_UTILITY;
+   wrapper->canSetTag = false;
+   wrapper->utilityStmt = (Node *) createStmt;
+   wrapper->stmt_location = context->pstmt->stmt_location;
+   wrapper->stmt_len = context->pstmt->stmt_len;
+
+   ProcessUtility(wrapper,
+                  context->queryString,
+                  false,
+                  PROCESS_UTILITY_SUBCOMMAND,
+                  NULL,
+                  NULL,
+                  None_Receiver,
+                  NULL);
+}
+
+/*
+ * moveMergedTablesRows: scan partitions to be merged (mergingPartitionsList)
+ * of the partitioned table (rel) and move rows into the new partition
+ * (newPartRel).
+ */
+static void
+moveMergedTablesRows(Relation rel, List *mergingPartitionsList,
+                    Relation newPartRel)
+{
+   CommandId   mycid;
+
+   /* The FSM is empty, so don't bother using it. */
+   int         ti_options = TABLE_INSERT_SKIP_FSM;
+   ListCell   *listptr;
+   BulkInsertState bistate;    /* state of bulk inserts for partition */
+   TupleTableSlot *dstslot;
+
+   mycid = GetCurrentCommandId(true);
+
+   /* Prepare a BulkInsertState for table_tuple_insert. */
+   bistate = GetBulkInsertState();
+
+   /* Create necessary tuple slot. */
+   dstslot = MakeSingleTupleTableSlot(RelationGetDescr(newPartRel),
+                                      table_slot_callbacks(newPartRel));
+   ExecStoreAllNullTuple(dstslot);
+
+   foreach(listptr, mergingPartitionsList)
+   {
+       Relation    mergingPartition = (Relation) lfirst(listptr);
+       TupleTableSlot *srcslot;
+       TupleConversionMap *tuple_map;
+       TableScanDesc scan;
+       Snapshot    snapshot;
+
+       /* Create tuple slot for new partition. */
+       srcslot = MakeSingleTupleTableSlot(RelationGetDescr(mergingPartition),
+                                          table_slot_callbacks(mergingPartition));
+
+       /*
+        * Map computing for moving attributes of merged partition to new
+        * partition.
+        */
+       tuple_map = convert_tuples_by_name(RelationGetDescr(mergingPartition),
+                                          RelationGetDescr(newPartRel));
+
+       /* Scan through the rows. */
+       snapshot = RegisterSnapshot(GetLatestSnapshot());
+       scan = table_beginscan(mergingPartition, snapshot, 0, NULL);
+
+       while (table_scan_getnextslot(scan, ForwardScanDirection, srcslot))
+       {
+           TupleTableSlot *insertslot;
+           bool        insert_indexes;
+
+           /* Extract data from old tuple. */
+           slot_getallattrs(srcslot);
+
+           if (tuple_map)
+           {
+               /* Need to use map for copy attributes. */
+               insertslot = execute_attr_map_slot(tuple_map->attrMap, srcslot, dstslot);
+           }
+           else
+           {
+               /* Copy attributes directly. */
+               insertslot = dstslot;
+
+               ExecClearTuple(insertslot);
+
+               memcpy(insertslot->tts_values, srcslot->tts_values,
+                      sizeof(Datum) * srcslot->tts_nvalid);
+               memcpy(insertslot->tts_isnull, srcslot->tts_isnull,
+                      sizeof(bool) * srcslot->tts_nvalid);
+
+               ExecStoreVirtualTuple(insertslot);
+           }
+
+           /*
+            * Write the tuple out to the new relation.  We ignore the
+            * 'insert_indexes' flag since newPartRel has no indexes anyway.
+            */
+           (void) table_tuple_insert(newPartRel, insertslot, mycid,
+                                     ti_options, bistate, &insert_indexes);
+
+           CHECK_FOR_INTERRUPTS();
+       }
+
+       table_endscan(scan);
+       UnregisterSnapshot(snapshot);
+
+       if (tuple_map)
+           free_conversion_map(tuple_map);
+
+       ExecDropSingleTupleTableSlot(srcslot);
+   }
+
+   ExecDropSingleTupleTableSlot(dstslot);
+   FreeBulkInsertState(bistate);
+
+   table_finish_bulk_insert(newPartRel, ti_options);
+}
+
+/*
+ * ALTER TABLE <name> MERGE PARTITIONS <partition-list> INTO <partition-name>
+ */
+static void
+ATExecMergePartitions(List **wqueue, AlteredTableInfo *tab, Relation rel,
+                     PartitionCmd *cmd, AlterTableUtilityContext *context)
+{
+   Relation    newPartRel;
+   ListCell   *listptr;
+   List       *mergingPartitionsList = NIL;
+   Oid         defaultPartOid;
+   char        tmpRelName[NAMEDATALEN];
+   RangeVar   *mergePartName = cmd->name;
+   bool        isSameName = false;
+
+   /*
+    * Lock all merged partitions, check them and create list with partitions
+    * contexts.
+    */
+   foreach(listptr, cmd->partlist)
+   {
+       RangeVar   *name = (RangeVar *) lfirst(listptr);
+       Relation    mergingPartition;
+
+       /*
+        * We are going to detach and remove this partition: need to use
+        * exclusive lock for prevent DML-queries to the partition.
+        */
+       mergingPartition = table_openrv(name, AccessExclusiveLock);
+
+       if (mergingPartition->rd_rel->relkind != RELKIND_RELATION)
+           ereport(ERROR,
+                   (errcode(ERRCODE_WRONG_OBJECT_TYPE),
+                    errmsg("cannot merge non-table partition \"%s\"",
+                           RelationGetRelationName(mergingPartition))));
+
+       /*
+        * Checking that two partitions have the same name was before, in
+        * function transformPartitionCmdForMerge().
+        */
+       if (equal(name, cmd->name))
+           /* One new partition can have the same name as merged partition. */
+           isSameName = true;
+
+       /* Store a next merging partition into the list. */
+       mergingPartitionsList = lappend(mergingPartitionsList,
+                                       mergingPartition);
+   }
+
+   /* Detach all merged partitions. */
+   defaultPartOid =
+       get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, true));
+   foreach(listptr, mergingPartitionsList)
+   {
+       Relation    mergingPartition = (Relation) lfirst(listptr);
+
+       /* Remove the pg_inherits row first. */
+       RemoveInheritance(mergingPartition, rel, false);
+       /* Do the final part of detaching. */
+       DetachPartitionFinalize(rel, mergingPartition, false, defaultPartOid);
+   }
+
+   /* Create table for new partition, use partitioned table as model. */
+   if (isSameName)
+   {
+       /* Create partition table with generated temparary name. */
+       sprintf(tmpRelName, "merge-%u-%X-tmp", RelationGetRelid(rel), MyProcPid);
+       mergePartName = makeRangeVar(get_namespace_name(RelationGetNamespace(rel)),
+                                    tmpRelName, -1);
+   }
+   createPartitionTable(mergePartName,
+                        makeRangeVar(get_namespace_name(RelationGetNamespace(rel)),
+                                     RelationGetRelationName(rel), -1),
+                        context);
+
+   /*
+    * Open the new partition and acquire exclusive lock on it.  This will
+    * stop all the operations with partitioned table.  This might seem
+    * excessive, but this is the way we make sure nobody is planning queries
+    * involving merging partitions.
+    */
+   newPartRel = table_openrv(mergePartName, AccessExclusiveLock);
+
+   /* Copy data from merged partitions to new partition. */
+   moveMergedTablesRows(rel, mergingPartitionsList, newPartRel);
+
+   /*
+    * Attach a new partition to the partitioned table. wqueue = NULL:
+    * verification for each cloned constraint is not need.
+    */
+   attachPartitionTable(NULL, rel, newPartRel, cmd->bound);
+
+   /* Unlock and drop merged partitions. */
+   foreach(listptr, mergingPartitionsList)
+   {
+       ObjectAddress object;
+       Relation    mergingPartition = (Relation) lfirst(listptr);
+
+       /* Get relation id before table_close() call. */
+       object.objectId = RelationGetRelid(mergingPartition);
+       object.classId = RelationRelationId;
+       object.objectSubId = 0;
+
+       /* Keep the lock until commit. */
+       table_close(mergingPartition, NoLock);
+
+       performDeletion(&object, DROP_RESTRICT, 0);
+   }
+   list_free(mergingPartitionsList);
+
+   /* Rename new partition if it is needed. */
+   if (isSameName)
+   {
+       /*
+        * We must bump the command counter to make the new partition tuple
+        * visible for rename.
+        */
+       CommandCounterIncrement();
+       /* Rename partition. */
+       RenameRelationInternal(RelationGetRelid(newPartRel),
+                              cmd->name->relname, false, false);
+   }
+   /* Keep the lock until commit. */
+   table_close(newPartRel, NoLock);
+}
index 6ea68722e3d6ed53da9d0ebc8e1ae7567c349cb9..1fcdb25142f004e5b9e8011614dd2603a4cc56e2 100644 (file)
@@ -761,7 +761,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
    ORDER ORDINALITY OTHERS OUT_P OUTER_P
    OVER OVERLAPS OVERLAY OVERRIDING OWNED OWNER
 
-   PARALLEL PARAMETER PARSER PARTIAL PARTITION PASSING PASSWORD PATH
+   PARALLEL PARAMETER PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PATH
    PERIOD PLACING PLAN PLANS POLICY
 
    POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY
@@ -2317,6 +2317,7 @@ partition_cmd:
                    n->subtype = AT_AttachPartition;
                    cmd->name = $3;
                    cmd->bound = $4;
+                   cmd->partlist = NULL;
                    cmd->concurrent = false;
                    n->def = (Node *) cmd;
 
@@ -2331,6 +2332,7 @@ partition_cmd:
                    n->subtype = AT_DetachPartition;
                    cmd->name = $3;
                    cmd->bound = NULL;
+                   cmd->partlist = NULL;
                    cmd->concurrent = $4;
                    n->def = (Node *) cmd;
 
@@ -2344,6 +2346,21 @@ partition_cmd:
                    n->subtype = AT_DetachPartitionFinalize;
                    cmd->name = $3;
                    cmd->bound = NULL;
+                   cmd->partlist = NULL;
+                   cmd->concurrent = false;
+                   n->def = (Node *) cmd;
+                   $$ = (Node *) n;
+               }
+           /* ALTER TABLE <name> MERGE PARTITIONS () INTO <partition_name> */
+           | MERGE PARTITIONS '(' qualified_name_list ')' INTO qualified_name
+               {
+                   AlterTableCmd *n = makeNode(AlterTableCmd);
+                   PartitionCmd *cmd = makeNode(PartitionCmd);
+
+                   n->subtype = AT_MergePartitions;
+                   cmd->name = $7;
+                   cmd->bound = NULL;
+                   cmd->partlist = $4;
                    cmd->concurrent = false;
                    n->def = (Node *) cmd;
                    $$ = (Node *) n;
@@ -2360,6 +2377,7 @@ index_partition_cmd:
                    n->subtype = AT_AttachPartition;
                    cmd->name = $3;
                    cmd->bound = NULL;
+                   cmd->partlist = NULL;
                    cmd->concurrent = false;
                    n->def = (Node *) cmd;
 
@@ -17668,6 +17686,7 @@ unreserved_keyword:
            | PARSER
            | PARTIAL
            | PARTITION
+           | PARTITIONS
            | PASSING
            | PASSWORD
            | PATH
@@ -18292,6 +18311,7 @@ bare_label_keyword:
            | PARSER
            | PARTIAL
            | PARTITION
+           | PARTITIONS
            | PASSING
            | PASSWORD
            | PATH
index c7efd8d8cee13016fe588cf8f0266197111719f6..48a23c7ca4252b3d5e90afb53645c05228a09bb7 100644 (file)
@@ -58,6 +58,8 @@
 #include "parser/parse_type.h"
 #include "parser/parse_utilcmd.h"
 #include "parser/parser.h"
+#include "partitioning/partdesc.h"
+#include "partitioning/partbounds.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/acl.h"
 #include "utils/builtins.h"
@@ -3413,6 +3415,80 @@ transformRuleStmt(RuleStmt *stmt, const char *queryString,
 }
 
 
+/*
+ * transformPartitionCmdForMerge
+ *     Analyze the ALTER TABLLE ... MERGE PARTITIONS command
+ *
+ * Does simple checks for merged partitions. Calculates bound of result
+ * partition.
+ */
+static void
+transformPartitionCmdForMerge(CreateStmtContext *cxt, PartitionCmd *partcmd)
+{
+   Oid         defaultPartOid;
+   Oid         partOid;
+   Relation    parent = cxt->rel;
+   PartitionKey key;
+   char        strategy;
+   ListCell   *listptr,
+              *listptr2;
+   bool        isDefaultPart = false;
+   List       *partOids = NIL;
+
+   if (parent->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+       ereport(ERROR,
+               (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+                errmsg("\"%s\" is not a partitioned table", RelationGetRelationName(parent))));
+
+   key = RelationGetPartitionKey(parent);
+   strategy = get_partition_strategy(key);
+
+   if (strategy == PARTITION_STRATEGY_HASH)
+       ereport(ERROR,
+               (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+                errmsg("partition of hash-partitioned table cannot be merged")));
+
+   /* Is current partition a DEFAULT partition? */
+   defaultPartOid = get_default_oid_from_partdesc(
+                                                  RelationGetPartitionDesc(parent, true));
+
+   foreach(listptr, partcmd->partlist)
+   {
+       RangeVar   *name = (RangeVar *) lfirst(listptr);
+
+       /* Partitions in the list should have different names. */
+       for_each_cell(listptr2, partcmd->partlist, lnext(partcmd->partlist, listptr))
+       {
+           RangeVar   *name2 = (RangeVar *) lfirst(listptr2);
+
+           if (equal(name, name2))
+               ereport(ERROR,
+                       (errcode(ERRCODE_DUPLICATE_TABLE),
+                        errmsg("partition with name \"%s\" already used", name->relname)),
+                       parser_errposition(cxt->pstate, name2->location));
+       }
+
+       /* Search DEFAULT partition in the list. */
+       partOid = RangeVarGetRelid(name, NoLock, false);
+       if (partOid == defaultPartOid)
+           isDefaultPart = true;
+       partOids = lappend_oid(partOids, partOid);
+   }
+
+   /* Allocate bound of result partition. */
+   Assert(partcmd->bound == NULL);
+   partcmd->bound = makeNode(PartitionBoundSpec);
+
+   /* Fill partition bound. */
+   partcmd->bound->strategy = strategy;
+   partcmd->bound->location = -1;
+   partcmd->bound->is_default = isDefaultPart;
+   if (!isDefaultPart)
+       calculate_partition_bound_for_merge(parent, partcmd->partlist,
+                                           partOids, partcmd->bound,
+                                           cxt->pstate);
+}
+
 /*
  * transformAlterTableStmt -
  *     parse analysis for ALTER TABLE
@@ -3683,6 +3759,19 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
                newcmds = lappend(newcmds, cmd);
                break;
 
+           case AT_MergePartitions:
+               {
+                   PartitionCmd *partcmd = (PartitionCmd *) cmd->def;
+
+                   if (list_length(partcmd->partlist) < 2)
+                       ereport(ERROR,
+                               (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+                                errmsg("list of new partitions should contains at least two items")));
+                   transformPartitionCmdForMerge(&cxt, partcmd);
+                   newcmds = lappend(newcmds, cmd);
+                   break;
+               }
+
            default:
 
                /*
index c28639d2e3f4f91ac1f9d575d78ef08402549d24..9ed80d721edc2f4ff652501b4cf7b49e720e04fe 100644 (file)
@@ -4977,3 +4977,210 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
    PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * check_two_partitions_bounds_range
+ *
+ * (function for BY RANGE partitioning)
+ *
+ * This is a helper function for check_partitions_for_split() and
+ * calculate_partition_bound_for_merge().
+ * This function compares upper bound of first_bound and lower bound of
+ * second_bound. These bounds should be equals except case
+ * "defaultPart == true" (this means that one of split partitions is DEFAULT).
+ * In this case upper bound of first_bound can be less than lower bound of
+ * second_bound because space between of these bounds will be included in
+ * DEFAULT partition.
+ *
+ * parent:         partitioned table
+ * first_name:     name of first partition
+ * first_bound:        bound of first partition
+ * second_name:        name of second partition
+ * second_bound:   bound of second partition
+ * defaultPart:        true if one of split partitions is DEFAULT
+ * pstate:         pointer to ParseState struct for determine error position
+ */
+static void
+check_two_partitions_bounds_range(Relation parent,
+                                 RangeVar *first_name,
+                                 PartitionBoundSpec *first_bound,
+                                 RangeVar *second_name,
+                                 PartitionBoundSpec *second_bound,
+                                 bool defaultPart,
+                                 ParseState *pstate)
+{
+   PartitionKey key = RelationGetPartitionKey(parent);
+   PartitionRangeBound *first_upper;
+   PartitionRangeBound *second_lower;
+   int         cmpval;
+
+   Assert(key->strategy == PARTITION_STRATEGY_RANGE);
+
+   first_upper = make_one_partition_rbound(key, -1, first_bound->upperdatums, false);
+   second_lower = make_one_partition_rbound(key, -1, second_bound->lowerdatums, true);
+
+   /*
+    * lower1=false (the second to last argument) for correct comparison lower
+    * and upper bounds.
+    */
+   cmpval = partition_rbound_cmp(key->partnatts,
+                                 key->partsupfunc,
+                                 key->partcollation,
+                                 second_lower->datums, second_lower->kind,
+                                 false, first_upper);
+   if ((!defaultPart && cmpval) || (defaultPart && cmpval < 0))
+   {
+       PartitionRangeDatum *datum = linitial(second_bound->lowerdatums);
+
+       ereport(ERROR,
+               (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+                errmsg("lower bound of partition \"%s\" conflicts with upper bound of previous partition \"%s\"",
+                       second_name->relname, first_name->relname),
+                parser_errposition(pstate, datum->location)));
+   }
+}
+
+/*
+ * get_partition_bound_spec
+ *
+ * Returns description of partition with Oid "partOid" and name "name".
+ */
+static PartitionBoundSpec *
+get_partition_bound_spec(Oid partOid, RangeVar *name)
+{
+   HeapTuple   tuple;
+   Datum       datum;
+   bool        isnull;
+   PartitionBoundSpec *boundspec = NULL;
+
+   /* Try fetching the tuple from the catcache, for speed. */
+   tuple = SearchSysCache1(RELOID, partOid);
+   if (!HeapTupleIsValid(tuple))
+       elog(ERROR, "cache lookup failed for relation \"%s\"",
+            name->relname);
+
+   datum = SysCacheGetAttr(RELOID, tuple,
+                           Anum_pg_class_relpartbound,
+                           &isnull);
+   if (isnull)
+       elog(ERROR, "partition bound for relation \"%s\" is null",
+            name->relname);
+
+   boundspec = stringToNode(TextDatumGetCString(datum));
+
+   if (!IsA(boundspec, PartitionBoundSpec))
+       elog(ERROR, "expected PartitionBoundSpec for relation \"%s\"",
+            name->relname);
+
+   ReleaseSysCache(tuple);
+   return boundspec;
+}
+
+/*
+ * calculate_partition_bound_for_merge
+ *
+ * Calculates the bound of merged partition "spec" by using the bounds of
+ * partitions to be merged.
+ *
+ * parent:         partitioned table
+ * partNames:      names of partitions to be merged
+ * partOids:       Oids of partitions to be merged
+ * spec (out):     bounds specification of the merged partition
+ * pstate:         pointer to ParseState struct for determine error position
+ */
+void
+calculate_partition_bound_for_merge(Relation parent,
+                                   List *partNames,
+                                   List *partOids,
+                                   PartitionBoundSpec *spec,
+                                   ParseState *pstate)
+{
+   PartitionKey key = RelationGetPartitionKey(parent);
+   PartitionBoundSpec *bound;
+
+   Assert(!spec->is_default);
+
+   switch (key->strategy)
+   {
+       case PARTITION_STRATEGY_RANGE:
+           {
+               int         i;
+               PartitionRangeBound **lower_bounds;
+               int         nparts = list_length(partOids);
+               List       *bounds = NIL;
+
+               lower_bounds = (PartitionRangeBound **)
+                   palloc0(nparts * sizeof(PartitionRangeBound *));
+
+               /*
+                * Create array of lower bounds and list of
+                * PartitionBoundSpec.
+                */
+               for (i = 0; i < nparts; i++)
+               {
+                   bound = get_partition_bound_spec(list_nth_oid(partOids, i),
+                                                    (RangeVar *) list_nth(partNames, i));
+
+                   lower_bounds[i] = make_one_partition_rbound(key, i, bound->lowerdatums, true);
+                   bounds = lappend(bounds, bound);
+               }
+
+               /* Sort array of lower bounds. */
+               qsort_arg(lower_bounds, nparts, sizeof(PartitionRangeBound *),
+                         qsort_partition_rbound_cmp, (void *) key);
+
+               /* Ranges of partitions should not overlap. */
+               for (i = 1; i < nparts; i++)
+               {
+                   int         index = lower_bounds[i]->index;
+                   int         prev_index = lower_bounds[i - 1]->index;
+
+                   check_two_partitions_bounds_range(parent,
+                                                     (RangeVar *) list_nth(partNames, prev_index),
+                                                     (PartitionBoundSpec *) list_nth(bounds, prev_index),
+                                                     (RangeVar *) list_nth(partNames, index),
+                                                     (PartitionBoundSpec *) list_nth(bounds, index),
+                                                     false, pstate);
+               }
+
+               /*
+                * Lower bound of first partition is a lower bound of merged
+                * partition.
+                */
+               spec->lowerdatums =
+                   ((PartitionBoundSpec *) list_nth(bounds, lower_bounds[0]->index))->lowerdatums;
+
+               /*
+                * Upper bound of last partition is a upper bound of merged
+                * partition.
+                */
+               spec->upperdatums =
+                   ((PartitionBoundSpec *) list_nth(bounds, lower_bounds[nparts - 1]->index))->upperdatums;
+
+               pfree(lower_bounds);
+               list_free(bounds);
+               break;
+           }
+
+       case PARTITION_STRATEGY_LIST:
+           {
+               ListCell   *listptr,
+                          *listptr2;
+
+               /* Consolidate bounds for all partitions in the list. */
+               forboth(listptr, partOids, listptr2, partNames)
+               {
+                   RangeVar   *name = (RangeVar *) lfirst(listptr2);
+                   Oid         curOid = lfirst_oid(listptr);
+
+                   bound = get_partition_bound_spec(curOid, name);
+                   spec->listdatums = list_concat(spec->listdatums, bound->listdatums);
+               }
+               break;
+           }
+
+       default:
+           elog(ERROR, "unexpected partition strategy: %d",
+                (int) key->strategy);
+   }
+}
index 76d91e547b76cc1959145e259f14a01e1c8e5b51..6e8a2b59e8337ca8b2ce42922dca8d7544f240c2 100644 (file)
@@ -937,6 +937,17 @@ typedef struct PartitionRangeDatum
    ParseLoc    location;       /* token location, or -1 if unknown */
 } PartitionRangeDatum;
 
+/*
+ * PartitionDesc - info about single partition for ALTER TABLE SPLIT PARTITION command
+ */
+typedef struct SinglePartitionSpec
+{
+   NodeTag     type;
+
+   RangeVar   *name;           /* name of partition */
+   PartitionBoundSpec *bound;  /* FOR VALUES, if attaching */
+} SinglePartitionSpec;
+
 /*
  * PartitionCmd - info for ALTER TABLE/INDEX ATTACH/DETACH PARTITION commands
  */
@@ -945,6 +956,8 @@ typedef struct PartitionCmd
    NodeTag     type;
    RangeVar   *name;           /* name of partition to attach/detach */
    PartitionBoundSpec *bound;  /* FOR VALUES, if attaching */
+   List       *partlist;       /* list of partitions, for SPLIT PARTITION
+                                * command */
    bool        concurrent;
 } PartitionCmd;
 
@@ -2398,6 +2411,7 @@ typedef enum AlterTableType
    AT_AttachPartition,         /* ATTACH PARTITION */
    AT_DetachPartition,         /* DETACH PARTITION */
    AT_DetachPartitionFinalize, /* DETACH PARTITION FINALIZE */
+   AT_MergePartitions,         /* MERGE PARTITIONS */
    AT_AddIdentity,             /* ADD IDENTITY */
    AT_SetIdentity,             /* SET identity column options */
    AT_DropIdentity,            /* DROP IDENTITY */
index 2d4a0c6a07906dc42b36ab6b01bbdcfdc6d2f135..a4834eb9b7de58314565d85b9170383b6d3c96c6 100644 (file)
@@ -334,6 +334,7 @@ PG_KEYWORD("parameter", PARAMETER, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD, BARE_LABEL)
+PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD, BARE_LABEL)
 PG_KEYWORD("path", PATH, UNRESERVED_KEYWORD, BARE_LABEL)
index 3d9cc1031f7319b1b16a1d06027f73e273299d86..0329d7bd848c882e49c2b341457d827b36e568f7 100644 (file)
@@ -143,4 +143,10 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
                                   int modulus, int remainder);
 
+extern void calculate_partition_bound_for_merge(Relation parent,
+                                               List *partNames,
+                                               List *partOids,
+                                               PartitionBoundSpec *spec,
+                                               ParseState *pstate);
+
 #endif                         /* PARTBOUNDS_H */
diff --git a/src/test/isolation/expected/partition-merge.out b/src/test/isolation/expected/partition-merge.out
new file mode 100644 (file)
index 0000000..98446aa
--- /dev/null
@@ -0,0 +1,199 @@
+Parsed test spec with 2 sessions
+
+starting permutation: s2b s2i s2c s1b s1merg s2b s2u s1c s2c s2s
+step s2b: BEGIN;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1b: BEGIN;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2b s2i s2c s1brr s1merg s2b s2u s1c s2c s2s
+step s2b: BEGIN;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2b s2i s2c s1bs s1merg s2b s2u s1c s2c s2s
+step s2b: BEGIN;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1bs: BEGIN ISOLATION LEVEL SERIALIZABLE;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2brr s2i s2c s1b s1merg s2b s2u s1c s2c s2s
+step s2brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1b: BEGIN;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2brr s2i s2c s1brr s1merg s2b s2u s1c s2c s2s
+step s2brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2brr s2i s2c s1bs s1merg s2b s2u s1c s2c s2s
+step s2brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1bs: BEGIN ISOLATION LEVEL SERIALIZABLE;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2bs s2i s2c s1b s1merg s2b s2u s1c s2c s2s
+step s2bs: BEGIN ISOLATION LEVEL SERIALIZABLE;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1b: BEGIN;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2bs s2i s2c s1brr s1merg s2b s2u s1c s2c s2s
+step s2bs: BEGIN ISOLATION LEVEL SERIALIZABLE;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
+
+starting permutation: s2bs s2i s2c s1bs s1merg s2b s2u s1c s2c s2s
+step s2bs: BEGIN ISOLATION LEVEL SERIALIZABLE;
+step s2i: INSERT INTO tpart VALUES (1, 'text01');
+step s2c: COMMIT;
+step s1bs: BEGIN ISOLATION LEVEL SERIALIZABLE;
+step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20;
+step s2b: BEGIN;
+step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; <waiting ...>
+step s1c: COMMIT;
+step s2u: <... completed>
+step s2c: COMMIT;
+step s2s: SELECT * FROM tpart;
+ i|t          
+--+-----------
+ 5|text05     
+15|text15     
+ 1|text01modif
+25|text25     
+35|text35     
+(5 rows)
+
index 188fc04f85e062b51839e9288d1060cce0016f9c..8bcaa8a6254a0b205e6a5f3ddb7de1dcc60410b4 100644 (file)
@@ -103,6 +103,7 @@ test: partition-key-update-1
 test: partition-key-update-2
 test: partition-key-update-3
 test: partition-key-update-4
+test: partition-merge
 test: plpgsql-toast
 test: cluster-conflict
 test: cluster-conflict-partition
diff --git a/src/test/isolation/specs/partition-merge.spec b/src/test/isolation/specs/partition-merge.spec
new file mode 100644 (file)
index 0000000..ec48732
--- /dev/null
@@ -0,0 +1,54 @@
+# Verify that MERGE operation locks DML operations with partitioned table
+
+setup
+{
+  DROP TABLE IF EXISTS tpart;
+  CREATE TABLE tpart(i int, t text) partition by range(i);
+  CREATE TABLE tpart_00_10 PARTITION OF tpart FOR VALUES FROM (0) TO (10);
+  CREATE TABLE tpart_10_20 PARTITION OF tpart FOR VALUES FROM (10) TO (20);
+  CREATE TABLE tpart_20_30 PARTITION OF tpart FOR VALUES FROM (20) TO (30);
+  CREATE TABLE tpart_default PARTITION OF tpart DEFAULT;
+  INSERT INTO tpart VALUES (5, 'text05');
+  INSERT INTO tpart VALUES (15, 'text15');
+  INSERT INTO tpart VALUES (25, 'text25');
+  INSERT INTO tpart VALUES (35, 'text35');
+}
+
+teardown
+{
+  DROP TABLE tpart;
+}
+
+session s1
+step s1b   { BEGIN; }
+step s1brr { BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step s1bs  { BEGIN ISOLATION LEVEL SERIALIZABLE; }
+step s1merg    { ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20; }
+step s1c   { COMMIT; }
+
+
+session s2
+step s2b   { BEGIN; }
+step s2brr { BEGIN ISOLATION LEVEL REPEATABLE READ; }
+step s2bs  { BEGIN ISOLATION LEVEL SERIALIZABLE; }
+step s2i   { INSERT INTO tpart VALUES (1, 'text01'); }
+step s2u   { UPDATE tpart SET t = 'text01modif' where i = 1; }
+step s2c   { COMMIT; }
+step s2s   { SELECT * FROM tpart; }
+
+
+# s2 inserts row into table. s1 starts MERGE PARTITIONS then
+# s2 trying to update inserted row and waits until s1 finished
+# MERGE operation.
+
+permutation s2b s2i s2c s1b s1merg s2b s2u s1c s2c s2s
+permutation s2b s2i s2c s1brr s1merg s2b s2u s1c s2c s2s
+permutation s2b s2i s2c s1bs s1merg s2b s2u s1c s2c s2s
+
+permutation s2brr s2i s2c s1b s1merg s2b s2u s1c s2c s2s
+permutation s2brr s2i s2c s1brr s1merg s2b s2u s1c s2c s2s
+permutation s2brr s2i s2c s1bs s1merg s2b s2u s1c s2c s2s
+
+permutation s2bs s2i s2c s1b s1merg s2b s2u s1c s2c s2s
+permutation s2bs s2i s2c s1brr s1merg s2b s2u s1c s2c s2s
+permutation s2bs s2i s2c s1bs s1merg s2b s2u s1c s2c s2s
index 48563b2cf0158306419a7a05f18a5497d9df0468..69920757b21598adde3904a157f75a8f8b64bd14 100644 (file)
@@ -300,6 +300,9 @@ get_altertable_subcmdinfo(PG_FUNCTION_ARGS)
            case AT_DetachPartitionFinalize:
                strtype = "DETACH PARTITION ... FINALIZE";
                break;
+           case AT_MergePartitions:
+               strtype = "MERGE PARTITIONS";
+               break;
            case AT_AddIdentity:
                strtype = "ADD IDENTITY";
                break;
diff --git a/src/test/regress/expected/partition_merge.out b/src/test/regress/expected/partition_merge.out
new file mode 100644 (file)
index 0000000..2ba0ec4
--- /dev/null
@@ -0,0 +1,732 @@
+--
+-- PARTITIONS_MERGE
+-- Tests for "ALTER TABLE ... MERGE PARTITIONS ..." command
+--
+CREATE SCHEMA partitions_merge_schema;
+SET search_path = partitions_merge_schema, public;
+--
+-- BY RANGE partitioning
+--
+--
+-- Test for error codes
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_dec2021 PARTITION OF sales_range FOR VALUES FROM ('2021-12-01') TO ('2021-12-31');
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+CREATE TABLE sales_apr2022 (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_apr_1 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-04-15');
+CREATE TABLE sales_apr_2 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-15') TO ('2022-05-01');
+ALTER TABLE sales_range ATTACH PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+-- ERROR:  partition with name "sales_feb2022" already used
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_feb2022) INTO sales_feb_mar_apr2022;
+ERROR:  partition with name "sales_feb2022" already used
+LINE 1: ...e MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_feb2...
+                                                             ^
+-- ERROR:  cannot merge non-table partition "sales_apr2022"
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO sales_feb_mar_apr2022;
+ERROR:  cannot merge non-table partition "sales_apr2022"
+-- ERROR:  invalid partitions order, partition "sales_mar2022" can not be merged
+-- (space between sections sales_jan2022 and sales_mar2022)
+ALTER TABLE sales_range MERGE PARTITIONS (sales_jan2022, sales_mar2022) INTO sales_jan_mar2022;
+ERROR:  lower bound of partition "sales_mar2022" conflicts with upper bound of previous partition "sales_jan2022"
+-- ERROR:  invalid partitions order, partition "sales_jan2022" can not be merged
+-- (space between sections sales_dec2021 and sales_jan2022)
+ALTER TABLE sales_range MERGE PARTITIONS (sales_dec2021, sales_jan2022, sales_feb2022) INTO sales_dec_jan_feb2022;
+ERROR:  lower bound of partition "sales_jan2022" conflicts with upper bound of previous partition "sales_dec2021"
+-- NO ERROR: test for custom partitions order
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_jan2022) INTO sales_jan_feb_mar2022;
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+          oid          | relkind | inhdetachpending |                   pg_get_expr                    
+-----------------------+---------+------------------+--------------------------------------------------
+ sales_apr2022         | p       | f                | FOR VALUES FROM ('04-01-2022') TO ('05-01-2022')
+ sales_dec2021         | r       | f                | FOR VALUES FROM ('12-01-2021') TO ('12-31-2021')
+ sales_jan_feb_mar2022 | r       | f                | FOR VALUES FROM ('01-01-2022') TO ('04-01-2022')
+ sales_others          | r       | f                | DEFAULT
+(4 rows)
+
+DROP TABLE sales_range;
+--
+-- Add rows into partitioned table then merge partitions
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+CREATE TABLE sales_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+CREATE INDEX sales_range_sales_date_idx ON sales_range USING btree (sales_date);
+INSERT INTO sales_range VALUES (1,  'May',      1000, '2022-01-31');
+INSERT INTO sales_range VALUES (2,  'Smirnoff', 500,  '2022-02-10');
+INSERT INTO sales_range VALUES (3,  'Ford',     2000, '2022-04-30');
+INSERT INTO sales_range VALUES (4,  'Ivanov',   750,  '2022-04-13');
+INSERT INTO sales_range VALUES (5,  'Deev',     250,  '2022-04-07');
+INSERT INTO sales_range VALUES (6,  'Poirot',   150,  '2022-02-11');
+INSERT INTO sales_range VALUES (7,  'Li',       175,  '2022-03-08');
+INSERT INTO sales_range VALUES (8,  'Ericsson', 185,  '2022-02-23');
+INSERT INTO sales_range VALUES (9,  'Muller',   250,  '2022-03-11');
+INSERT INTO sales_range VALUES (10, 'Halder',   350,  '2022-01-28');
+INSERT INTO sales_range VALUES (11, 'Trump',    380,  '2022-04-06');
+INSERT INTO sales_range VALUES (12, 'Plato',    350,  '2022-03-19');
+INSERT INTO sales_range VALUES (13, 'Gandi',    377,  '2022-01-09');
+INSERT INTO sales_range VALUES (14, 'Smith',    510,  '2022-05-04');
+SELECT pg_catalog.pg_get_partkeydef('sales_range'::regclass);
+ pg_get_partkeydef  
+--------------------
+ RANGE (sales_date)
+(1 row)
+
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+      oid      | relkind | inhdetachpending |                   pg_get_expr                    
+---------------+---------+------------------+--------------------------------------------------
+ sales_apr2022 | r       | f                | FOR VALUES FROM ('04-01-2022') TO ('05-01-2022')
+ sales_feb2022 | r       | f                | FOR VALUES FROM ('02-01-2022') TO ('03-01-2022')
+ sales_jan2022 | r       | f                | FOR VALUES FROM ('01-01-2022') TO ('02-01-2022')
+ sales_mar2022 | r       | f                | FOR VALUES FROM ('03-01-2022') TO ('04-01-2022')
+ sales_others  | r       | f                | DEFAULT
+(5 rows)
+
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO sales_feb_mar_apr2022;
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+          oid          | relkind | inhdetachpending |                   pg_get_expr                    
+-----------------------+---------+------------------+--------------------------------------------------
+ sales_feb_mar_apr2022 | r       | f                | FOR VALUES FROM ('02-01-2022') TO ('05-01-2022')
+ sales_jan2022         | r       | f                | FOR VALUES FROM ('01-01-2022') TO ('02-01-2022')
+ sales_others          | r       | f                | DEFAULT
+(3 rows)
+
+SELECT * FROM pg_indexes WHERE tablename = 'sales_feb_mar_apr2022' and schemaname = 'partitions_merge_schema';
+       schemaname        |       tablename       |              indexname               | tablespace |                                                          indexdef                                                           
+-------------------------+-----------------------+--------------------------------------+------------+-----------------------------------------------------------------------------------------------------------------------------
+ partitions_merge_schema | sales_feb_mar_apr2022 | sales_feb_mar_apr2022_sales_date_idx |            | CREATE INDEX sales_feb_mar_apr2022_sales_date_idx ON partitions_merge_schema.sales_feb_mar_apr2022 USING btree (sales_date)
+(1 row)
+
+SELECT * FROM sales_range;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           1 | May           |         1000 | 01-31-2022
+          10 | Halder        |          350 | 01-28-2022
+          13 | Gandi         |          377 | 01-09-2022
+           2 | Smirnoff      |          500 | 02-10-2022
+           6 | Poirot        |          150 | 02-11-2022
+           8 | Ericsson      |          185 | 02-23-2022
+           7 | Li            |          175 | 03-08-2022
+           9 | Muller        |          250 | 03-11-2022
+          12 | Plato         |          350 | 03-19-2022
+           3 | Ford          |         2000 | 04-30-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+          14 | Smith         |          510 | 05-04-2022
+(14 rows)
+
+SELECT * FROM sales_jan2022;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           1 | May           |         1000 | 01-31-2022
+          10 | Halder        |          350 | 01-28-2022
+          13 | Gandi         |          377 | 01-09-2022
+(3 rows)
+
+SELECT * FROM sales_feb_mar_apr2022;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           2 | Smirnoff      |          500 | 02-10-2022
+           6 | Poirot        |          150 | 02-11-2022
+           8 | Ericsson      |          185 | 02-23-2022
+           7 | Li            |          175 | 03-08-2022
+           9 | Muller        |          250 | 03-11-2022
+          12 | Plato         |          350 | 03-19-2022
+           3 | Ford          |         2000 | 04-30-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+(10 rows)
+
+SELECT * FROM sales_others;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+          14 | Smith         |          510 | 05-04-2022
+(1 row)
+
+-- Use indexscan for test indexes
+SET enable_seqscan = OFF;
+SELECT * FROM sales_feb_mar_apr2022 where sales_date > '2022-01-01';
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           2 | Smirnoff      |          500 | 02-10-2022
+           6 | Poirot        |          150 | 02-11-2022
+           8 | Ericsson      |          185 | 02-23-2022
+           7 | Li            |          175 | 03-08-2022
+           9 | Muller        |          250 | 03-11-2022
+          12 | Plato         |          350 | 03-19-2022
+          11 | Trump         |          380 | 04-06-2022
+           5 | Deev          |          250 | 04-07-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           3 | Ford          |         2000 | 04-30-2022
+(10 rows)
+
+RESET enable_seqscan;
+DROP TABLE sales_range;
+--
+-- Merge some partitions into DEFAULT partition
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+CREATE TABLE sales_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+CREATE INDEX sales_range_sales_date_idx ON sales_range USING btree (sales_date);
+INSERT INTO sales_range VALUES (1,  'May',      1000, '2022-01-31');
+INSERT INTO sales_range VALUES (2,  'Smirnoff', 500,  '2022-02-10');
+INSERT INTO sales_range VALUES (3,  'Ford',     2000, '2022-04-30');
+INSERT INTO sales_range VALUES (4,  'Ivanov',   750,  '2022-04-13');
+INSERT INTO sales_range VALUES (5,  'Deev',     250,  '2022-04-07');
+INSERT INTO sales_range VALUES (6,  'Poirot',   150,  '2022-02-11');
+INSERT INTO sales_range VALUES (7,  'Li',       175,  '2022-03-08');
+INSERT INTO sales_range VALUES (8,  'Ericsson', 185,  '2022-02-23');
+INSERT INTO sales_range VALUES (9,  'Muller',   250,  '2022-03-11');
+INSERT INTO sales_range VALUES (10, 'Halder',   350,  '2022-01-28');
+INSERT INTO sales_range VALUES (11, 'Trump',    380,  '2022-04-06');
+INSERT INTO sales_range VALUES (12, 'Plato',    350,  '2022-03-19');
+INSERT INTO sales_range VALUES (13, 'Gandi',    377,  '2022-01-09');
+INSERT INTO sales_range VALUES (14, 'Smith',    510,  '2022-05-04');
+-- Merge partitions (include DEFAULT partition) into partition with the same
+-- name
+ALTER TABLE sales_range MERGE PARTITIONS (sales_jan2022, sales_mar2022, sales_others) INTO sales_others;
+select * from sales_others;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           1 | May           |         1000 | 01-31-2022
+          10 | Halder        |          350 | 01-28-2022
+          13 | Gandi         |          377 | 01-09-2022
+           7 | Li            |          175 | 03-08-2022
+           9 | Muller        |          250 | 03-11-2022
+          12 | Plato         |          350 | 03-19-2022
+          14 | Smith         |          510 | 05-04-2022
+(7 rows)
+
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+      oid      | relkind | inhdetachpending |                   pg_get_expr                    
+---------------+---------+------------------+--------------------------------------------------
+ sales_apr2022 | r       | f                | FOR VALUES FROM ('04-01-2022') TO ('05-01-2022')
+ sales_feb2022 | r       | f                | FOR VALUES FROM ('02-01-2022') TO ('03-01-2022')
+ sales_others  | r       | f                | DEFAULT
+(3 rows)
+
+DROP TABLE sales_range;
+--
+-- Test for:
+--   * composite partition key;
+--   * GENERATED column;
+--   * column with DEFAULT value.
+--
+CREATE TABLE sales_date (salesman_name VARCHAR(30), sales_year INT, sales_month INT, sales_day INT,
+  sales_date VARCHAR(10) GENERATED ALWAYS AS
+    (LPAD(sales_year::text, 4, '0') || '.' || LPAD(sales_month::text, 2, '0') || '.' || LPAD(sales_day::text, 2, '0')) STORED,
+  sales_department VARCHAR(30) DEFAULT 'Sales department')
+  PARTITION BY RANGE (sales_year, sales_month, sales_day);
+CREATE TABLE sales_dec2022 PARTITION OF sales_date FOR VALUES FROM (2021, 12, 1) TO (2022, 1, 1);
+CREATE TABLE sales_jan2022 PARTITION OF sales_date FOR VALUES FROM (2022, 1, 1) TO (2022, 2, 1);
+CREATE TABLE sales_feb2022 PARTITION OF sales_date FOR VALUES FROM (2022, 2, 1) TO (2022, 3, 1);
+CREATE TABLE sales_other PARTITION OF sales_date FOR VALUES FROM (2022, 3, 1) TO (MAXVALUE, MAXVALUE, MAXVALUE);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2021, 12, 7);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2021, 12, 8);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 1, 1);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 2, 4);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 1, 2);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 2, 1);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 3, 3);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 3, 4);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 5, 1);
+SELECT * FROM sales_date;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager1      |       2021 |          12 |         7 | 2021.12.07 | Sales department
+ Manager2      |       2021 |          12 |         8 | 2021.12.08 | Sales department
+ Manager3      |       2022 |           1 |         1 | 2022.01.01 | Sales department
+ Manager2      |       2022 |           1 |         2 | 2022.01.02 | Sales department
+ Manager1      |       2022 |           2 |         4 | 2022.02.04 | Sales department
+ Manager3      |       2022 |           2 |         1 | 2022.02.01 | Sales department
+ Manager1      |       2022 |           3 |         3 | 2022.03.03 | Sales department
+ Manager2      |       2022 |           3 |         4 | 2022.03.04 | Sales department
+ Manager3      |       2022 |           5 |         1 | 2022.05.01 | Sales department
+(9 rows)
+
+SELECT * FROM sales_dec2022;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager1      |       2021 |          12 |         7 | 2021.12.07 | Sales department
+ Manager2      |       2021 |          12 |         8 | 2021.12.08 | Sales department
+(2 rows)
+
+SELECT * FROM sales_jan2022;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager3      |       2022 |           1 |         1 | 2022.01.01 | Sales department
+ Manager2      |       2022 |           1 |         2 | 2022.01.02 | Sales department
+(2 rows)
+
+SELECT * FROM sales_feb2022;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager1      |       2022 |           2 |         4 | 2022.02.04 | Sales department
+ Manager3      |       2022 |           2 |         1 | 2022.02.01 | Sales department
+(2 rows)
+
+SELECT * FROM sales_other;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager1      |       2022 |           3 |         3 | 2022.03.03 | Sales department
+ Manager2      |       2022 |           3 |         4 | 2022.03.04 | Sales department
+ Manager3      |       2022 |           5 |         1 | 2022.05.01 | Sales department
+(3 rows)
+
+ALTER TABLE sales_date MERGE PARTITIONS (sales_jan2022, sales_feb2022) INTO sales_jan_feb2022;
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 1, 10);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 2, 10);
+SELECT * FROM sales_date;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager1      |       2021 |          12 |         7 | 2021.12.07 | Sales department
+ Manager2      |       2021 |          12 |         8 | 2021.12.08 | Sales department
+ Manager3      |       2022 |           1 |         1 | 2022.01.01 | Sales department
+ Manager2      |       2022 |           1 |         2 | 2022.01.02 | Sales department
+ Manager1      |       2022 |           2 |         4 | 2022.02.04 | Sales department
+ Manager3      |       2022 |           2 |         1 | 2022.02.01 | Sales department
+ Manager1      |       2022 |           1 |        10 | 2022.01.10 | Sales department
+ Manager2      |       2022 |           2 |        10 | 2022.02.10 | Sales department
+ Manager1      |       2022 |           3 |         3 | 2022.03.03 | Sales department
+ Manager2      |       2022 |           3 |         4 | 2022.03.04 | Sales department
+ Manager3      |       2022 |           5 |         1 | 2022.05.01 | Sales department
+(11 rows)
+
+SELECT * FROM sales_dec2022;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager1      |       2021 |          12 |         7 | 2021.12.07 | Sales department
+ Manager2      |       2021 |          12 |         8 | 2021.12.08 | Sales department
+(2 rows)
+
+SELECT * FROM sales_jan_feb2022;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager3      |       2022 |           1 |         1 | 2022.01.01 | Sales department
+ Manager2      |       2022 |           1 |         2 | 2022.01.02 | Sales department
+ Manager1      |       2022 |           2 |         4 | 2022.02.04 | Sales department
+ Manager3      |       2022 |           2 |         1 | 2022.02.01 | Sales department
+ Manager1      |       2022 |           1 |        10 | 2022.01.10 | Sales department
+ Manager2      |       2022 |           2 |        10 | 2022.02.10 | Sales department
+(6 rows)
+
+SELECT * FROM sales_other;
+ salesman_name | sales_year | sales_month | sales_day | sales_date | sales_department 
+---------------+------------+-------------+-----------+------------+------------------
+ Manager1      |       2022 |           3 |         3 | 2022.03.03 | Sales department
+ Manager2      |       2022 |           3 |         4 | 2022.03.04 | Sales department
+ Manager3      |       2022 |           5 |         1 | 2022.05.01 | Sales department
+(3 rows)
+
+DROP TABLE sales_date;
+--
+-- Test: merge partitions of partitioned table with triggers
+--
+CREATE TABLE salesmans(salesman_id INT PRIMARY KEY, salesman_name VARCHAR(30)) PARTITION BY RANGE (salesman_id);
+CREATE TABLE salesmans01_10 PARTITION OF salesmans FOR VALUES FROM (1) TO (10);
+CREATE TABLE salesmans10_20 PARTITION OF salesmans FOR VALUES FROM (10) TO (20);
+CREATE TABLE salesmans20_30 PARTITION OF salesmans FOR VALUES FROM (20) TO (30);
+CREATE TABLE salesmans30_40 PARTITION OF salesmans FOR VALUES FROM (30) TO (40);
+INSERT INTO salesmans VALUES (1,  'Poirot');
+CREATE OR REPLACE FUNCTION after_insert_row_trigger() RETURNS trigger LANGUAGE 'plpgsql' AS $BODY$
+BEGIN
+   RAISE NOTICE 'trigger(%) called: action = %, when = %, level = %', TG_ARGV[0], TG_OP, TG_WHEN, TG_LEVEL;
+    RETURN NULL;
+END;
+$BODY$;
+CREATE TRIGGER salesmans_after_insert_statement_trigger
+    AFTER INSERT
+    ON salesmans
+    FOR EACH STATEMENT
+    EXECUTE PROCEDURE after_insert_row_trigger('salesmans');
+CREATE TRIGGER salesmans_after_insert_row_trigger
+    AFTER INSERT
+    ON salesmans
+    FOR EACH ROW
+    EXECUTE PROCEDURE after_insert_row_trigger('salesmans');
+-- 2 triggers should fire here (row + statement):
+INSERT INTO salesmans VALUES (10, 'May');
+NOTICE:  trigger(salesmans) called: action = INSERT, when = AFTER, level = ROW
+NOTICE:  trigger(salesmans) called: action = INSERT, when = AFTER, level = STATEMENT
+-- 1 trigger should fire here (row):
+INSERT INTO salesmans10_20 VALUES (19, 'Ivanov');
+NOTICE:  trigger(salesmans) called: action = INSERT, when = AFTER, level = ROW
+ALTER TABLE salesmans MERGE PARTITIONS (salesmans10_20, salesmans20_30, salesmans30_40) INTO salesmans10_40;
+-- 2 triggers should fire here (row + statement):
+INSERT INTO salesmans VALUES (20, 'Smirnoff');
+NOTICE:  trigger(salesmans) called: action = INSERT, when = AFTER, level = ROW
+NOTICE:  trigger(salesmans) called: action = INSERT, when = AFTER, level = STATEMENT
+-- 1 trigger should fire here (row):
+INSERT INTO salesmans10_40 VALUES (30, 'Ford');
+NOTICE:  trigger(salesmans) called: action = INSERT, when = AFTER, level = ROW
+SELECT * FROM salesmans01_10;
+ salesman_id | salesman_name 
+-------------+---------------
+           1 | Poirot
+(1 row)
+
+SELECT * FROM salesmans10_40;
+ salesman_id | salesman_name 
+-------------+---------------
+          10 | May
+          19 | Ivanov
+          20 | Smirnoff
+          30 | Ford
+(4 rows)
+
+DROP TABLE salesmans;
+DROP FUNCTION after_insert_row_trigger();
+--
+-- Test: merge partitions with deleted columns
+--
+CREATE TABLE salesmans(salesman_id INT PRIMARY KEY, salesman_name VARCHAR(30)) PARTITION BY RANGE (salesman_id);
+CREATE TABLE salesmans01_10 PARTITION OF salesmans FOR VALUES FROM (1) TO (10);
+-- Create partitions with some deleted columns:
+CREATE TABLE salesmans10_20(d1 VARCHAR(30), salesman_id INT PRIMARY KEY, salesman_name VARCHAR(30));
+CREATE TABLE salesmans20_30(salesman_id INT PRIMARY KEY, d2 INT,  salesman_name VARCHAR(30));
+CREATE TABLE salesmans30_40(salesman_id INT PRIMARY KEY, d3 DATE, salesman_name VARCHAR(30));
+INSERT INTO salesmans10_20 VALUES ('dummy value 1', 19, 'Ivanov');
+INSERT INTO salesmans20_30 VALUES (20, 101, 'Smirnoff');
+INSERT INTO salesmans30_40 VALUES (31, now(), 'Popov');
+ALTER TABLE salesmans10_20 DROP COLUMN d1;
+ALTER TABLE salesmans20_30 DROP COLUMN d2;
+ALTER TABLE salesmans30_40 DROP COLUMN d3;
+ALTER TABLE salesmans ATTACH PARTITION salesmans10_20 FOR VALUES FROM (10) TO (20);
+ALTER TABLE salesmans ATTACH PARTITION salesmans20_30 FOR VALUES FROM (20) TO (30);
+ALTER TABLE salesmans ATTACH PARTITION salesmans30_40 FOR VALUES FROM (30) TO (40);
+INSERT INTO salesmans VALUES (1, 'Poirot');
+INSERT INTO salesmans VALUES (10, 'May');
+INSERT INTO salesmans VALUES (30, 'Ford');
+ALTER TABLE salesmans MERGE PARTITIONS (salesmans10_20, salesmans20_30, salesmans30_40) INTO salesmans10_40;
+select * from salesmans;
+ salesman_id | salesman_name 
+-------------+---------------
+           1 | Poirot
+          19 | Ivanov
+          10 | May
+          20 | Smirnoff
+          31 | Popov
+          30 | Ford
+(6 rows)
+
+select * from salesmans01_10;
+ salesman_id | salesman_name 
+-------------+---------------
+           1 | Poirot
+(1 row)
+
+select * from salesmans10_40;
+ salesman_id | salesman_name 
+-------------+---------------
+          19 | Ivanov
+          10 | May
+          20 | Smirnoff
+          31 | Popov
+          30 | Ford
+(5 rows)
+
+DROP TABLE salesmans;
+--
+-- Test: merge sub-partitions
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+CREATE TABLE sales_apr2022 (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_apr2022_01_10 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-04-10');
+CREATE TABLE sales_apr2022_10_20 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-10') TO ('2022-04-20');
+CREATE TABLE sales_apr2022_20_30 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-20') TO ('2022-05-01');
+ALTER TABLE sales_range ATTACH PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+CREATE INDEX sales_range_sales_date_idx ON sales_range USING btree (sales_date);
+INSERT INTO sales_range VALUES (1,  'May',      1000, '2022-01-31');
+INSERT INTO sales_range VALUES (2,  'Smirnoff', 500,  '2022-02-10');
+INSERT INTO sales_range VALUES (3,  'Ford',     2000, '2022-04-30');
+INSERT INTO sales_range VALUES (4,  'Ivanov',   750,  '2022-04-13');
+INSERT INTO sales_range VALUES (5,  'Deev',     250,  '2022-04-07');
+INSERT INTO sales_range VALUES (6,  'Poirot',   150,  '2022-02-11');
+INSERT INTO sales_range VALUES (7,  'Li',       175,  '2022-03-08');
+INSERT INTO sales_range VALUES (8,  'Ericsson', 185,  '2022-02-23');
+INSERT INTO sales_range VALUES (9,  'Muller',   250,  '2022-03-11');
+INSERT INTO sales_range VALUES (10, 'Halder',   350,  '2022-01-28');
+INSERT INTO sales_range VALUES (11, 'Trump',    380,  '2022-04-06');
+INSERT INTO sales_range VALUES (12, 'Plato',    350,  '2022-03-19');
+INSERT INTO sales_range VALUES (13, 'Gandi',    377,  '2022-01-09');
+INSERT INTO sales_range VALUES (14, 'Smith',    510,  '2022-05-04');
+SELECT * FROM sales_range;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           1 | May           |         1000 | 01-31-2022
+          10 | Halder        |          350 | 01-28-2022
+          13 | Gandi         |          377 | 01-09-2022
+           2 | Smirnoff      |          500 | 02-10-2022
+           6 | Poirot        |          150 | 02-11-2022
+           8 | Ericsson      |          185 | 02-23-2022
+           7 | Li            |          175 | 03-08-2022
+           9 | Muller        |          250 | 03-11-2022
+          12 | Plato         |          350 | 03-19-2022
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           3 | Ford          |         2000 | 04-30-2022
+          14 | Smith         |          510 | 05-04-2022
+(14 rows)
+
+SELECT * FROM sales_apr2022;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           3 | Ford          |         2000 | 04-30-2022
+(4 rows)
+
+SELECT * FROM sales_apr2022_01_10;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+(2 rows)
+
+SELECT * FROM sales_apr2022_10_20;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           4 | Ivanov        |          750 | 04-13-2022
+(1 row)
+
+SELECT * FROM sales_apr2022_20_30;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           3 | Ford          |         2000 | 04-30-2022
+(1 row)
+
+ALTER TABLE sales_apr2022 MERGE PARTITIONS (sales_apr2022_01_10, sales_apr2022_10_20, sales_apr2022_20_30) INTO sales_apr_all;
+SELECT * FROM sales_range;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           1 | May           |         1000 | 01-31-2022
+          10 | Halder        |          350 | 01-28-2022
+          13 | Gandi         |          377 | 01-09-2022
+           2 | Smirnoff      |          500 | 02-10-2022
+           6 | Poirot        |          150 | 02-11-2022
+           8 | Ericsson      |          185 | 02-23-2022
+           7 | Li            |          175 | 03-08-2022
+           9 | Muller        |          250 | 03-11-2022
+          12 | Plato         |          350 | 03-19-2022
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           3 | Ford          |         2000 | 04-30-2022
+          14 | Smith         |          510 | 05-04-2022
+(14 rows)
+
+SELECT * FROM sales_apr2022;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           3 | Ford          |         2000 | 04-30-2022
+(4 rows)
+
+SELECT * FROM sales_apr_all;
+ salesman_id | salesman_name | sales_amount | sales_date 
+-------------+---------------+--------------+------------
+           5 | Deev          |          250 | 04-07-2022
+          11 | Trump         |          380 | 04-06-2022
+           4 | Ivanov        |          750 | 04-13-2022
+           3 | Ford          |         2000 | 04-30-2022
+(4 rows)
+
+DROP TABLE sales_range;
+--
+-- BY LIST partitioning
+--
+--
+-- Test: specific errors for BY LIST partitioning
+--
+CREATE TABLE sales_list
+(salesman_id INT GENERATED ALWAYS AS IDENTITY,
+ salesman_name VARCHAR(30),
+ sales_state VARCHAR(20),
+ sales_amount INT,
+ sales_date DATE)
+PARTITION BY LIST (sales_state);
+CREATE TABLE sales_nord PARTITION OF sales_list FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki');
+CREATE TABLE sales_west PARTITION OF sales_list FOR VALUES IN ('Lisbon', 'New York', 'Madrid');
+CREATE TABLE sales_east PARTITION OF sales_list FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok');
+CREATE TABLE sales_central PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv');
+CREATE TABLE sales_others PARTITION OF sales_list DEFAULT;
+CREATE TABLE sales_list2 (LIKE sales_list) PARTITION BY LIST (sales_state);
+CREATE TABLE sales_nord2 PARTITION OF sales_list2 FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki');
+CREATE TABLE sales_others2 PARTITION OF sales_list2 DEFAULT;
+CREATE TABLE sales_external (LIKE sales_list);
+CREATE TABLE sales_external2 (vch VARCHAR(5));
+-- ERROR:  partition bound for relation "sales_external" is null
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_external) INTO sales_all;
+ERROR:  partition bound for relation "sales_external" is null
+-- ERROR:  partition bound for relation "sales_external2" is null
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_external2) INTO sales_all;
+ERROR:  partition bound for relation "sales_external2" is null
+-- ERROR:  relation "sales_nord2" is not a partition of relation "sales_list"
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_nord2, sales_east) INTO sales_all;
+ERROR:  relation "sales_nord2" is not a partition of relation "sales_list"
+DROP TABLE sales_external2;
+DROP TABLE sales_external;
+DROP TABLE sales_list2;
+DROP TABLE sales_list;
+--
+-- Test: BY LIST partitioning, MERGE PARTITIONS with data
+--
+CREATE TABLE sales_list
+(salesman_id INT GENERATED ALWAYS AS IDENTITY,
+ salesman_name VARCHAR(30),
+ sales_state VARCHAR(20),
+ sales_amount INT,
+ sales_date DATE)
+PARTITION BY LIST (sales_state);
+CREATE INDEX sales_list_salesman_name_idx ON sales_list USING btree (salesman_name);
+CREATE INDEX sales_list_sales_state_idx ON sales_list USING btree (sales_state);
+CREATE TABLE sales_nord PARTITION OF sales_list FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki');
+CREATE TABLE sales_west PARTITION OF sales_list FOR VALUES IN ('Lisbon', 'New York', 'Madrid');
+CREATE TABLE sales_east PARTITION OF sales_list FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok');
+CREATE TABLE sales_central PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv');
+CREATE TABLE sales_others PARTITION OF sales_list DEFAULT;
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Trump',    'Bejing',         1000, '2022-03-01');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Smirnoff', 'New York',        500, '2022-03-03');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Ford',     'St. Petersburg', 2000, '2022-03-05');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Ivanov',   'Warsaw',          750, '2022-03-04');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Deev',     'Lisbon',          250, '2022-03-07');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Poirot',   'Berlin',         1000, '2022-03-01');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('May',      'Helsinki',       1200, '2022-03-06');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Li',       'Vladivostok',    1150, '2022-03-09');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('May',      'Helsinki',       1200, '2022-03-11');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Halder',   'Oslo',            800, '2022-03-02');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Muller',   'Madrid',          650, '2022-03-05');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Smith',    'Kyiv',            350, '2022-03-10');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Gandi',    'Warsaw',          150, '2022-03-08');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Plato',    'Lisbon',          950, '2022-03-05');
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_list'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+      oid      | relkind | inhdetachpending |                     pg_get_expr                      
+---------------+---------+------------------+------------------------------------------------------
+ sales_central | r       | f                | FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')
+ sales_east    | r       | f                | FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok')
+ sales_nord    | r       | f                | FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki')
+ sales_west    | r       | f                | FOR VALUES IN ('Lisbon', 'New York', 'Madrid')
+ sales_others  | r       | f                | DEFAULT
+(5 rows)
+
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_central) INTO sales_all;
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_list'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+     oid      | relkind | inhdetachpending |                                                 pg_get_expr                                                  
+--------------+---------+------------------+--------------------------------------------------------------------------------------------------------------
+ sales_all    | r       | f                | FOR VALUES IN ('Lisbon', 'New York', 'Madrid', 'Bejing', 'Delhi', 'Vladivostok', 'Warsaw', 'Berlin', 'Kyiv')
+ sales_nord   | r       | f                | FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki')
+ sales_others | r       | f                | DEFAULT
+(3 rows)
+
+SELECT * FROM sales_list;
+ salesman_id | salesman_name |  sales_state   | sales_amount | sales_date 
+-------------+---------------+----------------+--------------+------------
+           2 | Smirnoff      | New York       |          500 | 03-03-2022
+           5 | Deev          | Lisbon         |          250 | 03-07-2022
+          11 | Muller        | Madrid         |          650 | 03-05-2022
+          14 | Plato         | Lisbon         |          950 | 03-05-2022
+           1 | Trump         | Bejing         |         1000 | 03-01-2022
+           8 | Li            | Vladivostok    |         1150 | 03-09-2022
+           4 | Ivanov        | Warsaw         |          750 | 03-04-2022
+           6 | Poirot        | Berlin         |         1000 | 03-01-2022
+          12 | Smith         | Kyiv           |          350 | 03-10-2022
+          13 | Gandi         | Warsaw         |          150 | 03-08-2022
+           3 | Ford          | St. Petersburg |         2000 | 03-05-2022
+           7 | May           | Helsinki       |         1200 | 03-06-2022
+           9 | May           | Helsinki       |         1200 | 03-11-2022
+          10 | Halder        | Oslo           |          800 | 03-02-2022
+(14 rows)
+
+SELECT * FROM sales_nord;
+ salesman_id | salesman_name |  sales_state   | sales_amount | sales_date 
+-------------+---------------+----------------+--------------+------------
+           3 | Ford          | St. Petersburg |         2000 | 03-05-2022
+           7 | May           | Helsinki       |         1200 | 03-06-2022
+           9 | May           | Helsinki       |         1200 | 03-11-2022
+          10 | Halder        | Oslo           |          800 | 03-02-2022
+(4 rows)
+
+SELECT * FROM sales_all;
+ salesman_id | salesman_name | sales_state | sales_amount | sales_date 
+-------------+---------------+-------------+--------------+------------
+           2 | Smirnoff      | New York    |          500 | 03-03-2022
+           5 | Deev          | Lisbon      |          250 | 03-07-2022
+          11 | Muller        | Madrid      |          650 | 03-05-2022
+          14 | Plato         | Lisbon      |          950 | 03-05-2022
+           1 | Trump         | Bejing      |         1000 | 03-01-2022
+           8 | Li            | Vladivostok |         1150 | 03-09-2022
+           4 | Ivanov        | Warsaw      |          750 | 03-04-2022
+           6 | Poirot        | Berlin      |         1000 | 03-01-2022
+          12 | Smith         | Kyiv        |          350 | 03-10-2022
+          13 | Gandi         | Warsaw      |          150 | 03-08-2022
+(10 rows)
+
+-- Use indexscan for test indexes after merge partitions
+SET enable_seqscan = OFF;
+SELECT * FROM sales_all WHERE sales_state = 'Warsaw';
+ salesman_id | salesman_name | sales_state | sales_amount | sales_date 
+-------------+---------------+-------------+--------------+------------
+           4 | Ivanov        | Warsaw      |          750 | 03-04-2022
+          13 | Gandi         | Warsaw      |          150 | 03-08-2022
+(2 rows)
+
+SELECT * FROM sales_list WHERE sales_state = 'Warsaw';
+ salesman_id | salesman_name | sales_state | sales_amount | sales_date 
+-------------+---------------+-------------+--------------+------------
+           4 | Ivanov        | Warsaw      |          750 | 03-04-2022
+          13 | Gandi         | Warsaw      |          150 | 03-08-2022
+(2 rows)
+
+SELECT * FROM sales_list WHERE salesman_name = 'Ivanov';
+ salesman_id | salesman_name | sales_state | sales_amount | sales_date 
+-------------+---------------+-------------+--------------+------------
+           4 | Ivanov        | Warsaw      |          750 | 03-04-2022
+(1 row)
+
+RESET enable_seqscan;
+DROP TABLE sales_list;
+--
+DROP SCHEMA partitions_merge_schema;
index e9184b5a40919820e0a2c5430dd2f2ba6088eea8..0bd1444767b114589994616b949c2a6c11e5b113 100644 (file)
@@ -119,7 +119,7 @@ test: plancache limit plpgsql copy2 temp domain rangefuncs prepare conversion tr
 # The stats test resets stats, so nothing else needing stats access can be in
 # this group.
 # ----------
-test: partition_join partition_prune reloptions hash_part indexing partition_aggregate partition_info tuplesort explain compression memoize stats predicate
+test: partition_merge partition_join partition_prune reloptions hash_part indexing partition_aggregate partition_info tuplesort explain compression memoize stats predicate
 
 # event_trigger depends on create_am and cannot run concurrently with
 # any test that runs DDL
diff --git a/src/test/regress/sql/partition_merge.sql b/src/test/regress/sql/partition_merge.sql
new file mode 100644 (file)
index 0000000..bb461e6
--- /dev/null
@@ -0,0 +1,430 @@
+--
+-- PARTITIONS_MERGE
+-- Tests for "ALTER TABLE ... MERGE PARTITIONS ..." command
+--
+
+CREATE SCHEMA partitions_merge_schema;
+SET search_path = partitions_merge_schema, public;
+
+--
+-- BY RANGE partitioning
+--
+
+--
+-- Test for error codes
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_dec2021 PARTITION OF sales_range FOR VALUES FROM ('2021-12-01') TO ('2021-12-31');
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+
+CREATE TABLE sales_apr2022 (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_apr_1 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-04-15');
+CREATE TABLE sales_apr_2 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-15') TO ('2022-05-01');
+ALTER TABLE sales_range ATTACH PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+
+-- ERROR:  partition with name "sales_feb2022" already used
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_feb2022) INTO sales_feb_mar_apr2022;
+-- ERROR:  cannot merge non-table partition "sales_apr2022"
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO sales_feb_mar_apr2022;
+-- ERROR:  invalid partitions order, partition "sales_mar2022" can not be merged
+-- (space between sections sales_jan2022 and sales_mar2022)
+ALTER TABLE sales_range MERGE PARTITIONS (sales_jan2022, sales_mar2022) INTO sales_jan_mar2022;
+-- ERROR:  invalid partitions order, partition "sales_jan2022" can not be merged
+-- (space between sections sales_dec2021 and sales_jan2022)
+ALTER TABLE sales_range MERGE PARTITIONS (sales_dec2021, sales_jan2022, sales_feb2022) INTO sales_dec_jan_feb2022;
+
+-- NO ERROR: test for custom partitions order
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_jan2022) INTO sales_jan_feb_mar2022;
+
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+
+DROP TABLE sales_range;
+
+--
+-- Add rows into partitioned table then merge partitions
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+CREATE TABLE sales_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+CREATE INDEX sales_range_sales_date_idx ON sales_range USING btree (sales_date);
+
+INSERT INTO sales_range VALUES (1,  'May',      1000, '2022-01-31');
+INSERT INTO sales_range VALUES (2,  'Smirnoff', 500,  '2022-02-10');
+INSERT INTO sales_range VALUES (3,  'Ford',     2000, '2022-04-30');
+INSERT INTO sales_range VALUES (4,  'Ivanov',   750,  '2022-04-13');
+INSERT INTO sales_range VALUES (5,  'Deev',     250,  '2022-04-07');
+INSERT INTO sales_range VALUES (6,  'Poirot',   150,  '2022-02-11');
+INSERT INTO sales_range VALUES (7,  'Li',       175,  '2022-03-08');
+INSERT INTO sales_range VALUES (8,  'Ericsson', 185,  '2022-02-23');
+INSERT INTO sales_range VALUES (9,  'Muller',   250,  '2022-03-11');
+INSERT INTO sales_range VALUES (10, 'Halder',   350,  '2022-01-28');
+INSERT INTO sales_range VALUES (11, 'Trump',    380,  '2022-04-06');
+INSERT INTO sales_range VALUES (12, 'Plato',    350,  '2022-03-19');
+INSERT INTO sales_range VALUES (13, 'Gandi',    377,  '2022-01-09');
+INSERT INTO sales_range VALUES (14, 'Smith',    510,  '2022-05-04');
+
+SELECT pg_catalog.pg_get_partkeydef('sales_range'::regclass);
+
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+
+ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO sales_feb_mar_apr2022;
+
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+
+SELECT * FROM pg_indexes WHERE tablename = 'sales_feb_mar_apr2022' and schemaname = 'partitions_merge_schema';
+
+SELECT * FROM sales_range;
+SELECT * FROM sales_jan2022;
+SELECT * FROM sales_feb_mar_apr2022;
+SELECT * FROM sales_others;
+
+-- Use indexscan for test indexes
+SET enable_seqscan = OFF;
+
+SELECT * FROM sales_feb_mar_apr2022 where sales_date > '2022-01-01';
+
+RESET enable_seqscan;
+
+DROP TABLE sales_range;
+
+--
+-- Merge some partitions into DEFAULT partition
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+CREATE TABLE sales_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+CREATE INDEX sales_range_sales_date_idx ON sales_range USING btree (sales_date);
+
+INSERT INTO sales_range VALUES (1,  'May',      1000, '2022-01-31');
+INSERT INTO sales_range VALUES (2,  'Smirnoff', 500,  '2022-02-10');
+INSERT INTO sales_range VALUES (3,  'Ford',     2000, '2022-04-30');
+INSERT INTO sales_range VALUES (4,  'Ivanov',   750,  '2022-04-13');
+INSERT INTO sales_range VALUES (5,  'Deev',     250,  '2022-04-07');
+INSERT INTO sales_range VALUES (6,  'Poirot',   150,  '2022-02-11');
+INSERT INTO sales_range VALUES (7,  'Li',       175,  '2022-03-08');
+INSERT INTO sales_range VALUES (8,  'Ericsson', 185,  '2022-02-23');
+INSERT INTO sales_range VALUES (9,  'Muller',   250,  '2022-03-11');
+INSERT INTO sales_range VALUES (10, 'Halder',   350,  '2022-01-28');
+INSERT INTO sales_range VALUES (11, 'Trump',    380,  '2022-04-06');
+INSERT INTO sales_range VALUES (12, 'Plato',    350,  '2022-03-19');
+INSERT INTO sales_range VALUES (13, 'Gandi',    377,  '2022-01-09');
+INSERT INTO sales_range VALUES (14, 'Smith',    510,  '2022-05-04');
+
+-- Merge partitions (include DEFAULT partition) into partition with the same
+-- name
+ALTER TABLE sales_range MERGE PARTITIONS (sales_jan2022, sales_mar2022, sales_others) INTO sales_others;
+
+select * from sales_others;
+
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_range'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+
+DROP TABLE sales_range;
+
+--
+-- Test for:
+--   * composite partition key;
+--   * GENERATED column;
+--   * column with DEFAULT value.
+--
+CREATE TABLE sales_date (salesman_name VARCHAR(30), sales_year INT, sales_month INT, sales_day INT,
+  sales_date VARCHAR(10) GENERATED ALWAYS AS
+    (LPAD(sales_year::text, 4, '0') || '.' || LPAD(sales_month::text, 2, '0') || '.' || LPAD(sales_day::text, 2, '0')) STORED,
+  sales_department VARCHAR(30) DEFAULT 'Sales department')
+  PARTITION BY RANGE (sales_year, sales_month, sales_day);
+
+CREATE TABLE sales_dec2022 PARTITION OF sales_date FOR VALUES FROM (2021, 12, 1) TO (2022, 1, 1);
+CREATE TABLE sales_jan2022 PARTITION OF sales_date FOR VALUES FROM (2022, 1, 1) TO (2022, 2, 1);
+CREATE TABLE sales_feb2022 PARTITION OF sales_date FOR VALUES FROM (2022, 2, 1) TO (2022, 3, 1);
+CREATE TABLE sales_other PARTITION OF sales_date FOR VALUES FROM (2022, 3, 1) TO (MAXVALUE, MAXVALUE, MAXVALUE);
+
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2021, 12, 7);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2021, 12, 8);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 1, 1);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 2, 4);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 1, 2);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 2, 1);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 3, 3);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 3, 4);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 5, 1);
+
+SELECT * FROM sales_date;
+SELECT * FROM sales_dec2022;
+SELECT * FROM sales_jan2022;
+SELECT * FROM sales_feb2022;
+SELECT * FROM sales_other;
+
+ALTER TABLE sales_date MERGE PARTITIONS (sales_jan2022, sales_feb2022) INTO sales_jan_feb2022;
+
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 1, 10);
+INSERT INTO sales_date(salesman_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 2, 10);
+
+SELECT * FROM sales_date;
+SELECT * FROM sales_dec2022;
+SELECT * FROM sales_jan_feb2022;
+SELECT * FROM sales_other;
+
+DROP TABLE sales_date;
+
+--
+-- Test: merge partitions of partitioned table with triggers
+--
+CREATE TABLE salesmans(salesman_id INT PRIMARY KEY, salesman_name VARCHAR(30)) PARTITION BY RANGE (salesman_id);
+
+CREATE TABLE salesmans01_10 PARTITION OF salesmans FOR VALUES FROM (1) TO (10);
+CREATE TABLE salesmans10_20 PARTITION OF salesmans FOR VALUES FROM (10) TO (20);
+CREATE TABLE salesmans20_30 PARTITION OF salesmans FOR VALUES FROM (20) TO (30);
+CREATE TABLE salesmans30_40 PARTITION OF salesmans FOR VALUES FROM (30) TO (40);
+
+INSERT INTO salesmans VALUES (1,  'Poirot');
+
+CREATE OR REPLACE FUNCTION after_insert_row_trigger() RETURNS trigger LANGUAGE 'plpgsql' AS $BODY$
+BEGIN
+   RAISE NOTICE 'trigger(%) called: action = %, when = %, level = %', TG_ARGV[0], TG_OP, TG_WHEN, TG_LEVEL;
+    RETURN NULL;
+END;
+$BODY$;
+
+CREATE TRIGGER salesmans_after_insert_statement_trigger
+    AFTER INSERT
+    ON salesmans
+    FOR EACH STATEMENT
+    EXECUTE PROCEDURE after_insert_row_trigger('salesmans');
+
+CREATE TRIGGER salesmans_after_insert_row_trigger
+    AFTER INSERT
+    ON salesmans
+    FOR EACH ROW
+    EXECUTE PROCEDURE after_insert_row_trigger('salesmans');
+
+-- 2 triggers should fire here (row + statement):
+INSERT INTO salesmans VALUES (10, 'May');
+-- 1 trigger should fire here (row):
+INSERT INTO salesmans10_20 VALUES (19, 'Ivanov');
+
+ALTER TABLE salesmans MERGE PARTITIONS (salesmans10_20, salesmans20_30, salesmans30_40) INTO salesmans10_40;
+
+-- 2 triggers should fire here (row + statement):
+INSERT INTO salesmans VALUES (20, 'Smirnoff');
+-- 1 trigger should fire here (row):
+INSERT INTO salesmans10_40 VALUES (30, 'Ford');
+
+SELECT * FROM salesmans01_10;
+SELECT * FROM salesmans10_40;
+
+DROP TABLE salesmans;
+DROP FUNCTION after_insert_row_trigger();
+
+--
+-- Test: merge partitions with deleted columns
+--
+CREATE TABLE salesmans(salesman_id INT PRIMARY KEY, salesman_name VARCHAR(30)) PARTITION BY RANGE (salesman_id);
+
+CREATE TABLE salesmans01_10 PARTITION OF salesmans FOR VALUES FROM (1) TO (10);
+-- Create partitions with some deleted columns:
+CREATE TABLE salesmans10_20(d1 VARCHAR(30), salesman_id INT PRIMARY KEY, salesman_name VARCHAR(30));
+CREATE TABLE salesmans20_30(salesman_id INT PRIMARY KEY, d2 INT,  salesman_name VARCHAR(30));
+CREATE TABLE salesmans30_40(salesman_id INT PRIMARY KEY, d3 DATE, salesman_name VARCHAR(30));
+
+INSERT INTO salesmans10_20 VALUES ('dummy value 1', 19, 'Ivanov');
+INSERT INTO salesmans20_30 VALUES (20, 101, 'Smirnoff');
+INSERT INTO salesmans30_40 VALUES (31, now(), 'Popov');
+
+ALTER TABLE salesmans10_20 DROP COLUMN d1;
+ALTER TABLE salesmans20_30 DROP COLUMN d2;
+ALTER TABLE salesmans30_40 DROP COLUMN d3;
+
+ALTER TABLE salesmans ATTACH PARTITION salesmans10_20 FOR VALUES FROM (10) TO (20);
+ALTER TABLE salesmans ATTACH PARTITION salesmans20_30 FOR VALUES FROM (20) TO (30);
+ALTER TABLE salesmans ATTACH PARTITION salesmans30_40 FOR VALUES FROM (30) TO (40);
+
+INSERT INTO salesmans VALUES (1, 'Poirot');
+INSERT INTO salesmans VALUES (10, 'May');
+INSERT INTO salesmans VALUES (30, 'Ford');
+
+ALTER TABLE salesmans MERGE PARTITIONS (salesmans10_20, salesmans20_30, salesmans30_40) INTO salesmans10_40;
+
+select * from salesmans;
+select * from salesmans01_10;
+select * from salesmans10_40;
+
+DROP TABLE salesmans;
+
+--
+-- Test: merge sub-partitions
+--
+CREATE TABLE sales_range (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01');
+CREATE TABLE sales_feb2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-03-01');
+CREATE TABLE sales_mar2022 PARTITION OF sales_range FOR VALUES FROM ('2022-03-01') TO ('2022-04-01');
+
+CREATE TABLE sales_apr2022 (salesman_id INT, salesman_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date);
+CREATE TABLE sales_apr2022_01_10 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-04-10');
+CREATE TABLE sales_apr2022_10_20 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-10') TO ('2022-04-20');
+CREATE TABLE sales_apr2022_20_30 PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-20') TO ('2022-05-01');
+ALTER TABLE sales_range ATTACH PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01');
+
+CREATE TABLE sales_others PARTITION OF sales_range DEFAULT;
+
+CREATE INDEX sales_range_sales_date_idx ON sales_range USING btree (sales_date);
+
+INSERT INTO sales_range VALUES (1,  'May',      1000, '2022-01-31');
+INSERT INTO sales_range VALUES (2,  'Smirnoff', 500,  '2022-02-10');
+INSERT INTO sales_range VALUES (3,  'Ford',     2000, '2022-04-30');
+INSERT INTO sales_range VALUES (4,  'Ivanov',   750,  '2022-04-13');
+INSERT INTO sales_range VALUES (5,  'Deev',     250,  '2022-04-07');
+INSERT INTO sales_range VALUES (6,  'Poirot',   150,  '2022-02-11');
+INSERT INTO sales_range VALUES (7,  'Li',       175,  '2022-03-08');
+INSERT INTO sales_range VALUES (8,  'Ericsson', 185,  '2022-02-23');
+INSERT INTO sales_range VALUES (9,  'Muller',   250,  '2022-03-11');
+INSERT INTO sales_range VALUES (10, 'Halder',   350,  '2022-01-28');
+INSERT INTO sales_range VALUES (11, 'Trump',    380,  '2022-04-06');
+INSERT INTO sales_range VALUES (12, 'Plato',    350,  '2022-03-19');
+INSERT INTO sales_range VALUES (13, 'Gandi',    377,  '2022-01-09');
+INSERT INTO sales_range VALUES (14, 'Smith',    510,  '2022-05-04');
+
+SELECT * FROM sales_range;
+SELECT * FROM sales_apr2022;
+SELECT * FROM sales_apr2022_01_10;
+SELECT * FROM sales_apr2022_10_20;
+SELECT * FROM sales_apr2022_20_30;
+
+ALTER TABLE sales_apr2022 MERGE PARTITIONS (sales_apr2022_01_10, sales_apr2022_10_20, sales_apr2022_20_30) INTO sales_apr_all;
+
+SELECT * FROM sales_range;
+SELECT * FROM sales_apr2022;
+SELECT * FROM sales_apr_all;
+
+DROP TABLE sales_range;
+
+--
+-- BY LIST partitioning
+--
+
+--
+-- Test: specific errors for BY LIST partitioning
+--
+CREATE TABLE sales_list
+(salesman_id INT GENERATED ALWAYS AS IDENTITY,
+ salesman_name VARCHAR(30),
+ sales_state VARCHAR(20),
+ sales_amount INT,
+ sales_date DATE)
+PARTITION BY LIST (sales_state);
+CREATE TABLE sales_nord PARTITION OF sales_list FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki');
+CREATE TABLE sales_west PARTITION OF sales_list FOR VALUES IN ('Lisbon', 'New York', 'Madrid');
+CREATE TABLE sales_east PARTITION OF sales_list FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok');
+CREATE TABLE sales_central PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv');
+CREATE TABLE sales_others PARTITION OF sales_list DEFAULT;
+
+
+CREATE TABLE sales_list2 (LIKE sales_list) PARTITION BY LIST (sales_state);
+CREATE TABLE sales_nord2 PARTITION OF sales_list2 FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki');
+CREATE TABLE sales_others2 PARTITION OF sales_list2 DEFAULT;
+
+
+CREATE TABLE sales_external (LIKE sales_list);
+CREATE TABLE sales_external2 (vch VARCHAR(5));
+
+-- ERROR:  partition bound for relation "sales_external" is null
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_external) INTO sales_all;
+-- ERROR:  partition bound for relation "sales_external2" is null
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_external2) INTO sales_all;
+-- ERROR:  relation "sales_nord2" is not a partition of relation "sales_list"
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_nord2, sales_east) INTO sales_all;
+
+DROP TABLE sales_external2;
+DROP TABLE sales_external;
+DROP TABLE sales_list2;
+DROP TABLE sales_list;
+
+--
+-- Test: BY LIST partitioning, MERGE PARTITIONS with data
+--
+CREATE TABLE sales_list
+(salesman_id INT GENERATED ALWAYS AS IDENTITY,
+ salesman_name VARCHAR(30),
+ sales_state VARCHAR(20),
+ sales_amount INT,
+ sales_date DATE)
+PARTITION BY LIST (sales_state);
+
+CREATE INDEX sales_list_salesman_name_idx ON sales_list USING btree (salesman_name);
+CREATE INDEX sales_list_sales_state_idx ON sales_list USING btree (sales_state);
+
+CREATE TABLE sales_nord PARTITION OF sales_list FOR VALUES IN ('Oslo', 'St. Petersburg', 'Helsinki');
+CREATE TABLE sales_west PARTITION OF sales_list FOR VALUES IN ('Lisbon', 'New York', 'Madrid');
+CREATE TABLE sales_east PARTITION OF sales_list FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok');
+CREATE TABLE sales_central PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv');
+CREATE TABLE sales_others PARTITION OF sales_list DEFAULT;
+
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Trump',    'Bejing',         1000, '2022-03-01');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Smirnoff', 'New York',        500, '2022-03-03');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Ford',     'St. Petersburg', 2000, '2022-03-05');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Ivanov',   'Warsaw',          750, '2022-03-04');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Deev',     'Lisbon',          250, '2022-03-07');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Poirot',   'Berlin',         1000, '2022-03-01');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('May',      'Helsinki',       1200, '2022-03-06');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Li',       'Vladivostok',    1150, '2022-03-09');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('May',      'Helsinki',       1200, '2022-03-11');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Halder',   'Oslo',            800, '2022-03-02');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Muller',   'Madrid',          650, '2022-03-05');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Smith',    'Kyiv',            350, '2022-03-10');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Gandi',    'Warsaw',          150, '2022-03-08');
+INSERT INTO sales_list (salesman_name, sales_state, sales_amount, sales_date) VALUES ('Plato',    'Lisbon',          950, '2022-03-05');
+
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_list'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+
+ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_central) INTO sales_all;
+
+-- show partitions with conditions:
+SELECT c.oid::pg_catalog.regclass, c.relkind, inhdetachpending, pg_catalog.pg_get_expr(c.relpartbound, c.oid)
+  FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i
+  WHERE c.oid = i.inhrelid AND i.inhparent = 'sales_list'::regclass
+  ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text;
+
+SELECT * FROM sales_list;
+SELECT * FROM sales_nord;
+SELECT * FROM sales_all;
+
+-- Use indexscan for test indexes after merge partitions
+SET enable_seqscan = OFF;
+
+SELECT * FROM sales_all WHERE sales_state = 'Warsaw';
+SELECT * FROM sales_list WHERE sales_state = 'Warsaw';
+SELECT * FROM sales_list WHERE salesman_name = 'Ivanov';
+
+RESET enable_seqscan;
+
+DROP TABLE sales_list;
+
+--
+DROP SCHEMA partitions_merge_schema;
index f87e8b80ec071e01bce06bcb4718b5ce10a3c3ec..83e4d7df2157b596b23f74a89c8ae32ac9009e39 100644 (file)
@@ -2631,6 +2631,7 @@ SimpleStats
 SimpleStringList
 SimpleStringListCell
 SingleBoundSortItem
+SinglePartitionSpec
 Size
 SkipPages
 SlabBlock