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>
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>
</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
<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>
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>
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
cmd_lockmode = ShareUpdateExclusiveLock;
break;
+ case AT_MergePartitions:
+ cmd_lockmode = AccessExclusiveLock;
+ break;
+
default: /* oops */
elog(ERROR, "unrecognized alter table type: %d",
(int) cmd->subtype);
/* 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);
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);
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:
}
}
+/*
+ * 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
*
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.
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);
+}
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
n->subtype = AT_AttachPartition;
cmd->name = $3;
cmd->bound = $4;
+ cmd->partlist = NULL;
cmd->concurrent = false;
n->def = (Node *) cmd;
n->subtype = AT_DetachPartition;
cmd->name = $3;
cmd->bound = NULL;
+ cmd->partlist = NULL;
cmd->concurrent = $4;
n->def = (Node *) 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;
n->subtype = AT_AttachPartition;
cmd->name = $3;
cmd->bound = NULL;
+ cmd->partlist = NULL;
cmd->concurrent = false;
n->def = (Node *) cmd;
| PARSER
| PARTIAL
| PARTITION
+ | PARTITIONS
| PASSING
| PASSWORD
| PATH
| PARSER
| PARTIAL
| PARTITION
+ | PARTITIONS
| PASSING
| PASSWORD
| PATH
#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"
}
+/*
+ * 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
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:
/*
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);
+ }
+}
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
*/
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;
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 */
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)
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 */
--- /dev/null
+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)
+
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
--- /dev/null
+# 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
case AT_DetachPartitionFinalize:
strtype = "DETACH PARTITION ... FINALIZE";
break;
+ case AT_MergePartitions:
+ strtype = "MERGE PARTITIONS";
+ break;
case AT_AddIdentity:
strtype = "ADD IDENTITY";
break;
--- /dev/null
+--
+-- 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;
# 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
--- /dev/null
+--
+-- 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;
SimpleStringList
SimpleStringListCell
SingleBoundSortItem
+SinglePartitionSpec
Size
SkipPages
SlabBlock