From a934c8855cb1b05474a34a4417eb0ed1c00bc5db Mon Sep 17 00:00:00 2001 From: Alexander Korotkov Date: Sun, 7 Apr 2024 00:57:22 +0300 Subject: [PATCH v32 1/2] Implement ALTER TABLE ... MERGE PARTITIONS ... command This new DDL command merges several partitions into the one partition of the target table. The target partition is created using new createPartitionTable() function with parent partition as the template. This commit comprises quite naive implementation which works in single process and holds the ACCESS EXCLUSIVE LOCK on the parent table during all the operations including the tuple routing. This is why this new DDL command can't be recommended for large partitioned tables under a high load. However, this implementation come in handy in certain cases even as is. Also, it could be used as a foundation for future implementations with lesser locking and possibly parallel. Discussion: https://postgr.es/m/c73a1746-0cd0-6bdd-6b23-3ae0b7c0c582%40postgrespro.ru Author: Dmitry Koval Reviewed-by: Matthias van de Meent, Laurenz Albe, Zhihong Yu, Justin Pryzby Reviewed-by: Alvaro Herrera, Robert Haas, Stephane Tachoires Fixes (summary information). Authors: Alexander Korotkov, Tender Wang, Richard Guo, Dagfinn Ilmari Mannsaker Authors: Fujii Masao Reviewed-by: Alexander Korotkov, Robert Haas, Justin Pryzby, Pavel Borisov Reviewed-by: Masahiko Sawada Reported-by: Alexander Lakhin, Justin Pryzby, Kyotaro Horiguchi Reported-by: Daniel Gustafsson, Tom Lane, Noah Misch --- doc/src/sgml/ddl.sgml | 19 + doc/src/sgml/ref/alter_table.sgml | 90 +- src/backend/commands/tablecmds.c | 420 +++++++- src/backend/parser/gram.y | 22 +- src/backend/parser/parse_utilcmd.c | 131 +++ src/backend/partitioning/partbounds.c | 212 +++- src/bin/psql/tab-complete.c | 10 + src/include/nodes/parsenodes.h | 14 + src/include/parser/kwlist.h | 1 + src/include/partitioning/partbounds.h | 6 + .../isolation/expected/partition-merge.out | 199 ++++ src/test/isolation/isolation_schedule | 1 + src/test/isolation/specs/partition-merge.spec | 54 + .../test_ddl_deparse/test_ddl_deparse.c | 3 + src/test/regress/expected/partition_merge.out | 945 ++++++++++++++++++ src/test/regress/parallel_schedule | 2 +- src/test/regress/sql/partition_merge.sql | 609 +++++++++++ src/tools/pgindent/typedefs.list | 1 + 18 files changed, 2715 insertions(+), 24 deletions(-) create mode 100644 src/test/isolation/expected/partition-merge.out create mode 100644 src/test/isolation/specs/partition-merge.spec create mode 100644 src/test/regress/expected/partition_merge.out create mode 100644 src/test/regress/sql/partition_merge.sql diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml index b671858627..2770e81656 100644 --- a/doc/src/sgml/ddl.sgml +++ b/doc/src/sgml/ddl.sgml @@ -4353,6 +4353,25 @@ ALTER TABLE measurement_y2006m02 ADD UNIQUE (city_id, logdate); ALTER INDEX measurement_city_id_logdate_key ATTACH PARTITION measurement_y2006m02_city_id_logdate_key; ... + + + + + There is also an option for merging multiple table partitions into + a single partition using the + ALTER TABLE ... MERGE PARTITIONS. + This feature simplifies the management of partitioned tables by allowing + users 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 + ACCESS EXCLUSIVE 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: + +ALTER TABLE measurement + MERGE PARTITIONS (measurement_y2006m01, + measurement_y2006m02, + measurement_y2006m03) INTO measurement_y2006q1; diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml index 1a49f321cf..b99d4972ac 100644 --- a/doc/src/sgml/ref/alter_table.sgml +++ b/doc/src/sgml/ref/alter_table.sgml @@ -37,6 +37,9 @@ ALTER TABLE [ IF EXISTS ] name ATTACH PARTITION partition_name { FOR VALUES partition_bound_spec | DEFAULT } ALTER TABLE [ IF EXISTS ] name DETACH PARTITION partition_name [ CONCURRENTLY | FINALIZE ] +ALTER TABLE [ IF EXISTS ] name + MERGE PARTITIONS (partition_name1, partition_name2 [, ...]) + INTO partition_name where action is one of: @@ -1117,14 +1120,87 @@ WITH ( MODULUS numeric_literal, REM + + MERGE PARTITIONS (partition_name1, partition_name2 [, ...]) INTO partition_name + + + + 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 partition_name1, + partition_name2 [, ...]: + + + + For range-partitioned tables it is necessary that the ranges + of the partitions partition_name1, + partition_name2 [, ...] 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 + partition_name. + + + + + For list-partitioned tables the value lists of all partitions + partition_name1, + partition_name2 [, ...] are + combined and form the list of values of partition + partition_name. + + + + If DEFAULT partition is in the list of partitions partition_name1, + partition_name2 [, ...]: + + + + The partition partition_name + will be the DEFAULT partition. + + + + + For range- and list-partitioned tables the ranges and lists of values + of the merged partitions can be any. + + + + The new partition partition_name + can have the same name as one of the merged partitions. Only simple, + non-partitioned partitions can be merged. + + + The new partition will be created the same as a table created with the + SQL command CREATE TABLE partition_name (LIKE name INCLUDING ALL EXCLUDING INDEXES EXCLUDING IDENTITY EXCLUDING STATISTICS). + The indexes and identity are created later, after moving the data + into the new partition. + Extended statistics aren't copied from the parent table, for consistency with + CREATE TABLE PARTITION OF. + The new partition will have the same table access method as the parent. + If the parent table is persistent then the new partition is created + persistent. If the parent table is temporary then the new partition + is also created temporary. The new partition will also be created in + the same tablespace as the parent. + + + + This command acquires an ACCESS EXCLUSIVE lock. + This is a significant limitation, which limits the usage of this + command with large partitioned tables under a high load. + + + + + All the forms of ALTER TABLE that act on a single table, except RENAME, SET SCHEMA, - ATTACH PARTITION, and - DETACH PARTITION can be combined into + ATTACH PARTITION, DETACH PARTITION, + and MERGE PARTITIONS 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 @@ -1367,7 +1443,8 @@ WITH ( MODULUS numeric_literal, REM partition_name - 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. @@ -1783,6 +1860,13 @@ ALTER TABLE measurement DETACH PARTITION measurement_y2015m12; + + To merge several partitions into one partition of the target table: + +ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_central) + INTO sales_all; + + diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c index dac39df83a..4d24278495 100644 --- a/src/backend/commands/tablecmds.c +++ b/src/backend/commands/tablecmds.c @@ -657,6 +657,8 @@ static void ATDetachCheckNoForeignKeyRefs(Relation partition); static char GetAttributeCompression(Oid atttypid, const char *compression); static char GetAttributeStorage(Oid atttypid, const char *storagemode); +static void ATExecMergePartitions(List **wqueue, AlteredTableInfo *tab, Relation rel, + PartitionCmd *cmd, AlterTableUtilityContext *context); /* ---------------------------------------------------------------- * DefineRelation @@ -4676,6 +4678,10 @@ AlterTableGetLockLevel(List *cmds) cmd_lockmode = AccessShareLock; break; + case AT_MergePartitions: + cmd_lockmode = AccessExclusiveLock; + break; + default: /* oops */ elog(ERROR, "unrecognized alter table type: %d", (int) cmd->subtype); @@ -5093,6 +5099,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd, /* No command-specific prep needed */ pass = AT_PASS_MISC; break; + case AT_MergePartitions: + ATSimplePermissions(cmd->subtype, rel, ATT_TABLE); + /* No command-specific prep needed */ + pass = AT_PASS_MISC; + break; default: /* oops */ elog(ERROR, "unrecognized alter table type: %d", (int) cmd->subtype); @@ -5489,6 +5500,14 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab, case AT_DetachPartitionFinalize: address = ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name); break; + case AT_MergePartitions: + cmd = ATParseTransformCmd(wqueue, tab, rel, cmd, false, lockmode, + cur_pass, context); + Assert(cmd != NULL); + Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE); + ATExecMergePartitions(wqueue, tab, rel, (PartitionCmd *) cmd->def, + context); + break; default: /* oops */ elog(ERROR, "unrecognized alter table type: %d", (int) cmd->subtype); @@ -6477,6 +6496,8 @@ alter_table_type_to_string(AlterTableType cmdtype) return "DETACH PARTITION"; case AT_DetachPartitionFinalize: return "DETACH PARTITION ... FINALIZE"; + case AT_MergePartitions: + return "MERGE PARTITIONS"; case AT_AddIdentity: return "ALTER COLUMN ... ADD IDENTITY"; case AT_SetIdentity: @@ -18285,6 +18306,37 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel, } } +/* + * attachPartitionTable: attach a new partition to the 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 ATTACH PARTITION FOR VALUES * @@ -18487,23 +18539,8 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd, check_new_partition_bound(RelationGetRelationName(attachrel), rel, cmd->bound, pstate); - /* OK to create inheritance. Rest of the checks performed there */ - CreateInheritance(attachrel, rel, true); - - /* Update the pg_class entry. */ - StorePartitionBound(attachrel, rel, cmd->bound); - - /* Ensure there exists a correct set of indexes in the partition. */ - AttachPartitionEnsureIndexes(wqueue, rel, attachrel); - - /* and triggers */ - CloneRowTriggersToPartition(rel, attachrel); - - /* - * Clone foreign key constraints. Callee is responsible for setting up - * for phase 3 constraint verification. - */ - CloneForeignKeyConstraints(wqueue, rel, attachrel); + /* Attach a new partition to the partitioned table. */ + attachPartitionTable(wqueue, rel, attachrel, cmd->bound); /* * Generate partition constraint from the partition bound specification. @@ -20018,3 +20055,352 @@ GetAttributeStorage(Oid atttypid, const char *storagemode) return cstorage; } + +/* + * createPartitionTable: create table for a new partition with given name + * (newPartName) like table (modelRel) + * + * Emulates command: CREATE [TEMP] TABLE (LIKE + * INCLUDING ALL EXCLUDING INDEXES EXCLUDING IDENTITY EXCLUDING STATISTICS) + * + * Also, this function sets the new partition access method same as parent + * table access methods (similarly to CREATE TABLE ... PARTITION OF). It + * checks that parent and child tables have compatible persistence. + * + * Function returns the created relation (locked in AccessExclusiveLock mode). + */ +static Relation +createPartitionTable(RangeVar *newPartName, Relation modelRel, + AlterTableUtilityContext *context) +{ + CreateStmt *createStmt; + TableLikeClause *tlc; + PlannedStmt *wrapper; + Relation newRel; + + /* If existing rel is temp, it must belong to this session */ + if (modelRel->rd_rel->relpersistence == RELPERSISTENCE_TEMP && + !modelRel->rd_islocaltemp) + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("cannot create as partition of temporary relation of another session"))); + + /* New partition should have the same persistence as modelRel */ + newPartName->relpersistence = modelRel->rd_rel->relpersistence; + + createStmt = makeNode(CreateStmt); + createStmt->relation = newPartName; + createStmt->tableElts = NIL; + createStmt->inhRelations = NIL; + createStmt->constraints = NIL; + createStmt->options = NIL; + createStmt->oncommit = ONCOMMIT_NOOP; + createStmt->tablespacename = get_tablespace_name(modelRel->rd_rel->reltablespace); + createStmt->if_not_exists = false; + createStmt->accessMethod = get_am_name(modelRel->rd_rel->relam); + + tlc = makeNode(TableLikeClause); + tlc->relation = makeRangeVar(get_namespace_name(RelationGetNamespace(modelRel)), + RelationGetRelationName(modelRel), -1); + + /* + * Indexes will be inherited on "attach new partitions" stage, after data + * moving. We also don't copy the extended statistics for consistency + * with CREATE TABLE PARTITION OF. + */ + tlc->options = CREATE_TABLE_LIKE_ALL & + ~(CREATE_TABLE_LIKE_INDEXES | CREATE_TABLE_LIKE_IDENTITY | CREATE_TABLE_LIKE_STATISTICS); + 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); + + /* + * Open the new partition with no lock, because we already have + * AccessExclusiveLock placed there after creation. + */ + newRel = table_openrv(newPartName, NoLock); + + /* + * We intended to create the partition with the same persistence as the + * parent table, but we still need to recheck because that might be + * affected by the search_path. If the parent is permanent, so must be + * all of its partitions. + */ + if (modelRel->rd_rel->relpersistence != RELPERSISTENCE_TEMP && + newRel->rd_rel->relpersistence == RELPERSISTENCE_TEMP) + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("cannot create a temporary relation as partition of permanent relation \"%s\"", + RelationGetRelationName(modelRel)))); + + /* Permanent rels cannot be partitions belonging to temporary parent */ + if (newRel->rd_rel->relpersistence != RELPERSISTENCE_TEMP && + modelRel->rd_rel->relpersistence == RELPERSISTENCE_TEMP) + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("cannot create a permanent relation as partition of temporary relation \"%s\"", + RelationGetRelationName(modelRel)))); + + return newRel; +} + +/* + * 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; + + /* Extract data from old tuple. */ + slot_getallattrs(srcslot); + + if (tuple_map) + { + /* Need to use map to 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. */ + table_tuple_insert(newPartRel, insertslot, mycid, + ti_options, bistate); + + 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 MERGE PARTITIONS INTO + */ +static void +ATExecMergePartitions(List **wqueue, AlteredTableInfo *tab, Relation rel, + PartitionCmd *cmd, AlterTableUtilityContext *context) +{ + Relation newPartRel; + ListCell *listptr; + List *mergingPartitionsList = NIL; + Oid defaultPartOid; + Oid namespaceId; + Oid existingRelid; + + /* + * 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 preventing DML-queries to the partition. + */ + mergingPartition = table_openrv(name, AccessExclusiveLock); + + /* Store a next merging partition into the list. */ + mergingPartitionsList = lappend(mergingPartitionsList, + mergingPartition); + } + + /* + * Look up the namespace in which we are supposed to create the partition, + * check we have permission to create there, lock it against concurrent + * drop, and mark stmt->relation as RELPERSISTENCE_TEMP if a temporary + * namespace is selected. + */ + cmd->name->relpersistence = rel->rd_rel->relpersistence; + namespaceId = + RangeVarGetAndCheckCreationNamespace(cmd->name, NoLock, NULL); + + /* + * Check if this name is already taken. This helps us to detect the + * situation when one of the merging partitions has the same name as the + * new partition. Otherwise, this would fail later on anyway but catching + * this here allows us to emit a nicer error message. + */ + existingRelid = get_relname_relid(cmd->name->relname, namespaceId); + + if (OidIsValid(existingRelid)) + { + Relation sameNamePartition = NULL; + + foreach_ptr(RelationData, mergingPartition, mergingPartitionsList) + { + if (RelationGetRelid(mergingPartition) == existingRelid) + { + sameNamePartition = mergingPartition; + break; + } + } + + if (sameNamePartition) + { + /* + * The new partition has the same name as one of merging + * partitions. + */ + char tmpRelName[NAMEDATALEN]; + + /* Generate temporary name. */ + sprintf(tmpRelName, "merge-%u-%X-tmp", RelationGetRelid(rel), MyProcPid); + + /* + * Rename the existing partition with a temporary name, leaving it + * free for the new partition. We don't need to care about this + * in the future because we're going to eventually drop the + * existing partition anyway. + */ + RenameRelationInternal(RelationGetRelid(sameNamePartition), + tmpRelName, false, false); + + /* + * We must bump the command counter to make the new partition + * tuple visible for rename. + */ + CommandCounterIncrement(); + } + else + { + ereport(ERROR, + (errcode(ERRCODE_DUPLICATE_TABLE), + errmsg("relation \"%s\" already exists", cmd->name->relname))); + } + } + + /* 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. */ + newPartRel = createPartitionTable(cmd->name, rel, context); + + /* Copy data from merged partitions to new partition. */ + moveMergedTablesRows(rel, mergingPartitionsList, newPartRel); + + /* Drop the current partitions before attaching the new one. */ + 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); + + /* + * Attach a new partition to the partitioned table. wqueue = NULL: + * verification for each cloned constraint is not needed. + */ + attachPartitionTable(NULL, rel, newPartRel, cmd->bound); + + /* Keep the lock until commit. */ + table_close(newPartRel, NoLock); +} diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y index 84cef57a70..d85c74cb7c 100644 --- a/src/backend/parser/gram.y +++ b/src/backend/parser/gram.y @@ -760,7 +760,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); ORDER ORDINALITY OTHERS OUT_P OUTER_P OVER OVERLAPS OVERLAY OVERRIDING OWNED OWNER - PARALLEL PARAMETER PARSER PARTIAL PARTITION PASSING PASSWORD PATH + PARALLEL PARAMETER PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PATH PLACING PLAN PLANS POLICY POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROCEDURES PROGRAM PUBLICATION @@ -2318,6 +2318,7 @@ partition_cmd: n->subtype = AT_AttachPartition; cmd->name = $3; cmd->bound = $4; + cmd->partlist = NULL; cmd->concurrent = false; n->def = (Node *) cmd; @@ -2332,6 +2333,7 @@ partition_cmd: n->subtype = AT_DetachPartition; cmd->name = $3; cmd->bound = NULL; + cmd->partlist = NULL; cmd->concurrent = $4; n->def = (Node *) cmd; @@ -2345,6 +2347,21 @@ partition_cmd: n->subtype = AT_DetachPartitionFinalize; cmd->name = $3; cmd->bound = NULL; + cmd->partlist = NULL; + cmd->concurrent = false; + n->def = (Node *) cmd; + $$ = (Node *) n; + } + /* ALTER TABLE MERGE PARTITIONS () INTO */ + | 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; @@ -2361,6 +2378,7 @@ index_partition_cmd: n->subtype = AT_AttachPartition; cmd->name = $3; cmd->bound = NULL; + cmd->partlist = NULL; cmd->concurrent = false; n->def = (Node *) cmd; @@ -17691,6 +17709,7 @@ unreserved_keyword: | PARSER | PARTIAL | PARTITION + | PARTITIONS | PASSING | PASSWORD | PATH @@ -18314,6 +18333,7 @@ bare_label_keyword: | PARSER | PARTIAL | PARTITION + | PARTITIONS | PASSING | PASSWORD | PATH diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c index 79cad4ab30..63ea2ec5ea 100644 --- a/src/backend/parser/parse_utilcmd.c +++ b/src/backend/parser/parse_utilcmd.c @@ -32,6 +32,7 @@ #include "catalog/heap.h" #include "catalog/index.h" #include "catalog/namespace.h" +#include "catalog/partition.h" #include "catalog/pg_am.h" #include "catalog/pg_collation.h" #include "catalog/pg_constraint.h" @@ -58,6 +59,8 @@ #include "parser/parse_type.h" #include "parser/parse_utilcmd.h" #include "parser/parser.h" +#include "partitioning/partdesc.h" +#include "partitioning/partbounds.h" #include "rewrite/rewriteManip.h" #include "utils/acl.h" #include "utils/builtins.h" @@ -3229,6 +3232,121 @@ transformRuleStmt(RuleStmt *stmt, const char *queryString, } +/* + * checkPartition + * Check that partRelOid is an oid of partition of the parent table rel + */ +static void +checkPartition(Relation rel, Oid partRelOid) +{ + Relation partRel; + + partRel = relation_open(partRelOid, AccessShareLock); + + if (partRel->rd_rel->relkind != RELKIND_RELATION) + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("\"%s\" is not a table", + RelationGetRelationName(partRel)))); + + if (!partRel->rd_rel->relispartition) + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("\"%s\" is not a partition", + RelationGetRelationName(partRel)))); + + if (get_partition_parent(partRelOid, false) != RelationGetRelid(rel)) + ereport(ERROR, + (errcode(ERRCODE_UNDEFINED_TABLE), + errmsg("relation \"%s\" is not a partition of relation \"%s\"", + RelationGetRelationName(partRel), + RelationGetRelationName(rel)))); + + /* Permissions checks */ + if (!object_ownercheck(RelationRelationId, RelationGetRelid(partRel), GetUserId())) + aclcheck_error(ACLCHECK_NOT_OWNER, get_relkind_objtype(partRel->rd_rel->relkind), + RelationGetRelationName(partRel)); + + relation_close(partRel, AccessShareLock); +} + +/* + * transformPartitionCmdForMerge + * Analyze the ALTER TABLE ... MERGE PARTITIONS command + * + * Does simple checks for merged partitions. Calculates bound of resulting + * 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\" is 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; + + checkPartition(parent, partOid); + + partOids = lappend_oid(partOids, partOid); + } + + /* Allocate bound of resulting 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 @@ -3505,6 +3623,19 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt, newcmds = lappend(newcmds, cmd); break; + case AT_MergePartitions: + { + PartitionCmd *partcmd = (PartitionCmd *) cmd->def; + + if (list_length(partcmd->partlist) < 2) + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("list of new partitions should contain at least two items"))); + transformPartitionCmdForMerge(&cxt, partcmd); + newcmds = lappend(newcmds, cmd); + break; + } + default: /* diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c index c28639d2e3..6cb32d57ef 100644 --- a/src/backend/partitioning/partbounds.c +++ b/src/backend/partitioning/partbounds.c @@ -3214,8 +3214,9 @@ check_new_partition_bound(char *relname, Relation parent, PartitionRangeDatum *datum; /* - * Point to problematic key in the lower datums list; - * if we have equality, point to the first one. + * Point to problematic key in the list of lower + * datums; if we have equality, point to the first + * one. */ datum = cmpval == 0 ? linitial(spec->lowerdatums) : list_nth(spec->lowerdatums, abs(cmpval) - 1); @@ -4977,3 +4978,210 @@ satisfies_hash_partition(PG_FUNCTION_ARGS) PG_RETURN_BOOL(rowHash % modulus == remainder); } + +/* + * check_two_partitions_bounds_range + * + * (function for BY RANGE partitioning) + * + * This is a helper function for check_partitions_for_split() and + * calculate_partition_bound_for_merge(). + * This function compares upper bound of first_bound and lower bound of + * second_bound. These bounds should be equal except when + * "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 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 determining 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 of + * 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 the lower bound of merged + * partition. + */ + spec->lowerdatums = + ((PartitionBoundSpec *) list_nth(bounds, lower_bounds[0]->index))->lowerdatums; + + /* + * Upper bound of last partition is the 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); + } +} diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c index a7ccde6d7d..c3ea87fcb9 100644 --- a/src/bin/psql/tab-complete.c +++ b/src/bin/psql/tab-complete.c @@ -2353,6 +2353,7 @@ psql_completion(const char *text, int start, int end) "OWNER TO", "SET", "VALIDATE CONSTRAINT", "REPLICA IDENTITY", "ATTACH PARTITION", "DETACH PARTITION", "FORCE ROW LEVEL SECURITY", + "MERGE PARTITIONS (", "OF", "NOT OF"); /* ALTER TABLE xxx ADD */ else if (Matches("ALTER", "TABLE", MatchAny, "ADD")) @@ -2620,6 +2621,15 @@ psql_completion(const char *text, int start, int end) else if (Matches("ALTER", "TABLE", MatchAny, "DETACH", "PARTITION", MatchAny)) COMPLETE_WITH("CONCURRENTLY", "FINALIZE"); + /* ALTER TABLE MERGE PARTITIONS ( */ + else if (Matches("ALTER", "TABLE", MatchAny, "MERGE", "PARTITIONS", "(")) + { + set_completion_reference(prev4_wd); + COMPLETE_WITH_SCHEMA_QUERY(Query_for_partition_of_table); + } + else if (Matches("ALTER", "TABLE", MatchAny, "MERGE", "PARTITIONS", "(*)")) + COMPLETE_WITH("INTO"); + /* ALTER TABLE OF */ else if (Matches("ALTER", "TABLE", MatchAny, "OF")) COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_composite_datatypes); diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h index 124d853e49..b6e7957ebb 100644 --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -937,6 +937,17 @@ typedef struct PartitionRangeDatum ParseLoc location; /* token location, or -1 if unknown */ } PartitionRangeDatum; +/* + * PartitionDesc - info about single partition for ALTER TABLE SPLIT PARTITION command + */ +typedef struct SinglePartitionSpec +{ + NodeTag type; + + RangeVar *name; /* name of partition */ + PartitionBoundSpec *bound; /* FOR VALUES, if attaching */ +} SinglePartitionSpec; + /* * PartitionCmd - info for ALTER TABLE/INDEX ATTACH/DETACH PARTITION commands */ @@ -945,6 +956,8 @@ typedef struct PartitionCmd NodeTag type; RangeVar *name; /* name of partition to attach/detach */ PartitionBoundSpec *bound; /* FOR VALUES, if attaching */ + List *partlist; /* list of partitions, for MERGE/SPLIT + * PARTITION command */ bool concurrent; } PartitionCmd; @@ -2400,6 +2413,7 @@ typedef enum AlterTableType AT_AttachPartition, /* ATTACH PARTITION */ AT_DetachPartition, /* DETACH PARTITION */ AT_DetachPartitionFinalize, /* DETACH PARTITION FINALIZE */ + AT_MergePartitions, /* MERGE PARTITIONS */ AT_AddIdentity, /* ADD IDENTITY */ AT_SetIdentity, /* SET identity column options */ AT_DropIdentity, /* DROP IDENTITY */ diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h index f8659078ce..6b256fcece 100644 --- a/src/include/parser/kwlist.h +++ b/src/include/parser/kwlist.h @@ -335,6 +335,7 @@ PG_KEYWORD("parameter", PARAMETER, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD, BARE_LABEL) +PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("path", PATH, UNRESERVED_KEYWORD, BARE_LABEL) diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h index 3d9cc1031f..0329d7bd84 100644 --- a/src/include/partitioning/partbounds.h +++ b/src/include/partitioning/partbounds.h @@ -143,4 +143,10 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc, extern int partition_hash_bsearch(PartitionBoundInfo boundinfo, int modulus, int remainder); +extern void calculate_partition_bound_for_merge(Relation parent, + List *partNames, + List *partOids, + PartitionBoundSpec *spec, + ParseState *pstate); + #endif /* PARTBOUNDS_H */ diff --git a/src/test/isolation/expected/partition-merge.out b/src/test/isolation/expected/partition-merge.out new file mode 100644 index 0000000000..98446aaab5 --- /dev/null +++ b/src/test/isolation/expected/partition-merge.out @@ -0,0 +1,199 @@ +Parsed test spec with 2 sessions + +starting permutation: s2b s2i s2c s1b s1merg s2b s2u s1c s2c s2s +step s2b: BEGIN; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s2c: COMMIT; +step s1b: BEGIN; +step s1merg: ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20; +step s2b: BEGIN; +step s2u: UPDATE tpart SET t = 'text01modif' where i = 1; +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; +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; +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; +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; +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; +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; +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; +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; +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) + diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule index 143109aa4d..c1499338d6 100644 --- a/src/test/isolation/isolation_schedule +++ b/src/test/isolation/isolation_schedule @@ -106,6 +106,7 @@ test: partition-key-update-1 test: partition-key-update-2 test: partition-key-update-3 test: partition-key-update-4 +test: partition-merge test: plpgsql-toast test: cluster-conflict test: cluster-conflict-partition diff --git a/src/test/isolation/specs/partition-merge.spec b/src/test/isolation/specs/partition-merge.spec new file mode 100644 index 0000000000..dc2b9d3445 --- /dev/null +++ b/src/test/isolation/specs/partition-merge.spec @@ -0,0 +1,54 @@ +# Verify that MERGE operation locks DML operations with partitioned table + +setup +{ + DROP TABLE IF EXISTS tpart; + CREATE TABLE tpart(i int, t text) partition by range(i); + CREATE TABLE tpart_00_10 PARTITION OF tpart FOR VALUES FROM (0) TO (10); + CREATE TABLE tpart_10_20 PARTITION OF tpart FOR VALUES FROM (10) TO (20); + CREATE TABLE tpart_20_30 PARTITION OF tpart FOR VALUES FROM (20) TO (30); + CREATE TABLE tpart_default PARTITION OF tpart DEFAULT; + INSERT INTO tpart VALUES (5, 'text05'); + INSERT INTO tpart VALUES (15, 'text15'); + INSERT INTO tpart VALUES (25, 'text25'); + INSERT INTO tpart VALUES (35, 'text35'); +} + +teardown +{ + DROP TABLE tpart; +} + +session s1 +step s1b { BEGIN; } +step s1brr { BEGIN ISOLATION LEVEL REPEATABLE READ; } +step s1bs { BEGIN ISOLATION LEVEL SERIALIZABLE; } +step s1merg { ALTER TABLE tpart MERGE PARTITIONS (tpart_00_10, tpart_10_20) INTO tpart_00_20; } +step s1c { COMMIT; } + + +session s2 +step s2b { BEGIN; } +step s2brr { BEGIN ISOLATION LEVEL REPEATABLE READ; } +step s2bs { BEGIN ISOLATION LEVEL SERIALIZABLE; } +step s2i { INSERT INTO tpart VALUES (1, 'text01'); } +step s2u { UPDATE tpart SET t = 'text01modif' where i = 1; } +step s2c { COMMIT; } +step s2s { SELECT * FROM tpart; } + + +# s2 inserts row into table. s1 starts MERGE PARTITIONS then +# s2 is trying to update inserted row and waits until s1 finishes +# 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 diff --git a/src/test/modules/test_ddl_deparse/test_ddl_deparse.c b/src/test/modules/test_ddl_deparse/test_ddl_deparse.c index 2758ae82d7..cc341afdf4 100644 --- a/src/test/modules/test_ddl_deparse/test_ddl_deparse.c +++ b/src/test/modules/test_ddl_deparse/test_ddl_deparse.c @@ -300,6 +300,9 @@ get_altertable_subcmdinfo(PG_FUNCTION_ARGS) case AT_DetachPartitionFinalize: strtype = "DETACH PARTITION ... FINALIZE"; break; + case AT_MergePartitions: + strtype = "MERGE PARTITIONS"; + break; case AT_AddIdentity: strtype = "ADD IDENTITY"; break; diff --git a/src/test/regress/expected/partition_merge.out b/src/test/regress/expected/partition_merge.out new file mode 100644 index 0000000000..59836e2d35 --- /dev/null +++ b/src/test/regress/expected/partition_merge.out @@ -0,0 +1,945 @@ +-- +-- PARTITIONS_MERGE +-- Tests for "ALTER TABLE ... MERGE PARTITIONS ..." command +-- +CREATE SCHEMA partitions_merge_schema; +CREATE SCHEMA partitions_merge_schema2; +SET search_path = partitions_merge_schema, public; +-- +-- BY RANGE partitioning +-- +-- +-- Test for error codes +-- +CREATE TABLE sales_range (salesperson_id INT, salesperson_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 (salesperson_id INT, salesperson_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" is 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" is already used +LINE 1: ...e MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_feb2... + ^ +-- ERROR: "sales_apr2022" is not a table +ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO sales_feb_mar_apr2022; +ERROR: "sales_apr2022" is not a table +-- ERROR: lower bound of partition "sales_mar2022" conflicts with upper bound of previous partition "sales_jan2022" +-- (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: lower bound of partition "sales_jan2022" conflicts with upper bound of previous partition "sales_dec2021" +-- (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, source partitions not in the search_path +SET search_path = partitions_merge_schema2, public; +ALTER TABLE partitions_merge_schema.sales_range MERGE PARTITIONS ( + partitions_merge_schema.sales_feb2022, + partitions_merge_schema.sales_mar2022, + partitions_merge_schema.sales_jan2022) INTO sales_jan_feb_mar2022; +SET search_path = partitions_merge_schema, public; +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 +------------------------------------------------+---------+------------------+-------------------------------------------------- + partitions_merge_schema2.sales_jan_feb_mar2022 | r | f | FOR VALUES FROM ('01-01-2022') TO ('04-01-2022') + 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_others | r | f | DEFAULT +(4 rows) + +DROP TABLE sales_range; +-- +-- Add rows into partitioned table, then merge partitions +-- +CREATE TABLE sales_range (salesperson_id INT, salesperson_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) + +-- check schema-qualified name of the new partition +ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO partitions_merge_schema2.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 +------------------------------------------------+---------+------------------+-------------------------------------------------- + partitions_merge_schema2.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_schema2'; + schemaname | tablename | indexname | tablespace | indexdef +--------------------------+-----------------------+--------------------------------------+------------+------------------------------------------------------------------------------------------------------------------------------ + partitions_merge_schema2 | sales_feb_mar_apr2022 | sales_feb_mar_apr2022_sales_date_idx | | CREATE INDEX sales_feb_mar_apr2022_sales_date_idx ON partitions_merge_schema2.sales_feb_mar_apr2022 USING btree (sales_date) +(1 row) + +SELECT * FROM sales_range; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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 partitions_merge_schema2.sales_feb_mar_apr2022; + salesperson_id | salesperson_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; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 14 | Smith | 510 | 05-04-2022 +(1 row) + +-- Use indexscan for testing indexes +SET enable_seqscan = OFF; +SELECT * FROM partitions_merge_schema2.sales_feb_mar_apr2022 where sales_date > '2022-01-01'; + salesperson_id | salesperson_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 (salesperson_id INT, salesperson_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, partitions_merge_schema.sales_others) INTO sales_others; +select * from sales_others; + salesperson_id | salesperson_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 (salesperson_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(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2021, 12, 7); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2021, 12, 8); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 1, 1); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 2, 4); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 1, 2); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 2, 1); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 3, 3); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 3, 4); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 5, 1); +SELECT * FROM sales_date; + salesperson_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; + salesperson_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; + salesperson_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; + salesperson_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; + salesperson_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(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 1, 10); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 2, 10); +SELECT * FROM sales_date; + salesperson_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; + salesperson_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; + salesperson_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; + salesperson_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 salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); +CREATE TABLE salespeople01_10 PARTITION OF salespeople FOR VALUES FROM (1) TO (10); +CREATE TABLE salespeople10_20 PARTITION OF salespeople FOR VALUES FROM (10) TO (20); +CREATE TABLE salespeople20_30 PARTITION OF salespeople FOR VALUES FROM (20) TO (30); +CREATE TABLE salespeople30_40 PARTITION OF salespeople FOR VALUES FROM (30) TO (40); +INSERT INTO salespeople 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 salespeople_after_insert_statement_trigger + AFTER INSERT + ON salespeople + FOR EACH STATEMENT + EXECUTE PROCEDURE after_insert_row_trigger('salespeople'); +CREATE TRIGGER salespeople_after_insert_row_trigger + AFTER INSERT + ON salespeople + FOR EACH ROW + EXECUTE PROCEDURE after_insert_row_trigger('salespeople'); +-- 2 triggers should fire here (row + statement): +INSERT INTO salespeople VALUES (10, 'May'); +NOTICE: trigger(salespeople) called: action = INSERT, when = AFTER, level = ROW +NOTICE: trigger(salespeople) called: action = INSERT, when = AFTER, level = STATEMENT +-- 1 trigger should fire here (row): +INSERT INTO salespeople10_20 VALUES (19, 'Ivanov'); +NOTICE: trigger(salespeople) called: action = INSERT, when = AFTER, level = ROW +ALTER TABLE salespeople MERGE PARTITIONS (salespeople10_20, salespeople20_30, salespeople30_40) INTO salespeople10_40; +-- 2 triggers should fire here (row + statement): +INSERT INTO salespeople VALUES (20, 'Smirnoff'); +NOTICE: trigger(salespeople) called: action = INSERT, when = AFTER, level = ROW +NOTICE: trigger(salespeople) called: action = INSERT, when = AFTER, level = STATEMENT +-- 1 trigger should fire here (row): +INSERT INTO salespeople10_40 VALUES (30, 'Ford'); +NOTICE: trigger(salespeople) called: action = INSERT, when = AFTER, level = ROW +SELECT * FROM salespeople01_10; + salesperson_id | salesperson_name +----------------+------------------ + 1 | Poirot +(1 row) + +SELECT * FROM salespeople10_40; + salesperson_id | salesperson_name +----------------+------------------ + 10 | May + 19 | Ivanov + 20 | Smirnoff + 30 | Ford +(4 rows) + +DROP TABLE salespeople; +DROP FUNCTION after_insert_row_trigger(); +-- +-- Test: merge partitions with deleted columns +-- +CREATE TABLE salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); +CREATE TABLE salespeople01_10 PARTITION OF salespeople FOR VALUES FROM (1) TO (10); +-- Create partitions with some deleted columns: +CREATE TABLE salespeople10_20(d1 VARCHAR(30), salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)); +CREATE TABLE salespeople20_30(salesperson_id INT PRIMARY KEY, d2 INT, salesperson_name VARCHAR(30)); +CREATE TABLE salespeople30_40(salesperson_id INT PRIMARY KEY, d3 DATE, salesperson_name VARCHAR(30)); +INSERT INTO salespeople10_20 VALUES ('dummy value 1', 19, 'Ivanov'); +INSERT INTO salespeople20_30 VALUES (20, 101, 'Smirnoff'); +INSERT INTO salespeople30_40 VALUES (31, now(), 'Popov'); +ALTER TABLE salespeople10_20 DROP COLUMN d1; +ALTER TABLE salespeople20_30 DROP COLUMN d2; +ALTER TABLE salespeople30_40 DROP COLUMN d3; +ALTER TABLE salespeople ATTACH PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20); +ALTER TABLE salespeople ATTACH PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30); +ALTER TABLE salespeople ATTACH PARTITION salespeople30_40 FOR VALUES FROM (30) TO (40); +INSERT INTO salespeople VALUES (1, 'Poirot'); +INSERT INTO salespeople VALUES (10, 'May'); +INSERT INTO salespeople VALUES (30, 'Ford'); +ALTER TABLE salespeople MERGE PARTITIONS (salespeople10_20, salespeople20_30, salespeople30_40) INTO salespeople10_40; +select * from salespeople; + salesperson_id | salesperson_name +----------------+------------------ + 1 | Poirot + 19 | Ivanov + 10 | May + 20 | Smirnoff + 31 | Popov + 30 | Ford +(6 rows) + +select * from salespeople01_10; + salesperson_id | salesperson_name +----------------+------------------ + 1 | Poirot +(1 row) + +select * from salespeople10_40; + salesperson_id | salesperson_name +----------------+------------------ + 19 | Ivanov + 10 | May + 20 | Smirnoff + 31 | Popov + 30 | Ford +(5 rows) + +DROP TABLE salespeople; +-- +-- Test: merge sub-partitions +-- +CREATE TABLE sales_range (salesperson_id INT, salesperson_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 (salesperson_id INT, salesperson_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; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 4 | Ivanov | 750 | 04-13-2022 +(1 row) + +SELECT * FROM sales_apr2022_20_30; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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 +(salesperson_id INT GENERATED ALWAYS AS IDENTITY, + salesperson_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: "sales_external" is not a partition +ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_external) INTO sales_all; +ERROR: "sales_external" is not a partition +-- ERROR: "sales_external2" is not a partition +ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_external2) INTO sales_all; +ERROR: "sales_external2" is not a partition +-- 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 +(salesperson_id INT GENERATED ALWAYS AS IDENTITY, + salesperson_name VARCHAR(30), + sales_state VARCHAR(20), + sales_amount INT, + sales_date DATE) +PARTITION BY LIST (sales_state); +CREATE INDEX sales_list_salesperson_name_idx ON sales_list USING btree (salesperson_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 (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Trump', 'Bejing', 1000, '2022-03-01'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Smirnoff', 'New York', 500, '2022-03-03'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Ford', 'St. Petersburg', 2000, '2022-03-05'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Ivanov', 'Warsaw', 750, '2022-03-04'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Deev', 'Lisbon', 250, '2022-03-07'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Poirot', 'Berlin', 1000, '2022-03-01'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('May', 'Helsinki', 1200, '2022-03-06'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Li', 'Vladivostok', 1150, '2022-03-09'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('May', 'Helsinki', 1200, '2022-03-11'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Halder', 'Oslo', 800, '2022-03-02'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Muller', 'Madrid', 650, '2022-03-05'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Smith', 'Kyiv', 350, '2022-03-10'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Gandi', 'Warsaw', 150, '2022-03-08'); +INSERT INTO sales_list (salesperson_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; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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; + salesperson_id | salesperson_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 testing indexes after merging partitions +SET enable_seqscan = OFF; +SELECT * FROM sales_all WHERE sales_state = 'Warsaw'; + salesperson_id | salesperson_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'; + salesperson_id | salesperson_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 salesperson_name = 'Ivanov'; + salesperson_id | salesperson_name | sales_state | sales_amount | sales_date +----------------+------------------+-------------+--------------+------------ + 4 | Ivanov | Warsaw | 750 | 03-04-2022 +(1 row) + +RESET enable_seqscan; +DROP TABLE sales_list; +-- +-- Try to MERGE partitions of another table. +-- +CREATE TABLE t1 (i int, a int, b int, c int) PARTITION BY RANGE (a, b); +CREATE TABLE t1p1 PARTITION OF t1 FOR VALUES FROM (1, 1) TO (1, 2); +CREATE TABLE t2 (i int, t text) PARTITION BY RANGE (t); +CREATE TABLE t2pa PARTITION OF t2 FOR VALUES FROM ('A') TO ('C'); +CREATE TABLE t3 (i int, t text); +-- ERROR: relation "t1p1" is not a partition of relation "t2" +ALTER TABLE t2 MERGE PARTITIONS (t1p1, t2pa) INTO t2p; +ERROR: relation "t1p1" is not a partition of relation "t2" +-- ERROR: "t3" is not a partition +ALTER TABLE t2 MERGE PARTITIONS (t2pa, t3) INTO t2p; +ERROR: "t3" is not a partition +DROP TABLE t3; +DROP TABLE t2; +DROP TABLE t1; +-- +-- Try to MERGE partitions of temporary table. +-- +CREATE TEMP TABLE t (i int) PARTITION BY RANGE (i); +CREATE TEMP TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TEMP TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + oid | pg_get_expr | relpersistence +--------+----------------------------+---------------- + tp_0_1 | FOR VALUES FROM (0) TO (1) | t + tp_1_2 | FOR VALUES FROM (1) TO (2) | t +(2 rows) + +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +-- Partition should be temporary. +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + oid | pg_get_expr | relpersistence +--------+----------------------------+---------------- + tp_0_2 | FOR VALUES FROM (0) TO (2) | t +(1 row) + +DROP TABLE t; +-- +-- Check the partition index name if the partition name is the same as one +-- of the merged partitions. +-- +CREATE TABLE t (i int, PRIMARY KEY(i)) PARTITION BY RANGE (i); +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +CREATE INDEX tidx ON t(i); +ALTER TABLE t MERGE PARTITIONS (tp_1_2, tp_0_1) INTO tp_1_2; +-- Indexname values should be 'tp_1_2_pkey' and 'tp_1_2_i_idx'. +-- Not-null constraint name should be 'tp_1_2_i_not_null'. +\d+ tp_1_2 + Table "partitions_merge_schema.tp_1_2" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + i | integer | | not null | | plain | | +Partition of: t FOR VALUES FROM (0) TO (2) +Partition constraint: ((i IS NOT NULL) AND (i >= 0) AND (i < 2)) +Indexes: + "tp_1_2_pkey" PRIMARY KEY, btree (i) + "tp_1_2_i_idx" btree (i) + +DROP TABLE t; +-- +-- Try mixing permanent and temporary partitions. +-- +SET search_path = partitions_merge_schema, pg_temp, public; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +SELECT c.oid::pg_catalog.regclass, c.relpersistence FROM pg_catalog.pg_class c WHERE c.oid = 't'::regclass; + oid | relpersistence +-----+---------------- + t | p +(1 row) + +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + oid | pg_get_expr | relpersistence +--------+----------------------------+---------------- + tp_0_1 | FOR VALUES FROM (0) TO (1) | p + tp_1_2 | FOR VALUES FROM (1) TO (2) | p +(2 rows) + +SET search_path = pg_temp, partitions_merge_schema, public; +-- Can't merge persistent partitions into a temporary partition +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +ERROR: cannot create a temporary relation as partition of permanent relation "t" +SET search_path = partitions_merge_schema, public; +-- Can't merge persistent partitions into a temporary partition +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO pg_temp.tp_0_2; +ERROR: cannot create a temporary relation as partition of permanent relation "t" +DROP TABLE t; +SET search_path = pg_temp, partitions_merge_schema, public; +BEGIN; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +SELECT c.oid::pg_catalog.regclass, c.relpersistence FROM pg_catalog.pg_class c WHERE c.oid = 't'::regclass; + oid | relpersistence +-----+---------------- + t | t +(1 row) + +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + oid | pg_get_expr | relpersistence +--------+----------------------------+---------------- + tp_0_1 | FOR VALUES FROM (0) TO (1) | t + tp_1_2 | FOR VALUES FROM (1) TO (2) | t +(2 rows) + +SET search_path = partitions_merge_schema, pg_temp, public; +-- Can't merge temporary partitions into a persistent partition +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +ROLLBACK; +-- Check the new partition inherits parent's tablespace +SET search_path = partitions_merge_schema, public; +CREATE TABLE t (i int PRIMARY KEY USING INDEX TABLESPACE regress_tblspace) + PARTITION BY RANGE (i) TABLESPACE regress_tblspace; +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +SELECT tablename, tablespace FROM pg_tables + WHERE tablename IN ('t', 'tp_0_2') AND schemaname = 'partitions_merge_schema' + ORDER BY tablename, tablespace; + tablename | tablespace +-----------+------------------ + t | regress_tblspace + tp_0_2 | regress_tblspace +(2 rows) + +SELECT tablename, indexname, tablespace FROM pg_indexes + WHERE tablename IN ('t', 'tp_0_2') AND schemaname = 'partitions_merge_schema' + ORDER BY tablename, indexname, tablespace; + tablename | indexname | tablespace +-----------+-------------+------------------ + t | t_pkey | regress_tblspace + tp_0_2 | tp_0_2_pkey | regress_tblspace +(2 rows) + +DROP TABLE t; +-- Check the new partition inherits parent's table access method +SET search_path = partitions_merge_schema, public; +CREATE ACCESS METHOD partitions_merge_heap TYPE TABLE HANDLER heap_tableam_handler; +CREATE TABLE t (i int) PARTITION BY RANGE (i) USING partitions_merge_heap; +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +SELECT c.relname, a.amname +FROM pg_class c JOIN pg_am a ON c.relam = a.oid +WHERE c.oid IN ('t'::regclass, 'tp_0_2'::regclass) +ORDER BY c.relname; + relname | amname +---------+----------------------- + t | partitions_merge_heap + tp_0_2 | partitions_merge_heap +(2 rows) + +DROP TABLE t; +DROP ACCESS METHOD partitions_merge_heap; +-- Test permission checks. The user needs to own the parent table and all +-- the merging partitions to do the merge. +CREATE ROLE regress_partition_merge_alice; +CREATE ROLE regress_partition_merge_bob; +GRANT ALL ON SCHEMA partitions_merge_schema TO regress_partition_merge_alice; +GRANT ALL ON SCHEMA partitions_merge_schema TO regress_partition_merge_bob; +SET SESSION AUTHORIZATION regress_partition_merge_alice; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +ERROR: must be owner of table t +RESET SESSION AUTHORIZATION; +ALTER TABLE t OWNER TO regress_partition_merge_bob; +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +ERROR: must be owner of table tp_0_1 +RESET SESSION AUTHORIZATION; +ALTER TABLE tp_0_1 OWNER TO regress_partition_merge_bob; +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +ERROR: must be owner of table tp_1_2 +RESET SESSION AUTHORIZATION; +ALTER TABLE tp_1_2 OWNER TO regress_partition_merge_bob; +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +RESET SESSION AUTHORIZATION; +DROP TABLE t; +REVOKE ALL ON SCHEMA partitions_merge_schema FROM regress_partition_merge_alice; +REVOKE ALL ON SCHEMA partitions_merge_schema FROM regress_partition_merge_bob; +DROP ROLE regress_partition_merge_alice; +DROP ROLE regress_partition_merge_bob; +RESET search_path; +-- +DROP SCHEMA partitions_merge_schema; +DROP SCHEMA partitions_merge_schema2; diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule index f53a526f7c..0014801684 100644 --- a/src/test/regress/parallel_schedule +++ b/src/test/regress/parallel_schedule @@ -119,7 +119,7 @@ test: plancache limit plpgsql copy2 temp domain rangefuncs prepare conversion tr # The stats test resets stats, so nothing else needing stats access can be in # this group. # ---------- -test: partition_join partition_prune reloptions hash_part indexing partition_aggregate partition_info tuplesort explain compression memoize stats predicate +test: partition_merge partition_join partition_prune reloptions hash_part indexing partition_aggregate partition_info tuplesort explain compression memoize stats predicate # event_trigger depends on create_am and cannot run concurrently with # any test that runs DDL diff --git a/src/test/regress/sql/partition_merge.sql b/src/test/regress/sql/partition_merge.sql new file mode 100644 index 0000000000..bede819af9 --- /dev/null +++ b/src/test/regress/sql/partition_merge.sql @@ -0,0 +1,609 @@ +-- +-- PARTITIONS_MERGE +-- Tests for "ALTER TABLE ... MERGE PARTITIONS ..." command +-- + +CREATE SCHEMA partitions_merge_schema; +CREATE SCHEMA partitions_merge_schema2; +SET search_path = partitions_merge_schema, public; + +-- +-- BY RANGE partitioning +-- + +-- +-- Test for error codes +-- +CREATE TABLE sales_range (salesperson_id INT, salesperson_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 (salesperson_id INT, salesperson_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" is already used +ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_feb2022) INTO sales_feb_mar_apr2022; +-- ERROR: "sales_apr2022" is not a table +ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO sales_feb_mar_apr2022; +-- ERROR: lower bound of partition "sales_mar2022" conflicts with upper bound of previous partition "sales_jan2022" +-- (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_jan2022" conflicts with upper bound of previous partition "sales_dec2021" +-- (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, source partitions not in the search_path +SET search_path = partitions_merge_schema2, public; +ALTER TABLE partitions_merge_schema.sales_range MERGE PARTITIONS ( + partitions_merge_schema.sales_feb2022, + partitions_merge_schema.sales_mar2022, + partitions_merge_schema.sales_jan2022) INTO sales_jan_feb_mar2022; +SET search_path = partitions_merge_schema, public; + +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 (salesperson_id INT, salesperson_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; + +-- check schema-qualified name of the new partition +ALTER TABLE sales_range MERGE PARTITIONS (sales_feb2022, sales_mar2022, sales_apr2022) INTO partitions_merge_schema2.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_schema2'; + +SELECT * FROM sales_range; +SELECT * FROM sales_jan2022; +SELECT * FROM partitions_merge_schema2.sales_feb_mar_apr2022; +SELECT * FROM sales_others; + +-- Use indexscan for testing indexes +SET enable_seqscan = OFF; + +SELECT * FROM partitions_merge_schema2.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 (salesperson_id INT, salesperson_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, partitions_merge_schema.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 (salesperson_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(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2021, 12, 7); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2021, 12, 8); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 1, 1); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 2, 4); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 1, 2); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager3', 2022, 2, 1); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 3, 3); +INSERT INTO sales_date(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager2', 2022, 3, 4); +INSERT INTO sales_date(salesperson_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(salesperson_name, sales_year, sales_month, sales_day) VALUES ('Manager1', 2022, 1, 10); +INSERT INTO sales_date(salesperson_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 salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); + +CREATE TABLE salespeople01_10 PARTITION OF salespeople FOR VALUES FROM (1) TO (10); +CREATE TABLE salespeople10_20 PARTITION OF salespeople FOR VALUES FROM (10) TO (20); +CREATE TABLE salespeople20_30 PARTITION OF salespeople FOR VALUES FROM (20) TO (30); +CREATE TABLE salespeople30_40 PARTITION OF salespeople FOR VALUES FROM (30) TO (40); + +INSERT INTO salespeople 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 salespeople_after_insert_statement_trigger + AFTER INSERT + ON salespeople + FOR EACH STATEMENT + EXECUTE PROCEDURE after_insert_row_trigger('salespeople'); + +CREATE TRIGGER salespeople_after_insert_row_trigger + AFTER INSERT + ON salespeople + FOR EACH ROW + EXECUTE PROCEDURE after_insert_row_trigger('salespeople'); + +-- 2 triggers should fire here (row + statement): +INSERT INTO salespeople VALUES (10, 'May'); +-- 1 trigger should fire here (row): +INSERT INTO salespeople10_20 VALUES (19, 'Ivanov'); + +ALTER TABLE salespeople MERGE PARTITIONS (salespeople10_20, salespeople20_30, salespeople30_40) INTO salespeople10_40; + +-- 2 triggers should fire here (row + statement): +INSERT INTO salespeople VALUES (20, 'Smirnoff'); +-- 1 trigger should fire here (row): +INSERT INTO salespeople10_40 VALUES (30, 'Ford'); + +SELECT * FROM salespeople01_10; +SELECT * FROM salespeople10_40; + +DROP TABLE salespeople; +DROP FUNCTION after_insert_row_trigger(); + +-- +-- Test: merge partitions with deleted columns +-- +CREATE TABLE salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); + +CREATE TABLE salespeople01_10 PARTITION OF salespeople FOR VALUES FROM (1) TO (10); +-- Create partitions with some deleted columns: +CREATE TABLE salespeople10_20(d1 VARCHAR(30), salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)); +CREATE TABLE salespeople20_30(salesperson_id INT PRIMARY KEY, d2 INT, salesperson_name VARCHAR(30)); +CREATE TABLE salespeople30_40(salesperson_id INT PRIMARY KEY, d3 DATE, salesperson_name VARCHAR(30)); + +INSERT INTO salespeople10_20 VALUES ('dummy value 1', 19, 'Ivanov'); +INSERT INTO salespeople20_30 VALUES (20, 101, 'Smirnoff'); +INSERT INTO salespeople30_40 VALUES (31, now(), 'Popov'); + +ALTER TABLE salespeople10_20 DROP COLUMN d1; +ALTER TABLE salespeople20_30 DROP COLUMN d2; +ALTER TABLE salespeople30_40 DROP COLUMN d3; + +ALTER TABLE salespeople ATTACH PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20); +ALTER TABLE salespeople ATTACH PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30); +ALTER TABLE salespeople ATTACH PARTITION salespeople30_40 FOR VALUES FROM (30) TO (40); + +INSERT INTO salespeople VALUES (1, 'Poirot'); +INSERT INTO salespeople VALUES (10, 'May'); +INSERT INTO salespeople VALUES (30, 'Ford'); + +ALTER TABLE salespeople MERGE PARTITIONS (salespeople10_20, salespeople20_30, salespeople30_40) INTO salespeople10_40; + +select * from salespeople; +select * from salespeople01_10; +select * from salespeople10_40; + +DROP TABLE salespeople; + +-- +-- Test: merge sub-partitions +-- +CREATE TABLE sales_range (salesperson_id INT, salesperson_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 (salesperson_id INT, salesperson_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 +(salesperson_id INT GENERATED ALWAYS AS IDENTITY, + salesperson_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: "sales_external" is not a partition +ALTER TABLE sales_list MERGE PARTITIONS (sales_west, sales_east, sales_external) INTO sales_all; +-- ERROR: "sales_external2" is not a partition +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 +(salesperson_id INT GENERATED ALWAYS AS IDENTITY, + salesperson_name VARCHAR(30), + sales_state VARCHAR(20), + sales_amount INT, + sales_date DATE) +PARTITION BY LIST (sales_state); + +CREATE INDEX sales_list_salesperson_name_idx ON sales_list USING btree (salesperson_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 (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Trump', 'Bejing', 1000, '2022-03-01'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Smirnoff', 'New York', 500, '2022-03-03'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Ford', 'St. Petersburg', 2000, '2022-03-05'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Ivanov', 'Warsaw', 750, '2022-03-04'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Deev', 'Lisbon', 250, '2022-03-07'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Poirot', 'Berlin', 1000, '2022-03-01'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('May', 'Helsinki', 1200, '2022-03-06'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Li', 'Vladivostok', 1150, '2022-03-09'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('May', 'Helsinki', 1200, '2022-03-11'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Halder', 'Oslo', 800, '2022-03-02'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Muller', 'Madrid', 650, '2022-03-05'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Smith', 'Kyiv', 350, '2022-03-10'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Gandi', 'Warsaw', 150, '2022-03-08'); +INSERT INTO sales_list (salesperson_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 testing indexes after merging 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 salesperson_name = 'Ivanov'; + +RESET enable_seqscan; + +DROP TABLE sales_list; + +-- +-- Try to MERGE partitions of another table. +-- +CREATE TABLE t1 (i int, a int, b int, c int) PARTITION BY RANGE (a, b); +CREATE TABLE t1p1 PARTITION OF t1 FOR VALUES FROM (1, 1) TO (1, 2); +CREATE TABLE t2 (i int, t text) PARTITION BY RANGE (t); +CREATE TABLE t2pa PARTITION OF t2 FOR VALUES FROM ('A') TO ('C'); +CREATE TABLE t3 (i int, t text); + +-- ERROR: relation "t1p1" is not a partition of relation "t2" +ALTER TABLE t2 MERGE PARTITIONS (t1p1, t2pa) INTO t2p; +-- ERROR: "t3" is not a partition +ALTER TABLE t2 MERGE PARTITIONS (t2pa, t3) INTO t2p; + +DROP TABLE t3; +DROP TABLE t2; +DROP TABLE t1; + +-- +-- Try to MERGE partitions of temporary table. +-- +CREATE TEMP TABLE t (i int) PARTITION BY RANGE (i); +CREATE TEMP TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TEMP TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); + +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; + +-- Partition should be temporary. +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + +DROP TABLE t; + +-- +-- Check the partition index name if the partition name is the same as one +-- of the merged partitions. +-- +CREATE TABLE t (i int, PRIMARY KEY(i)) PARTITION BY RANGE (i); + +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); + +CREATE INDEX tidx ON t(i); +ALTER TABLE t MERGE PARTITIONS (tp_1_2, tp_0_1) INTO tp_1_2; + +-- Indexname values should be 'tp_1_2_pkey' and 'tp_1_2_i_idx'. +-- Not-null constraint name should be 'tp_1_2_i_not_null'. +\d+ tp_1_2 + +DROP TABLE t; + +-- +-- Try mixing permanent and temporary partitions. +-- +SET search_path = partitions_merge_schema, pg_temp, public; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); + +SELECT c.oid::pg_catalog.regclass, c.relpersistence FROM pg_catalog.pg_class c WHERE c.oid = 't'::regclass; +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + +SET search_path = pg_temp, partitions_merge_schema, public; + +-- Can't merge persistent partitions into a temporary partition +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; + +SET search_path = partitions_merge_schema, public; + +-- Can't merge persistent partitions into a temporary partition +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO pg_temp.tp_0_2; +DROP TABLE t; + +SET search_path = pg_temp, partitions_merge_schema, public; + +BEGIN; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); + +SELECT c.oid::pg_catalog.regclass, c.relpersistence FROM pg_catalog.pg_class c WHERE c.oid = 't'::regclass; +SELECT c.oid::pg_catalog.regclass, pg_catalog.pg_get_expr(c.relpartbound, c.oid), c.relpersistence + FROM pg_catalog.pg_class c, pg_catalog.pg_inherits i + WHERE c.oid = i.inhrelid AND i.inhparent = 't'::regclass + ORDER BY pg_catalog.pg_get_expr(c.relpartbound, c.oid) = 'DEFAULT', c.oid::pg_catalog.regclass::pg_catalog.text; + +SET search_path = partitions_merge_schema, pg_temp, public; + +-- Can't merge temporary partitions into a persistent partition +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +ROLLBACK; + +-- Check the new partition inherits parent's tablespace +SET search_path = partitions_merge_schema, public; +CREATE TABLE t (i int PRIMARY KEY USING INDEX TABLESPACE regress_tblspace) + PARTITION BY RANGE (i) TABLESPACE regress_tblspace; +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +SELECT tablename, tablespace FROM pg_tables + WHERE tablename IN ('t', 'tp_0_2') AND schemaname = 'partitions_merge_schema' + ORDER BY tablename, tablespace; +SELECT tablename, indexname, tablespace FROM pg_indexes + WHERE tablename IN ('t', 'tp_0_2') AND schemaname = 'partitions_merge_schema' + ORDER BY tablename, indexname, tablespace; +DROP TABLE t; + +-- Check the new partition inherits parent's table access method +SET search_path = partitions_merge_schema, public; +CREATE ACCESS METHOD partitions_merge_heap TYPE TABLE HANDLER heap_tableam_handler; +CREATE TABLE t (i int) PARTITION BY RANGE (i) USING partitions_merge_heap; +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +SELECT c.relname, a.amname +FROM pg_class c JOIN pg_am a ON c.relam = a.oid +WHERE c.oid IN ('t'::regclass, 'tp_0_2'::regclass) +ORDER BY c.relname; +DROP TABLE t; +DROP ACCESS METHOD partitions_merge_heap; + +-- Test permission checks. The user needs to own the parent table and all +-- the merging partitions to do the merge. +CREATE ROLE regress_partition_merge_alice; +CREATE ROLE regress_partition_merge_bob; +GRANT ALL ON SCHEMA partitions_merge_schema TO regress_partition_merge_alice; +GRANT ALL ON SCHEMA partitions_merge_schema TO regress_partition_merge_bob; + +SET SESSION AUTHORIZATION regress_partition_merge_alice; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_1 PARTITION OF t FOR VALUES FROM (0) TO (1); +CREATE TABLE tp_1_2 PARTITION OF t FOR VALUES FROM (1) TO (2); + +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +RESET SESSION AUTHORIZATION; + +ALTER TABLE t OWNER TO regress_partition_merge_bob; +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +RESET SESSION AUTHORIZATION; + +ALTER TABLE tp_0_1 OWNER TO regress_partition_merge_bob; +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +RESET SESSION AUTHORIZATION; + +ALTER TABLE tp_1_2 OWNER TO regress_partition_merge_bob; +SET SESSION AUTHORIZATION regress_partition_merge_bob; +ALTER TABLE t MERGE PARTITIONS (tp_0_1, tp_1_2) INTO tp_0_2; +RESET SESSION AUTHORIZATION; + +DROP TABLE t; +REVOKE ALL ON SCHEMA partitions_merge_schema FROM regress_partition_merge_alice; +REVOKE ALL ON SCHEMA partitions_merge_schema FROM regress_partition_merge_bob; +DROP ROLE regress_partition_merge_alice; +DROP ROLE regress_partition_merge_bob; + +RESET search_path; + +-- +DROP SCHEMA partitions_merge_schema; +DROP SCHEMA partitions_merge_schema2; diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list index 9e951a9e6f..ddc4c78fb9 100644 --- a/src/tools/pgindent/typedefs.list +++ b/src/tools/pgindent/typedefs.list @@ -2659,6 +2659,7 @@ SimpleStats SimpleStringList SimpleStringListCell SingleBoundSortItem +SinglePartitionSpec Size SkipPages SlabBlock -- 2.40.1.windows.1