From 71a05398ce52c634cdf3a1cee45ada938118d125 Mon Sep 17 00:00:00 2001 From: Alexander Korotkov Date: Sun, 7 Apr 2024 00:58:09 +0300 Subject: [PATCH v32 2/2] Implement ALTER TABLE ... SPLIT PARTITION ... command This new DDL command splits a single partition into several parititions. Just like ALTER TABLE ... MERGE PARTITIONS ... command, new patitions are created using 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 | 79 +- src/backend/commands/tablecmds.c | 398 +++++ src/backend/parser/gram.y | 38 +- src/backend/parser/parse_utilcmd.c | 64 +- src/backend/partitioning/partbounds.c | 689 +++++++ src/backend/utils/adt/ruleutils.c | 18 + src/bin/psql/tab-complete.c | 10 +- src/include/nodes/parsenodes.h | 1 + src/include/parser/kwlist.h | 1 + src/include/partitioning/partbounds.h | 5 + src/include/utils/ruleutils.h | 2 + .../isolation/expected/partition-split.out | 190 ++ src/test/isolation/isolation_schedule | 1 + src/test/isolation/specs/partition-split.spec | 54 + .../test_ddl_deparse/test_ddl_deparse.c | 3 + src/test/regress/expected/partition_split.out | 1589 +++++++++++++++++ src/test/regress/parallel_schedule | 2 +- src/test/regress/sql/partition_split.sql | 962 ++++++++++ src/tools/pgindent/typedefs.list | 1 + 20 files changed, 4109 insertions(+), 17 deletions(-) create mode 100644 src/test/isolation/expected/partition-split.out create mode 100644 src/test/isolation/specs/partition-split.spec create mode 100644 src/test/regress/expected/partition_split.out create mode 100644 src/test/regress/sql/partition_split.sql diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml index 2770e81656..5f481dda3a 100644 --- a/doc/src/sgml/ddl.sgml +++ b/doc/src/sgml/ddl.sgml @@ -4374,6 +4374,25 @@ ALTER TABLE measurement measurement_y2006m03) INTO measurement_y2006q1; + + + Similarly to merging multiple table partitions, there is an option for + splitting a single partition into multiple using the + ALTER TABLE ... SPLIT PARTITION. + This feature could come in handy when one partition grows too big + and needs to be split into multiple. 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 split + the quarter partition back to monthly partitions: + +ALTER TABLE measurement SPLIT PARTITION measurement_y2006q1 INTO + (PARTITION measurement_y2006m01 FOR VALUES FROM ('2006-01-01') TO ('2006-02-01'), + PARTITION measurement_y2006m02 FOR VALUES FROM ('2006-02-01') TO ('2006-03-01'), + PARTITION measurement_y2006m03 FOR VALUES FROM ('2006-03-01') TO ('2006-04-01')); + + + diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml index b99d4972ac..6a2822adad 100644 --- a/doc/src/sgml/ref/alter_table.sgml +++ b/doc/src/sgml/ref/alter_table.sgml @@ -37,6 +37,10 @@ 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 + SPLIT PARTITION partition_name INTO + (PARTITION partition_name1 { FOR VALUES partition_bound_spec | DEFAULT }, + PARTITION partition_name2 { FOR VALUES partition_bound_spec | DEFAULT } [, ...]) ALTER TABLE [ IF EXISTS ] name MERGE PARTITIONS (partition_name1, partition_name2 [, ...]) INTO partition_name @@ -1120,6 +1124,58 @@ WITH ( MODULUS numeric_literal, REM + + SPLIT PARTITION partition_name INTO (PARTITION partition_name1 { FOR VALUES partition_bound_spec | DEFAULT }, PARTITION partition_name2 { FOR VALUES partition_bound_spec | DEFAULT } [, ...]) + + + + This form splits a single partition of the target table. Hash-partitioning + is not supported. Bounds of new partitions should not overlap with new and + existing partitions (except partition_name). + If the split partition is a DEFAULT partition, one of the new partitions must be DEFAULT. + In case one of the new partitions or one of existing partitions is DEFAULT, + new partitions partition_name1, + partition_name2, ... can have spaces + between partitions bounds. If the partitioned table does not have a DEFAULT + partition, the DEFAULT partition can be defined as one of the new partitions. + + + In case new partitions do not contain a DEFAULT partition and the partitioned table + does not have a DEFAULT partition, the following must be true: sum bounds of + new partitions partition_name1, + partition_name2, ... should be + equal to bound of split partition partition_name. + One of the new partitions partition_name1, + partition_name2, ... can have + the same name as split partition partition_name + (this is suitable in case of splitting a DEFAULT partition: we split it, but after + splitting we have a partition with the same name). + Only simple, non-partitioned partition can be split. + + + The new partitions will be created the same as tables created with the + SQL command CREATE TABLE partition_nameN (LIKE name INCLUDING ALL EXCLUDING INDEXES EXCLUDING IDENTITY EXCLUDING STATISTICS). + The indexes and identity are created later, after moving the data + into the new partitions. + Extended statistics aren't copied from the parent table, for consistency with + CREATE TABLE PARTITION OF. + + New partitions will have the same table access method as the parent. + If the parent table is persistent then new partitions are created + persistent. If the parent table is temporary then new partitions + are also created temporary. New partitions 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. + + + + + MERGE PARTITIONS (partition_name1, partition_name2 [, ...]) INTO partition_name @@ -1200,7 +1256,8 @@ WITH ( MODULUS numeric_literal, REM All the forms of ALTER TABLE that act on a single table, except RENAME, SET SCHEMA, ATTACH PARTITION, DETACH PARTITION, - and MERGE PARTITIONS can be combined into + SPLIT 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 @@ -1444,7 +1501,7 @@ WITH ( MODULUS numeric_literal, REM 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. + or the name of split partition, or the name of the new merged partition. @@ -1860,6 +1917,24 @@ ALTER TABLE measurement DETACH PARTITION measurement_y2015m12; + + To split a single partition of the range-partitioned table: + +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2023 INTO + (PARTITION sales_feb2023 FOR VALUES FROM ('2023-02-01') TO ('2023-03-01'), + PARTITION sales_mar2023 FOR VALUES FROM ('2023-03-01') TO ('2023-04-01'), + PARTITION sales_apr2023 FOR VALUES FROM ('2023-04-01') TO ('2023-05-01')); + + + + To split a single partition of the list-partitioned table: + +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); + + To merge several partitions into one partition of the target table: diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c index 4d24278495..55db12ef35 100644 --- a/src/backend/commands/tablecmds.c +++ b/src/backend/commands/tablecmds.c @@ -657,6 +657,9 @@ static void ATDetachCheckNoForeignKeyRefs(Relation partition); static char GetAttributeCompression(Oid atttypid, const char *compression); static char GetAttributeStorage(Oid atttypid, const char *storagemode); +static void ATExecSplitPartition(List **wqueue, AlteredTableInfo *tab, + Relation rel, PartitionCmd *cmd, + AlterTableUtilityContext *context); static void ATExecMergePartitions(List **wqueue, AlteredTableInfo *tab, Relation rel, PartitionCmd *cmd, AlterTableUtilityContext *context); @@ -4678,6 +4681,10 @@ AlterTableGetLockLevel(List *cmds) cmd_lockmode = AccessShareLock; break; + case AT_SplitPartition: + cmd_lockmode = AccessExclusiveLock; + break; + case AT_MergePartitions: cmd_lockmode = AccessExclusiveLock; break; @@ -5099,6 +5106,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd, /* No command-specific prep needed */ pass = AT_PASS_MISC; break; + case AT_SplitPartition: + ATSimplePermissions(cmd->subtype, rel, ATT_TABLE); + /* No command-specific prep needed */ + pass = AT_PASS_MISC; + break; case AT_MergePartitions: ATSimplePermissions(cmd->subtype, rel, ATT_TABLE); /* No command-specific prep needed */ @@ -5500,6 +5512,14 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab, case AT_DetachPartitionFinalize: address = ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name); break; + case AT_SplitPartition: + cmd = ATParseTransformCmd(wqueue, tab, rel, cmd, false, lockmode, + cur_pass, context); + Assert(cmd != NULL); + Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE); + ATExecSplitPartition(wqueue, tab, rel, (PartitionCmd *) cmd->def, + context); + break; case AT_MergePartitions: cmd = ATParseTransformCmd(wqueue, tab, rel, cmd, false, lockmode, cur_pass, context); @@ -6496,6 +6516,8 @@ alter_table_type_to_string(AlterTableType cmdtype) return "DETACH PARTITION"; case AT_DetachPartitionFinalize: return "DETACH PARTITION ... FINALIZE"; + case AT_SplitPartition: + return "SPLIT PARTITION"; case AT_MergePartitions: return "MERGE PARTITIONS"; case AT_AddIdentity: @@ -20056,6 +20078,256 @@ GetAttributeStorage(Oid atttypid, const char *storagemode) return cstorage; } +/* + * Struct with context of new partition for inserting rows from split partition + */ +typedef struct SplitPartitionContext +{ + ExprState *partqualstate; /* expression for checking slot for partition + * (NULL for DEFAULT partition) */ + BulkInsertState bistate; /* state of bulk inserts for partition */ + TupleTableSlot *dstslot; /* slot for inserting row into partition */ + Relation partRel; /* relation for partition */ +} SplitPartitionContext; + + +/* + * createSplitPartitionContext: create context for partition and fill it + */ +static SplitPartitionContext * +createSplitPartitionContext(Relation partRel) +{ + SplitPartitionContext *pc; + + pc = (SplitPartitionContext *) palloc0(sizeof(SplitPartitionContext)); + pc->partRel = partRel; + + /* + * Prepare a BulkInsertState for table_tuple_insert. The FSM is empty, so + * don't bother using it. + */ + pc->bistate = GetBulkInsertState(); + + /* Create tuple slot for new partition. */ + pc->dstslot = MakeSingleTupleTableSlot(RelationGetDescr(pc->partRel), + table_slot_callbacks(pc->partRel)); + ExecStoreAllNullTuple(pc->dstslot); + + return pc; +} + +/* + * deleteSplitPartitionContext: delete context for partition + */ +static void +deleteSplitPartitionContext(SplitPartitionContext *pc, int ti_options) +{ + ExecDropSingleTupleTableSlot(pc->dstslot); + FreeBulkInsertState(pc->bistate); + + table_finish_bulk_insert(pc->partRel, ti_options); + + pfree(pc); +} + +/* + * moveSplitTableRows: scan split partition (splitRel) of partitioned table + * (rel) and move rows into new partitions. + * + * New partitions description: + * partlist: list of pointers to SinglePartitionSpec structures. + * newPartRels: list of Relations. + * defaultPartOid: oid of DEFAULT partition, for table rel. + */ +static void +moveSplitTableRows(Relation rel, Relation splitRel, List *partlist, List *newPartRels, Oid defaultPartOid) +{ + /* The FSM is empty, so don't bother using it. */ + int ti_options = TABLE_INSERT_SKIP_FSM; + CommandId mycid; + EState *estate; + ListCell *listptr, + *listptr2; + TupleTableSlot *srcslot; + ExprContext *econtext; + TableScanDesc scan; + Snapshot snapshot; + MemoryContext oldCxt; + List *partContexts = NIL; + TupleConversionMap *tuple_map; + SplitPartitionContext *defaultPartCtx = NULL, + *pc; + bool isOldDefaultPart = false; + + mycid = GetCurrentCommandId(true); + + estate = CreateExecutorState(); + + forboth(listptr, partlist, listptr2, newPartRels) + { + SinglePartitionSpec *sps = (SinglePartitionSpec *) lfirst(listptr); + + pc = createSplitPartitionContext((Relation) lfirst(listptr2)); + + if (sps->bound->is_default) + { + /* We should not create constraint for detached DEFAULT partition. */ + defaultPartCtx = pc; + } + else + { + List *partConstraint; + + /* Build expression execution states for partition check quals. */ + partConstraint = get_qual_from_partbound(rel, sps->bound); + partConstraint = + (List *) eval_const_expressions(NULL, + (Node *) partConstraint); + /* Make boolean expression for ExecCheck(). */ + partConstraint = list_make1(make_ands_explicit(partConstraint)); + + /* + * Map the vars in the constraint expression from rel's attnos to + * splitRel's. + */ + partConstraint = map_partition_varattnos(partConstraint, + 1, splitRel, rel); + + pc->partqualstate = + ExecPrepareExpr((Expr *) linitial(partConstraint), estate); + Assert(pc->partqualstate != NULL); + } + + /* Store partition context into list. */ + partContexts = lappend(partContexts, pc); + } + + /* + * Create partition context for DEFAULT partition. We can insert values + * into this partition in case spaces with values between new partitions. + */ + if (!defaultPartCtx && OidIsValid(defaultPartOid)) + { + /* Indicate that we allocate context for old DEFAULT partition */ + isOldDefaultPart = true; + defaultPartCtx = createSplitPartitionContext(table_open(defaultPartOid, AccessExclusiveLock)); + } + + econtext = GetPerTupleExprContext(estate); + + /* Create necessary tuple slot. */ + srcslot = MakeSingleTupleTableSlot(RelationGetDescr(splitRel), + table_slot_callbacks(splitRel)); + + /* + * Map computing for moving attributes of split partition to new partition + * (for first new partition, but other new partitions can use the same + * map). + */ + pc = (SplitPartitionContext *) lfirst(list_head(partContexts)); + tuple_map = convert_tuples_by_name(RelationGetDescr(splitRel), + RelationGetDescr(pc->partRel)); + + /* Scan through the rows. */ + snapshot = RegisterSnapshot(GetLatestSnapshot()); + scan = table_beginscan(splitRel, snapshot, 0, NULL); + + /* + * Switch to per-tuple memory context and reset it for each tuple + * produced, so we don't leak memory. + */ + oldCxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate)); + + while (table_scan_getnextslot(scan, ForwardScanDirection, srcslot)) + { + bool found = false; + TupleTableSlot *insertslot; + + /* Extract data from old tuple. */ + slot_getallattrs(srcslot); + + econtext->ecxt_scantuple = srcslot; + + /* Search partition for current slot srcslot. */ + foreach(listptr, partContexts) + { + pc = (SplitPartitionContext *) lfirst(listptr); + + if (pc->partqualstate /* skip DEFAULT partition */ && + ExecCheck(pc->partqualstate, econtext)) + { + found = true; + break; + } + ResetExprContext(econtext); + } + if (!found) + { + /* Use DEFAULT partition if it exists. */ + if (defaultPartCtx) + pc = defaultPartCtx; + else + ereport(ERROR, + (errcode(ERRCODE_CHECK_VIOLATION), + errmsg("can not find partition for split partition row"), + errtable(splitRel))); + } + + if (tuple_map) + { + /* Need to use map to copy attributes. */ + insertslot = execute_attr_map_slot(tuple_map->attrMap, srcslot, pc->dstslot); + } + else + { + /* Copy attributes directly. */ + insertslot = pc->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(pc->partRel, insertslot, mycid, + ti_options, pc->bistate); + + ResetExprContext(econtext); + + CHECK_FOR_INTERRUPTS(); + } + + MemoryContextSwitchTo(oldCxt); + + table_endscan(scan); + UnregisterSnapshot(snapshot); + + if (tuple_map) + free_conversion_map(tuple_map); + + ExecDropSingleTupleTableSlot(srcslot); + + FreeExecutorState(estate); + + foreach(listptr, partContexts) + deleteSplitPartitionContext((SplitPartitionContext *) lfirst(listptr), ti_options); + + /* Need to close table and free buffers for DEFAULT partition. */ + if (isOldDefaultPart) + { + Relation defaultPartRel = defaultPartCtx->partRel; + + deleteSplitPartitionContext(defaultPartCtx, ti_options); + /* Keep the lock until commit. */ + table_close(defaultPartRel, NoLock); + } +} + /* * createPartitionTable: create table for a new partition with given name * (newPartName) like table (modelRel) @@ -20160,6 +20432,132 @@ createPartitionTable(RangeVar *newPartName, Relation modelRel, return newRel; } +/* + * ALTER TABLE SPLIT PARTITION INTO + */ +static void +ATExecSplitPartition(List **wqueue, AlteredTableInfo *tab, Relation rel, + PartitionCmd *cmd, AlterTableUtilityContext *context) +{ + Relation splitRel; + Oid splitRelOid; + char relname[NAMEDATALEN]; + Oid namespaceId; + ListCell *listptr, + *listptr2; + bool isSameName = false; + char tmpRelName[NAMEDATALEN]; + List *newPartRels = NIL; + ObjectAddress object; + Oid defaultPartOid; + + defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, true)); + + /* + * We are going to detach and remove this partition: need to use exclusive + * lock for preventing DML-queries to the partition. + */ + splitRel = table_openrv(cmd->name, AccessExclusiveLock); + + splitRelOid = RelationGetRelid(splitRel); + + /* Check descriptions of new partitions. */ + foreach(listptr, cmd->partlist) + { + Oid existing_relid; + SinglePartitionSpec *sps = (SinglePartitionSpec *) lfirst(listptr); + + strlcpy(relname, sps->name->relname, NAMEDATALEN); + + /* + * 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. + */ + sps->name->relpersistence = rel->rd_rel->relpersistence; + namespaceId = + RangeVarGetAndCheckCreationNamespace(sps->name, NoLock, NULL); + + /* + * This would fail later on anyway if the relation already exists. But + * by catching it here we can emit a nicer error message. + */ + existing_relid = get_relname_relid(relname, namespaceId); + if (existing_relid == splitRelOid && !isSameName) + /* One new partition can have the same name as split partition. */ + isSameName = true; + else if (existing_relid != InvalidOid) + ereport(ERROR, + (errcode(ERRCODE_DUPLICATE_TABLE), + errmsg("relation \"%s\" already exists", relname))); + } + + /* Detach split partition. */ + RemoveInheritance(splitRel, rel, false); + /* Do the final part of detaching. */ + DetachPartitionFinalize(rel, splitRel, false, defaultPartOid); + + /* + * If new partition has the same name as split partition then we should + * rename split partition for reusing name. + */ + if (isSameName) + { + /* + * We must bump the command counter to make the split partition tuple + * visible for renaming. + */ + CommandCounterIncrement(); + /* Rename partition. */ + sprintf(tmpRelName, "split-%u-%X-tmp", RelationGetRelid(rel), MyProcPid); + RenameRelationInternal(splitRelOid, tmpRelName, false, false); + + /* + * We must bump the command counter to make the split partition tuple + * visible after renaming. + */ + CommandCounterIncrement(); + } + + /* Create new partitions (like split partition), without indexes. */ + foreach(listptr, cmd->partlist) + { + SinglePartitionSpec *sps = (SinglePartitionSpec *) lfirst(listptr); + Relation newPartRel; + + newPartRel = createPartitionTable(sps->name, rel, context); + newPartRels = lappend(newPartRels, newPartRel); + } + + /* Copy data from split partition to new partitions. */ + moveSplitTableRows(rel, splitRel, cmd->partlist, newPartRels, defaultPartOid); + /* Keep the lock until commit. */ + table_close(splitRel, NoLock); + + /* Attach new partitions to partitioned table. */ + forboth(listptr, cmd->partlist, listptr2, newPartRels) + { + SinglePartitionSpec *sps = (SinglePartitionSpec *) lfirst(listptr); + Relation newPartRel = (Relation) lfirst(listptr2); + + /* + * wqueue = NULL: verification for each cloned constraint is not + * needed. + */ + attachPartitionTable(NULL, rel, newPartRel, sps->bound); + /* Keep the lock until commit. */ + table_close(newPartRel, NoLock); + } + + /* Drop split partition. */ + object.classId = RelationRelationId; + object.objectId = splitRelOid; + object.objectSubId = 0; + /* Probably DROP_CASCADE is not needed. */ + performDeletion(&object, DROP_RESTRICT, 0); +} + /* * moveMergedTablesRows: scan partitions to be merged (mergingPartitionsList) * of the partitioned table (rel) and move rows into the new partition diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y index d85c74cb7c..c3f25582c3 100644 --- a/src/backend/parser/gram.y +++ b/src/backend/parser/gram.y @@ -269,6 +269,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); PartitionElem *partelem; PartitionSpec *partspec; PartitionBoundSpec *partboundspec; + SinglePartitionSpec *singlepartspec; RoleSpec *rolespec; PublicationObjSpec *publicationobjectspec; struct SelectLimit *selectlimit; @@ -645,6 +646,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); %type part_elem %type part_params %type PartitionBoundSpec +%type SinglePartitionSpec +%type partitions_list %type hash_partbound %type hash_partbound_elem @@ -775,7 +778,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); SAVEPOINT SCALAR SCHEMA SCHEMAS SCROLL SEARCH SECOND_P SECURITY SELECT SEQUENCE SEQUENCES SERIALIZABLE SERVER SESSION SESSION_USER SET SETS SETOF SHARE SHOW - SIMILAR SIMPLE SKIP SMALLINT SNAPSHOT SOME SOURCE SQL_P STABLE STANDALONE_P + SIMILAR SIMPLE SKIP SMALLINT SNAPSHOT SOME SPLIT SOURCE SQL_P STABLE STANDALONE_P START STATEMENT STATISTICS STDIN STDOUT STORAGE STORED STRICT_P STRING_P STRIP_P SUBSCRIPTION SUBSTRING SUPPORT SYMMETRIC SYSID SYSTEM_P SYSTEM_USER @@ -2308,6 +2311,23 @@ alter_table_cmds: | alter_table_cmds ',' alter_table_cmd { $$ = lappend($1, $3); } ; +partitions_list: + SinglePartitionSpec { $$ = list_make1($1); } + | partitions_list ',' SinglePartitionSpec { $$ = lappend($1, $3); } + ; + +SinglePartitionSpec: + PARTITION qualified_name PartitionBoundSpec + { + SinglePartitionSpec *n = makeNode(SinglePartitionSpec); + + n->name = $2; + n->bound = $3; + + $$ = n; + } + ; + partition_cmd: /* ALTER TABLE ATTACH PARTITION FOR VALUES */ ATTACH PARTITION qualified_name PartitionBoundSpec @@ -2352,6 +2372,20 @@ partition_cmd: n->def = (Node *) cmd; $$ = (Node *) n; } + /* ALTER TABLE SPLIT PARTITION INTO () */ + | SPLIT PARTITION qualified_name INTO '(' partitions_list ')' + { + AlterTableCmd *n = makeNode(AlterTableCmd); + PartitionCmd *cmd = makeNode(PartitionCmd); + + n->subtype = AT_SplitPartition; + cmd->name = $3; + cmd->bound = NULL; + cmd->partlist = $6; + cmd->concurrent = false; + n->def = (Node *) cmd; + $$ = (Node *) n; + } /* ALTER TABLE MERGE PARTITIONS () INTO */ | MERGE PARTITIONS '(' qualified_name_list ')' INTO qualified_name { @@ -17777,6 +17811,7 @@ unreserved_keyword: | SKIP | SNAPSHOT | SOURCE + | SPLIT | SQL_P | STABLE | STANDALONE_P @@ -18413,6 +18448,7 @@ bare_label_keyword: | SNAPSHOT | SOME | SOURCE + | SPLIT | SQL_P | STABLE | STANDALONE_P diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c index 63ea2ec5ea..d5c2b2ff0b 100644 --- a/src/backend/parser/parse_utilcmd.c +++ b/src/backend/parser/parse_utilcmd.c @@ -136,7 +136,7 @@ static void transformConstraintAttrs(CreateStmtContext *cxt, List *constraintList); static void transformColumnType(CreateStmtContext *cxt, ColumnDef *column); static void setSchemaName(const char *context_schema, char **stmt_schema_name); -static void transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd); +static void transformPartitionCmd(CreateStmtContext *cxt, PartitionBoundSpec *bound); static List *transformPartitionRangeBounds(ParseState *pstate, List *blist, Relation parent); static void validateInfiniteBounds(ParseState *pstate, List *blist); @@ -3270,6 +3270,45 @@ checkPartition(Relation rel, Oid partRelOid) relation_close(partRel, AccessShareLock); } +/* + * transformPartitionCmdForSplit + * Analyze the ALTER TABLE ... SPLIT PARTITION command + * + * For each new partition sps->bound is set to the transformed value of bound. + * Does checks for bounds of new partitions. + */ +static void +transformPartitionCmdForSplit(CreateStmtContext *cxt, PartitionCmd *partcmd) +{ + Relation parent = cxt->rel; + Oid splitPartOid; + ListCell *listptr; + + if (parent->rd_rel->relkind != RELKIND_PARTITIONED_TABLE) + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("\"%s\" is not a partitioned table", RelationGetRelationName(parent)))); + + /* Transform partition bounds for all partitions in the list: */ + foreach(listptr, partcmd->partlist) + { + SinglePartitionSpec *sps = (SinglePartitionSpec *) lfirst(listptr); + + cxt->partbound = NULL; + transformPartitionCmd(cxt, sps->bound); + /* Assign transformed value of the partition bound. */ + sps->bound = cxt->partbound; + } + + splitPartOid = RangeVarGetRelid(partcmd->name, NoLock, false); + + checkPartition(parent, splitPartOid); + + /* Then we should check partitions with transformed bounds. */ + check_partitions_for_split(parent, splitPartOid, partcmd->name, partcmd->partlist, cxt->pstate); +} + + /* * transformPartitionCmdForMerge * Analyze the ALTER TABLE ... MERGE PARTITIONS command @@ -3615,7 +3654,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt, { PartitionCmd *partcmd = (PartitionCmd *) cmd->def; - transformPartitionCmd(&cxt, partcmd); + transformPartitionCmd(&cxt, partcmd->bound); /* assign transformed value of the partition bound */ partcmd->bound = cxt.partbound; } @@ -3623,6 +3662,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt, newcmds = lappend(newcmds, cmd); break; + case AT_SplitPartition: case AT_MergePartitions: { PartitionCmd *partcmd = (PartitionCmd *) cmd->def; @@ -3631,7 +3671,11 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt, ereport(ERROR, (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), errmsg("list of new partitions should contain at least two items"))); - transformPartitionCmdForMerge(&cxt, partcmd); + + if (cmd->subtype == AT_SplitPartition) + transformPartitionCmdForSplit(&cxt, partcmd); + else + transformPartitionCmdForMerge(&cxt, partcmd); newcmds = lappend(newcmds, cmd); break; } @@ -4026,13 +4070,13 @@ setSchemaName(const char *context_schema, char **stmt_schema_name) /* * transformPartitionCmd - * Analyze the ATTACH/DETACH PARTITION command + * Analyze the ATTACH/DETACH/SPLIT PARTITION command * - * In case of the ATTACH PARTITION command, cxt->partbound is set to the - * transformed value of cmd->bound. + * In case of the ATTACH/SPLIT PARTITION command, cxt->partbound is set to the + * transformed value of bound. */ static void -transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd) +transformPartitionCmd(CreateStmtContext *cxt, PartitionBoundSpec *bound) { Relation parentRel = cxt->rel; @@ -4041,9 +4085,9 @@ transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd) case RELKIND_PARTITIONED_TABLE: /* transform the partition bound, if any */ Assert(RelationGetPartitionKey(parentRel) != NULL); - if (cmd->bound != NULL) + if (bound != NULL) cxt->partbound = transformPartitionBound(cxt->pstate, parentRel, - cmd->bound); + bound); break; case RELKIND_PARTITIONED_INDEX: @@ -4051,7 +4095,7 @@ transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd) * A partitioned index cannot have a partition bound set. ALTER * INDEX prevents that with its grammar, but not ALTER TABLE. */ - if (cmd->bound != NULL) + if (bound != NULL) ereport(ERROR, (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), errmsg("\"%s\" is not a partitioned table", diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c index 6cb32d57ef..4eda59767c 100644 --- a/src/backend/partitioning/partbounds.c +++ b/src/backend/partitioning/partbounds.c @@ -5041,10 +5041,69 @@ check_two_partitions_bounds_range(Relation parent, } } +/* + * check_partitions_not_overlap_list + * + * (function for BY LIST partitioning) + * + * This is a helper function for check_partitions_for_split(). + * Checks that the values of the new partitions do not overlap. + * + * parent: partitioned table + * parts: array of SinglePartitionSpec structs with info about split partitions + * nparts: size of array "parts" + */ +static void +check_partitions_not_overlap_list(Relation parent, + SinglePartitionSpec **parts, + int nparts, + ParseState *pstate) +{ + PartitionKey key PG_USED_FOR_ASSERTS_ONLY = RelationGetPartitionKey(parent); + int overlap_location = -1; + int i, + j; + SinglePartitionSpec *sps1, + *sps2; + List *overlap; + + Assert(key->strategy == PARTITION_STRATEGY_LIST); + + for (i = 0; i < nparts; i++) + { + sps1 = parts[i]; + + for (j = i + 1; j < nparts; j++) + { + sps2 = parts[j]; + + /* + * Calculate intersection between values of two partitions. + */ + overlap = list_intersection(sps1->bound->listdatums, + sps2->bound->listdatums); + if (list_length(overlap) > 0) + { + Const *val = (Const *) lfirst(list_head(overlap)); + + overlap_location = val->location; + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("new partition \"%s\" would overlap with another new partition \"%s\"", + sps1->name->relname, sps2->name->relname), + parser_errposition(pstate, overlap_location))); + } + } + } +} + /* * get_partition_bound_spec * * Returns description of partition with Oid "partOid" and name "name". + * + * partOid: partition Oid + * name: partition name */ static PartitionBoundSpec * get_partition_bound_spec(Oid partOid, RangeVar *name) @@ -5077,6 +5136,636 @@ get_partition_bound_spec(Oid partOid, RangeVar *name) return boundspec; } +/* + * check_partition_bounds_for_split_range + * + * (function for BY RANGE partitioning) + * + * Checks that bounds of new partition "spec" are inside bounds of split + * partition (with Oid splitPartOid). If first=true (this means that "spec" is + * the first of new partitions) then lower bound of "spec" should be equal (or + * greater than or equal in case defaultPart=true) to lower bound of split + * partition. If last=true (this means that "spec" is the last of new + * partitions) then upper bound of "spec" should be equal (or less than or + * equal in case defaultPart=true) to upper bound of split partition. + * + * parent: partitioned table + * relname: name of the new partition + * spec: bounds specification of the new partition + * splitPartOid: split partition Oid + * splitPartName: split partition name + * first: true in case new partition "spec" is first of new partitions + * last: true in case new partition "spec" is last of new partitions + * defaultPart: true in case partitioned table has DEFAULT partition + * pstate: pointer to ParseState struct for determine error position + */ +static void +check_partition_bounds_for_split_range(Relation parent, + char *relname, + PartitionBoundSpec *spec, + Oid splitPartOid, + RangeVar *splitPartName, + bool first, + bool last, + bool defaultPart, + ParseState *pstate) +{ + PartitionKey key = RelationGetPartitionKey(parent); + PartitionRangeBound *lower, + *upper; + int cmpval; + + Assert(key->strategy == PARTITION_STRATEGY_RANGE); + Assert(spec->strategy == PARTITION_STRATEGY_RANGE); + + lower = make_one_partition_rbound(key, -1, spec->lowerdatums, true); + upper = make_one_partition_rbound(key, -1, spec->upperdatums, false); + + /* + * First check if the resulting range would be empty with specified lower + * and upper bounds. partition_rbound_cmp cannot return zero here, since + * the lower-bound flags are different. + */ + cmpval = partition_rbound_cmp(key->partnatts, + key->partsupfunc, + key->partcollation, + lower->datums, lower->kind, + true, upper); + Assert(cmpval != 0); + if (cmpval > 0) + { + /* Point to problematic key in the lower datums list. */ + PartitionRangeDatum *datum = list_nth(spec->lowerdatums, cmpval - 1); + + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("empty range bound specified for partition \"%s\"", + relname), + errdetail("Specified lower bound %s is greater than or equal to upper bound %s.", + get_range_partbound_string(spec->lowerdatums), + get_range_partbound_string(spec->upperdatums)), + parser_errposition(pstate, datum->location))); + } + + /* Need to check first and last partitions (from set of new partitions) */ + if (first || last) + { + PartitionBoundSpec *split_spec = get_partition_bound_spec(splitPartOid, splitPartName); + PartitionRangeDatum *datum; + + if (first) + { + PartitionRangeBound *split_lower; + + split_lower = make_one_partition_rbound(key, -1, split_spec->lowerdatums, true); + + cmpval = partition_rbound_cmp(key->partnatts, + key->partsupfunc, + key->partcollation, + lower->datums, lower->kind, + true, split_lower); + + /* + * Lower bound of "spec" should be equal (or greater than or equal + * in case defaultPart=true) to lower bound of split partition. + */ + if (!defaultPart) + { + if (cmpval != 0) + { + datum = list_nth(spec->lowerdatums, abs(cmpval) - 1); + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("lower bound of partition \"%s\" is not equal to lower bound of split partition", + relname), + parser_errposition(pstate, datum->location))); + } + } + else + { + if (cmpval < 0) + { + datum = list_nth(spec->lowerdatums, abs(cmpval) - 1); + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("lower bound of partition \"%s\" is less than lower bound of split partition", + relname), + parser_errposition(pstate, datum->location))); + } + } + } + else + { + PartitionRangeBound *split_upper; + + split_upper = make_one_partition_rbound(key, -1, split_spec->upperdatums, false); + + cmpval = partition_rbound_cmp(key->partnatts, + key->partsupfunc, + key->partcollation, + upper->datums, upper->kind, + false, split_upper); + + /* + * Upper bound of "spec" should be equal (or less than or equal in + * case defaultPart=true) to upper bound of split partition. + */ + if (!defaultPart) + { + if (cmpval != 0) + { + datum = list_nth(spec->upperdatums, abs(cmpval) - 1); + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("upper bound of partition \"%s\" is not equal to upper bound of split partition", + relname), + parser_errposition(pstate, datum->location))); + } + } + else + { + if (cmpval > 0) + { + datum = list_nth(spec->upperdatums, abs(cmpval) - 1); + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("upper bound of partition \"%s\" is greater than upper bound of split partition", + relname), + parser_errposition(pstate, datum->location))); + } + } + } + } +} + +/* + * check_partition_bounds_for_split_list + * + * (function for BY LIST partitioning) + * + * Checks that bounds of new partition are inside bounds of split partition + * (with Oid splitPartOid). + * + * parent: partitioned table + * relname: name of the new partition + * spec: bounds specification of the new partition + * splitPartOid: split partition Oid + * pstate: pointer to ParseState struct for determine error position + */ +static void +check_partition_bounds_for_split_list(Relation parent, char *relname, + PartitionBoundSpec *spec, + Oid splitPartOid, + ParseState *pstate) +{ + PartitionKey key = RelationGetPartitionKey(parent); + PartitionDesc partdesc = RelationGetPartitionDesc(parent, false); + PartitionBoundInfo boundinfo = partdesc->boundinfo; + int with = -1; + bool overlap = false; + int overlap_location = -1; + ListCell *cell; + + Assert(key->strategy == PARTITION_STRATEGY_LIST); + Assert(spec->strategy == PARTITION_STRATEGY_LIST); + Assert(boundinfo && boundinfo->strategy == PARTITION_STRATEGY_LIST); + + /* + * Search each value of new partition "spec" in existing partitions. All + * of them should be in split partition (with Oid splitPartOid). + */ + foreach(cell, spec->listdatums) + { + Const *val = lfirst_node(Const, cell); + + overlap_location = val->location; + if (!val->constisnull) + { + int offset; + bool equal; + + offset = partition_list_bsearch(&key->partsupfunc[0], + key->partcollation, + boundinfo, + val->constvalue, + &equal); + if (offset >= 0 && equal) + { + with = boundinfo->indexes[offset]; + if (partdesc->oids[with] != splitPartOid) + { + overlap = true; + break; + } + } + else + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("new partition \"%s\" cannot have this value because split partition does not have", + relname), + parser_errposition(pstate, overlap_location))); + } + else if (partition_bound_accepts_nulls(boundinfo)) + { + with = boundinfo->null_index; + if (partdesc->oids[with] != splitPartOid) + { + overlap = true; + break; + } + } + else + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("new partition \"%s\" cannot have NULL value because split partition does not have", + relname), + parser_errposition(pstate, overlap_location))); + } + + if (overlap) + { + Assert(with >= 0); + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("new partition \"%s\" would overlap with another (not split) partition \"%s\"", + relname, get_rel_name(partdesc->oids[with])), + parser_errposition(pstate, overlap_location))); + } +} + +/* + * find_value_in_new_partitions_list + * + * (function for BY LIST partitioning) + * + * Function returns true in case any of new partitions contains value "value". + * + * partsupfunc: information about comparison function associated with the partition key + * partcollation: partitioning collation + * parts: pointer to array with new partitions descriptions + * nparts: number of new partitions + * value: the value that we are looking for + * isnull: true if the value that we are looking for is NULL + */ +static bool +find_value_in_new_partitions_list(FmgrInfo *partsupfunc, + Oid *partcollation, + SinglePartitionSpec **parts, + int nparts, + Datum value, + bool isnull) +{ + ListCell *valptr; + int i; + + for (i = 0; i < nparts; i++) + { + SinglePartitionSpec *sps = parts[i]; + + foreach(valptr, sps->bound->listdatums) + { + Const *val = lfirst_node(Const, valptr); + + if (isnull && val->constisnull) + return true; + + if (!isnull && !val->constisnull) + { + if (DatumGetInt32(FunctionCall2Coll(&partsupfunc[0], + partcollation[0], + val->constvalue, + value)) == 0) + return true; + } + } + } + return false; +} + +/* + * check_parent_values_in_new_partitions + * + * (function for BY LIST partitioning) + * + * Checks that all values of split partition (with Oid partOid) contains in new + * partitions. + * + * parent: partitioned table + * partOid: split partition Oid + * parts: pointer to array with new partitions descriptions + * nparts: number of new partitions + * pstate: pointer to ParseState struct for determine error position + */ +static void +check_parent_values_in_new_partitions(Relation parent, + Oid partOid, + SinglePartitionSpec **parts, + int nparts, + ParseState *pstate) +{ + PartitionKey key = RelationGetPartitionKey(parent); + PartitionDesc partdesc = RelationGetPartitionDesc(parent, false); + PartitionBoundInfo boundinfo = partdesc->boundinfo; + int i; + bool found = true; + bool searchNull = false; + Datum datum = PointerGetDatum(NULL); + + Assert(key->strategy == PARTITION_STRATEGY_LIST); + + /* + * Special processing for NULL value. Search NULL value if the split + * partition (partOid) contains it. + */ + if (partition_bound_accepts_nulls(boundinfo) && + partdesc->oids[boundinfo->null_index] == partOid) + { + if (!find_value_in_new_partitions_list(&key->partsupfunc[0], + key->partcollation, parts, nparts, datum, true)) + { + found = false; + searchNull = true; + } + } + + /* + * Search all values of split partition with partOid in PartitionDesc of + * partitioned table. + */ + for (i = 0; i < boundinfo->ndatums; i++) + { + if (partdesc->oids[boundinfo->indexes[i]] == partOid) + { + /* We found value that split partition contains. */ + datum = boundinfo->datums[i][0]; + if (!find_value_in_new_partitions_list(&key->partsupfunc[0], + key->partcollation, parts, nparts, datum, false)) + { + found = false; + break; + } + } + } + + if (!found) + { + Const *notFoundVal; + + if (!searchNull) + + /* + * Make Const for getting string representation of not found + * value. + */ + notFoundVal = makeConst(key->parttypid[0], + key->parttypmod[0], + key->parttypcoll[0], + key->parttyplen[0], + datum, + false, /* isnull */ + key->parttypbyval[0]); + + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("new partitions do not have value %s but split partition does", + searchNull ? "NULL" : get_list_partvalue_string(notFoundVal)))); + } +} + +/* + * check_partitions_for_split + * + * Checks new partitions for SPLIT PARTITIONS command: + * 1. DEFAULT partition should be one. + * 2. New partitions should have different names + * (with existing partitions too). + * 3. Bounds of new partitions should not overlap with new and existing + * partitions. + * 4. In case split partition is DEFAULT partition, one of new partitions + * should be DEFAULT. + * 5. In case new partitions or existing partitions contains DEFAULT + * partition, new partitions can have any bounds inside split + * partition bound (can be spaces between partitions bounds). + * 6. In case partitioned table does not have DEFAULT partition, DEFAULT + * partition can be defined as one of new partition. + * 7. In case new partitions not contains DEFAULT partition and + * partitioned table does not have DEFAULT partition the following + * should be true: sum bounds of new partitions should be equal + * to bound of split partition. + * + * parent: partitioned table + * splitPartOid: split partition Oid + * splitPartName: split partition name + * list: list of new partitions + * pstate: pointer to ParseState struct for determine error position + */ +void +check_partitions_for_split(Relation parent, + Oid splitPartOid, + RangeVar *splitPartName, + List *partlist, + ParseState *pstate) +{ + PartitionKey key; + char strategy; + Oid defaultPartOid; + bool isSplitPartDefault; + bool existsDefaultPart; + ListCell *listptr; + int default_index = -1; + int i, + j; + SinglePartitionSpec **new_parts; + SinglePartitionSpec *spsPrev = NULL; + int nparts = 0; + + key = RelationGetPartitionKey(parent); + strategy = get_partition_strategy(key); + + switch (strategy) + { + case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_RANGE: + { + /* + * Make array new_parts with new partitions except DEFAULT + * partition. + */ + new_parts = (SinglePartitionSpec **) + palloc0(list_length(partlist) * sizeof(SinglePartitionSpec *)); + i = 0; + foreach(listptr, partlist) + { + SinglePartitionSpec *sps = + (SinglePartitionSpec *) lfirst(listptr); + + if (sps->bound->is_default) + { + if (default_index >= 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("DEFAULT partition should be one")), + parser_errposition(pstate, sps->name->location)); + default_index = i; + } + else + { + new_parts[nparts++] = sps; + } + i++; + } + } + break; + + case PARTITION_STRATEGY_HASH: + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("partition of hash-partitioned table cannot be split"))); + break; + + default: + elog(ERROR, "unexpected partition strategy: %d", + (int) key->strategy); + break; + } + + if (strategy == PARTITION_STRATEGY_RANGE) + { + PartitionRangeBound **lower_bounds; + SinglePartitionSpec **tmp_new_parts; + + /* + * For simplify check for ranges of new partitions need to sort all + * partitions in ascending order of them bounds (we compare upper + * bound only). + */ + lower_bounds = (PartitionRangeBound **) + palloc0(nparts * sizeof(PartitionRangeBound *)); + + /* Create array of lower bounds. */ + for (i = 0; i < nparts; i++) + { + lower_bounds[i] = make_one_partition_rbound(key, i, + new_parts[i]->bound->lowerdatums, true); + } + + /* Sort array of lower bounds. */ + qsort_arg(lower_bounds, nparts, sizeof(PartitionRangeBound *), + qsort_partition_rbound_cmp, (void *) key); + + /* Reorder array of partitions. */ + tmp_new_parts = new_parts; + new_parts = (SinglePartitionSpec **) + palloc0(nparts * sizeof(SinglePartitionSpec *)); + for (i = 0; i < nparts; i++) + new_parts[i] = tmp_new_parts[lower_bounds[i]->index]; + + pfree(tmp_new_parts); + pfree(lower_bounds); + } + + defaultPartOid = + get_default_oid_from_partdesc(RelationGetPartitionDesc(parent, true)); + + /* isSplitPartDefault flag: is split partition a DEFAULT partition? */ + isSplitPartDefault = (defaultPartOid == splitPartOid); + + if (isSplitPartDefault && default_index < 0) + { + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("one partition in the list should be DEFAULT because split partition is DEFAULT")), + parser_errposition(pstate, ((SinglePartitionSpec *) linitial(partlist))->name->location)); + } + else if (!isSplitPartDefault && (default_index >= 0) && OidIsValid(defaultPartOid)) + { + SinglePartitionSpec *spsDef = + (SinglePartitionSpec *) list_nth(partlist, default_index); + + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("new partition cannot be DEFAULT because DEFAULT partition already exists")), + parser_errposition(pstate, spsDef->name->location)); + } + + /* Indicator that partitioned table has (or will have) DEFAULT partition */ + existsDefaultPart = OidIsValid(defaultPartOid) || (default_index >= 0); + + for (i = 0; i < nparts; i++) + { + SinglePartitionSpec *sps = new_parts[i]; + + if (isSplitPartDefault) + { + /* + * In case split partition is DEFAULT partition we can use any + * free ranges - as when creating a new partition. + */ + check_new_partition_bound(sps->name->relname, parent, sps->bound, + pstate); + } + else + { + /* + * Checks that bound of current partition is inside bound of split + * partition. For range partitioning: checks that upper bound of + * previous partition is equal to lower bound of current + * partition. For list partitioning: checks that split partition + * contains all values of current partition. + */ + if (strategy == PARTITION_STRATEGY_RANGE) + { + bool first = (i == 0); + bool last = (i == (nparts - 1)); + + check_partition_bounds_for_split_range(parent, sps->name->relname, sps->bound, + splitPartOid, splitPartName, + first, last, + existsDefaultPart, pstate); + } + else + check_partition_bounds_for_split_list(parent, sps->name->relname, + sps->bound, splitPartOid, pstate); + } + + /* Ranges of new partitions should not overlap. */ + if (strategy == PARTITION_STRATEGY_RANGE && spsPrev) + check_two_partitions_bounds_range(parent, spsPrev->name, spsPrev->bound, + sps->name, sps->bound, existsDefaultPart, pstate); + + spsPrev = sps; + + /* Check: new partitions should have different names. */ + for (j = i + 1; j < nparts; j++) + { + SinglePartitionSpec *sps2 = new_parts[j]; + + if (equal(sps->name, sps2->name)) + ereport(ERROR, + (errcode(ERRCODE_DUPLICATE_TABLE), + errmsg("name \"%s\" is already used", sps2->name->relname)), + parser_errposition(pstate, sps2->name->location)); + } + } + + if (strategy == PARTITION_STRATEGY_LIST) + { + /* Values of new partitions should not overlap. */ + check_partitions_not_overlap_list(parent, new_parts, nparts, + pstate); + + /* + * Need to check that all values of split partition contains in new + * partitions. Skip this check if DEFAULT partition exists. + */ + if (!existsDefaultPart) + check_parent_values_in_new_partitions(parent, splitPartOid, + new_parts, nparts, pstate); + } + + pfree(new_parts); +} + /* * calculate_partition_bound_for_merge * diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c index 00eda1b34c..4039ee0df7 100644 --- a/src/backend/utils/adt/ruleutils.c +++ b/src/backend/utils/adt/ruleutils.c @@ -13296,3 +13296,21 @@ get_range_partbound_string(List *bound_datums) return buf->data; } + +/* + * get_list_partvalue_string + * A C string representation of one list partition value + */ +char * +get_list_partvalue_string(Const *val) +{ + deparse_context context; + StringInfo buf = makeStringInfo(); + + memset(&context, 0, sizeof(deparse_context)); + context.buf = buf; + + get_const_expr(val, &context, -1); + + return buf->data; +} diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c index c3ea87fcb9..0d25981253 100644 --- a/src/bin/psql/tab-complete.c +++ b/src/bin/psql/tab-complete.c @@ -2353,7 +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 (", + "SPLIT PARTITION", "MERGE PARTITIONS (", "OF", "NOT OF"); /* ALTER TABLE xxx ADD */ else if (Matches("ALTER", "TABLE", MatchAny, "ADD")) @@ -2610,10 +2610,10 @@ psql_completion(const char *text, int start, int end) COMPLETE_WITH("FROM (", "IN (", "WITH ("); /* - * If we have ALTER TABLE DETACH PARTITION, provide a list of + * If we have ALTER TABLE DETACH|SPLIT PARTITION, provide a list of * partitions of . */ - else if (Matches("ALTER", "TABLE", MatchAny, "DETACH", "PARTITION")) + else if (Matches("ALTER", "TABLE", MatchAny, "DETACH|SPLIT", "PARTITION")) { set_completion_reference(prev3_wd); COMPLETE_WITH_SCHEMA_QUERY(Query_for_partition_of_table); @@ -2621,6 +2621,10 @@ psql_completion(const char *text, int start, int end) else if (Matches("ALTER", "TABLE", MatchAny, "DETACH", "PARTITION", MatchAny)) COMPLETE_WITH("CONCURRENTLY", "FINALIZE"); + /* ALTER TABLE SPLIT PARTITION */ + else if (Matches("ALTER", "TABLE", MatchAny, "SPLIT", "PARTITION", MatchAny)) + COMPLETE_WITH("INTO ( PARTITION"); + /* ALTER TABLE MERGE PARTITIONS ( */ else if (Matches("ALTER", "TABLE", MatchAny, "MERGE", "PARTITIONS", "(")) { diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h index b6e7957ebb..85a62b538e 100644 --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -2413,6 +2413,7 @@ typedef enum AlterTableType AT_AttachPartition, /* ATTACH PARTITION */ AT_DetachPartition, /* DETACH PARTITION */ AT_DetachPartitionFinalize, /* DETACH PARTITION FINALIZE */ + AT_SplitPartition, /* SPLIT PARTITION */ AT_MergePartitions, /* MERGE PARTITIONS */ AT_AddIdentity, /* ADD IDENTITY */ AT_SetIdentity, /* SET identity column options */ diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h index 6b256fcece..bb191b1f46 100644 --- a/src/include/parser/kwlist.h +++ b/src/include/parser/kwlist.h @@ -418,6 +418,7 @@ PG_KEYWORD("smallint", SMALLINT, COL_NAME_KEYWORD, BARE_LABEL) PG_KEYWORD("snapshot", SNAPSHOT, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("some", SOME, RESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("source", SOURCE, UNRESERVED_KEYWORD, BARE_LABEL) +PG_KEYWORD("split", SPLIT, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("sql", SQL_P, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("stable", STABLE, UNRESERVED_KEYWORD, BARE_LABEL) PG_KEYWORD("standalone", STANDALONE_P, UNRESERVED_KEYWORD, BARE_LABEL) diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h index 0329d7bd84..9d2a26705f 100644 --- a/src/include/partitioning/partbounds.h +++ b/src/include/partitioning/partbounds.h @@ -143,6 +143,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc, extern int partition_hash_bsearch(PartitionBoundInfo boundinfo, int modulus, int remainder); +extern void check_partitions_for_split(Relation parent, + Oid splitPartOid, + RangeVar *splitPartName, + List *partlist, + ParseState *pstate); extern void calculate_partition_bound_for_merge(Relation parent, List *partNames, List *partOids, diff --git a/src/include/utils/ruleutils.h b/src/include/utils/ruleutils.h index 161fb5ef02..6d86080622 100644 --- a/src/include/utils/ruleutils.h +++ b/src/include/utils/ruleutils.h @@ -49,4 +49,6 @@ extern char *get_range_partbound_string(List *bound_datums); extern char *pg_get_statisticsobjdef_string(Oid statextid); +extern char *get_list_partvalue_string(Const *val); + #endif /* RULEUTILS_H */ diff --git a/src/test/isolation/expected/partition-split.out b/src/test/isolation/expected/partition-split.out new file mode 100644 index 0000000000..5d9e8b0925 --- /dev/null +++ b/src/test/isolation/expected/partition-split.out @@ -0,0 +1,190 @@ +Parsed test spec with 2 sessions + +starting permutation: s1b s1splt s2b s2i s1c s2c s2s +step s1b: BEGIN; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2b: BEGIN; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1b s1splt s2brr s2i s1c s2c s2s +step s1b: BEGIN; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2brr: BEGIN ISOLATION LEVEL REPEATABLE READ; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1b s1splt s2bs s2i s1c s2c s2s +step s1b: BEGIN; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2bs: BEGIN ISOLATION LEVEL SERIALIZABLE; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1brr s1splt s2b s2i s1c s2c s2s +step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2b: BEGIN; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1brr s1splt s2brr s2i s1c s2c s2s +step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2brr: BEGIN ISOLATION LEVEL REPEATABLE READ; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1brr s1splt s2bs s2i s1c s2c s2s +step s1brr: BEGIN ISOLATION LEVEL REPEATABLE READ; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2bs: BEGIN ISOLATION LEVEL SERIALIZABLE; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1bs s1splt s2b s2i s1c s2c s2s +step s1bs: BEGIN ISOLATION LEVEL SERIALIZABLE; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2b: BEGIN; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1bs s1splt s2brr s2i s1c s2c s2s +step s1bs: BEGIN ISOLATION LEVEL SERIALIZABLE; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2brr: BEGIN ISOLATION LEVEL REPEATABLE READ; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + + +starting permutation: s1bs s1splt s2bs s2i s1c s2c s2s +step s1bs: BEGIN ISOLATION LEVEL SERIALIZABLE; +step s1splt: ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (20)); +step s2bs: BEGIN ISOLATION LEVEL SERIALIZABLE; +step s2i: INSERT INTO tpart VALUES (1, 'text01'); +step s1c: COMMIT; +step s2i: <... completed> +step s2c: COMMIT; +step s2s: SELECT * FROM tpart; + i|t +--+------ + 5|text05 + 1|text01 +15|text15 +25|text25 +35|text35 +(5 rows) + diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule index c1499338d6..6da98cffac 100644 --- a/src/test/isolation/isolation_schedule +++ b/src/test/isolation/isolation_schedule @@ -107,6 +107,7 @@ test: partition-key-update-2 test: partition-key-update-3 test: partition-key-update-4 test: partition-merge +test: partition-split test: plpgsql-toast test: cluster-conflict test: cluster-conflict-partition diff --git a/src/test/isolation/specs/partition-split.spec b/src/test/isolation/specs/partition-split.spec new file mode 100644 index 0000000000..087239a4a1 --- /dev/null +++ b/src/test/isolation/specs/partition-split.spec @@ -0,0 +1,54 @@ +# Verify that SPLIT 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 s1splt { ALTER TABLE tpart SPLIT PARTITION tpart_10_20 INTO + (PARTITION tpart_10_15 FOR VALUES FROM (10) TO (15), + PARTITION tpart_15_20 FOR VALUES FROM (15) TO (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 s2c { COMMIT; } +step s2s { SELECT * FROM tpart; } + + +# s1 starts SPLIT PARTITION then s2 trying to insert row and +# waits until s1 finished SPLIT operation. + +permutation s1b s1splt s2b s2i s1c s2c s2s +permutation s1b s1splt s2brr s2i s1c s2c s2s +permutation s1b s1splt s2bs s2i s1c s2c s2s + +permutation s1brr s1splt s2b s2i s1c s2c s2s +permutation s1brr s1splt s2brr s2i s1c s2c s2s +permutation s1brr s1splt s2bs s2i s1c s2c s2s + +permutation s1bs s1splt s2b s2i s1c s2c s2s +permutation s1bs s1splt s2brr s2i s1c s2c s2s +permutation s1bs s1splt s2bs s2i 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 cc341afdf4..67ff2b6367 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_SplitPartition: + strtype = "SPLIT PARTITION"; + break; case AT_MergePartitions: strtype = "MERGE PARTITIONS"; break; diff --git a/src/test/regress/expected/partition_split.out b/src/test/regress/expected/partition_split.out new file mode 100644 index 0000000000..dc9a5130cc --- /dev/null +++ b/src/test/regress/expected/partition_split.out @@ -0,0 +1,1589 @@ +-- +-- PARTITION_SPLIT +-- Tests for "ALTER TABLE ... SPLIT PARTITION ..." command +-- +CREATE SCHEMA partition_split_schema; +CREATE SCHEMA partition_split_schema2; +SET search_path = partition_split_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_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01'); +CREATE TABLE sales_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; +-- ERROR: relation "sales_xxx" does not exist +ALTER TABLE sales_range SPLIT PARTITION sales_xxx INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: relation "sales_xxx" does not exist +-- ERROR: relation "sales_jan2022" already exists +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_jan2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: relation "sales_jan2022" already exists +-- ERROR: invalid bound specification for a range partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_jan2022 FOR VALUES IN ('2022-05-01', '2022-06-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: invalid bound specification for a range partition +LINE 2: (PARTITION sales_jan2022 FOR VALUES IN ('2022-05-01', '202... + ^ +-- ERROR: empty range bound specified for partition "sales_mar2022" +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-02-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: empty range bound specified for partition "sales_mar2022" +LINE 3: PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO... + ^ +DETAIL: Specified lower bound ('03-01-2022') is greater than or equal to upper bound ('02-01-2022'). +--ERROR: list of split partitions should contain at least two items +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-10-01')); +ERROR: list of new partitions should contain at least two items +-- ERROR: lower bound of partition "sales_feb2022" is less than lower bound of split partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-01-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: lower bound of partition "sales_feb2022" is less than lower bound of split partition +LINE 2: (PARTITION sales_feb2022 FOR VALUES FROM ('2022-01-01') TO... + ^ +-- ERROR: name "sales_feb_mar_apr2022" is already used +-- (We can create partition with the same name as split partition, but can't create two partitions with the same name) +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb_mar_apr2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_feb_mar_apr2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: name "sales_feb_mar_apr2022" is already used +LINE 3: PARTITION sales_feb_mar_apr2022 FOR VALUES FROM ('2022-03... + ^ +-- ERROR: name "sales_feb2022" is already used +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_feb2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: name "sales_feb2022" is already used +LINE 3: PARTITION sales_feb2022 FOR VALUES FROM ('2022-03-01') TO... + ^ +-- ERROR: "sales_feb_mar_apr2022" is not a partitioned table +ALTER TABLE sales_feb_mar_apr2022 SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_jan2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_feb2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: "sales_feb_mar_apr2022" is not a partitioned table +-- ERROR: upper bound of partition "sales_apr2022" is greater than upper bound of split partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-06-01')); +ERROR: upper bound of partition "sales_apr2022" is greater than upper bound of split partition +LINE 4: ... sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-06-0... + ^ +-- ERROR: lower bound of partition "sales_mar2022" conflicts with upper bound of previous partition "sales_feb2022" +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-02-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: lower bound of partition "sales_mar2022" conflicts with upper bound of previous partition "sales_feb2022" +LINE 3: PARTITION sales_mar2022 FOR VALUES FROM ('2022-02-01') TO... + ^ +-- Tests for spaces between partitions, them should be executed without DEFAULT partition +ALTER TABLE sales_range DETACH PARTITION sales_others; +-- ERROR: lower bound of partition "sales_feb2022" is not equal to lower bound of split partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-02') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +ERROR: lower bound of partition "sales_feb2022" is not equal to lower bound of split partition +LINE 2: (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-02') TO... + ^ +-- Check the source partition not in the search path +SET search_path = partition_split_schema2, public; +ALTER TABLE partition_split_schema.sales_range +SPLIT PARTITION partition_split_schema.sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +SET search_path = partition_split_schema, public; +\d+ sales_range + Partitioned table "partition_split_schema.sales_range" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +------------------+-----------------------+-----------+----------+---------+----------+--------------+------------- + salesperson_id | integer | | | | plain | | + salesperson_name | character varying(30) | | | | extended | | + sales_amount | integer | | | | plain | | + sales_date | date | | | | plain | | +Partition key: RANGE (sales_date) +Partitions: partition_split_schema2.sales_apr2022 FOR VALUES FROM ('04-01-2022') TO ('05-01-2022'), + partition_split_schema2.sales_feb2022 FOR VALUES FROM ('02-01-2022') TO ('03-01-2022'), + partition_split_schema2.sales_mar2022 FOR VALUES FROM ('03-01-2022') TO ('04-01-2022'), + sales_jan2022 FOR VALUES FROM ('01-01-2022') TO ('02-01-2022') + +DROP TABLE sales_range; +DROP TABLE sales_others; +-- +-- Add rows into partitioned table then split 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_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; +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'); +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +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 sales_feb2022; + 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 +(3 rows) + +SELECT * FROM sales_mar2022; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 7 | Li | 175 | 03-08-2022 + 9 | Muller | 250 | 03-11-2022 + 12 | Plato | 350 | 03-19-2022 +(3 rows) + +SELECT * FROM sales_apr2022; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 5 | Deev | 250 | 04-07-2022 + 11 | Trump | 380 | 04-06-2022 +(4 rows) + +SELECT * FROM sales_others; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 14 | Smith | 510 | 05-04-2022 +(1 row) + +DROP TABLE sales_range CASCADE; +-- +-- Add split partition, then add rows into partitioned table +-- +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_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; +-- Split partition, also check schema qualification of new partitions +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION partition_split_schema.sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION partition_split_schema2.sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +\d+ sales_range + Partitioned table "partition_split_schema.sales_range" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +------------------+-----------------------+-----------+----------+---------+----------+--------------+------------- + salesperson_id | integer | | | | plain | | + salesperson_name | character varying(30) | | | | extended | | + sales_amount | integer | | | | plain | | + sales_date | date | | | | plain | | +Partition key: RANGE (sales_date) +Partitions: partition_split_schema2.sales_mar2022 FOR VALUES FROM ('03-01-2022') TO ('04-01-2022'), + sales_apr2022 FOR VALUES FROM ('04-01-2022') TO ('05-01-2022'), + sales_feb2022 FOR VALUES FROM ('02-01-2022') TO ('03-01-2022'), + sales_jan2022 FOR VALUES FROM ('01-01-2022') TO ('02-01-2022'), + sales_others DEFAULT + +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 + 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 sales_feb2022; + 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 +(3 rows) + +SELECT * FROM partition_split_schema2.sales_mar2022; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 7 | Li | 175 | 03-08-2022 + 9 | Muller | 250 | 03-11-2022 + 12 | Plato | 350 | 03-19-2022 +(3 rows) + +SELECT * FROM sales_apr2022; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 5 | Deev | 250 | 04-07-2022 + 11 | Trump | 380 | 04-06-2022 +(4 rows) + +SELECT * FROM sales_others; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 14 | Smith | 510 | 05-04-2022 +(1 row) + +DROP TABLE sales_range CASCADE; +-- +-- 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_jan_feb2022 PARTITION OF sales_date FOR VALUES FROM (2022, 1, 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 + Manager1 | 2022 | 2 | 4 | 2022.02.04 | Sales department + Manager2 | 2022 | 1 | 2 | 2022.01.02 | 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_jan_feb2022; + salesperson_name | sales_year | sales_month | sales_day | sales_date | sales_department +------------------+------------+-------------+-----------+------------+------------------ + Manager3 | 2022 | 1 | 1 | 2022.01.01 | Sales department + Manager1 | 2022 | 2 | 4 | 2022.02.04 | Sales department + Manager2 | 2022 | 1 | 2 | 2022.01.02 | Sales department + Manager3 | 2022 | 2 | 1 | 2022.02.01 | Sales department +(4 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 SPLIT PARTITION sales_jan_feb2022 INTO + (PARTITION sales_jan2022 FOR VALUES FROM (2022, 1, 1) TO (2022, 2, 1), + PARTITION sales_feb2022 FOR VALUES FROM (2022, 2, 1) TO (2022, 3, 1)); +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 | 1 | 10 | 2022.01.10 | Sales department + Manager1 | 2022 | 2 | 4 | 2022.02.04 | Sales department + Manager3 | 2022 | 2 | 1 | 2022.02.01 | 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_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 + Manager1 | 2022 | 1 | 10 | 2022.01.10 | Sales department +(3 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 + Manager2 | 2022 | 2 | 10 | 2022.02.10 | Sales department +(3 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) + +--ERROR: relation "sales_jan_feb2022" does not exist +SELECT * FROM sales_jan_feb2022; +ERROR: relation "sales_jan_feb2022" does not exist +LINE 1: SELECT * FROM sales_jan_feb2022; + ^ +DROP TABLE sales_date CASCADE; +-- +-- Test: split DEFAULT partition; use an index on partition key; check index after split +-- +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_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_others; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 2 | Smirnoff | 500 | 02-10-2022 + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 5 | Deev | 250 | 04-07-2022 + 6 | Poirot | 150 | 02-11-2022 + 7 | Li | 175 | 03-08-2022 + 8 | Ericsson | 185 | 02-23-2022 + 9 | Muller | 250 | 03-11-2022 + 11 | Trump | 380 | 04-06-2022 + 12 | Plato | 350 | 03-19-2022 + 14 | Smith | 510 | 05-04-2022 +(11 rows) + +SELECT * FROM pg_indexes WHERE tablename = 'sales_others' and schemaname = 'partition_split_schema' ORDER BY indexname; + schemaname | tablename | indexname | tablespace | indexdef +------------------------+--------------+-----------------------------+------------+---------------------------------------------------------------------------------------------------------- + partition_split_schema | sales_others | sales_others_sales_date_idx | | CREATE INDEX sales_others_sales_date_idx ON partition_split_schema.sales_others USING btree (sales_date) +(1 row) + +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01'), + PARTITION sales_others DEFAULT); +-- Use indexscan for testing indexes +SET enable_indexscan = ON; +SET enable_seqscan = OFF; +SELECT * FROM sales_feb2022 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 +(3 rows) + +SELECT * FROM sales_mar2022 where sales_date > '2022-01-01'; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 7 | Li | 175 | 03-08-2022 + 9 | Muller | 250 | 03-11-2022 + 12 | Plato | 350 | 03-19-2022 +(3 rows) + +SELECT * FROM sales_apr2022 where sales_date > '2022-01-01'; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 11 | Trump | 380 | 04-06-2022 + 5 | Deev | 250 | 04-07-2022 + 4 | Ivanov | 750 | 04-13-2022 + 3 | Ford | 2000 | 04-30-2022 +(4 rows) + +SELECT * FROM sales_others where sales_date > '2022-01-01'; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 14 | Smith | 510 | 05-04-2022 +(1 row) + +SET enable_indexscan = ON; +SET enable_seqscan = ON; +SELECT * FROM pg_indexes WHERE tablename = 'sales_feb2022' and schemaname = 'partition_split_schema' ORDER BY indexname; + schemaname | tablename | indexname | tablespace | indexdef +------------------------+---------------+------------------------------+------------+------------------------------------------------------------------------------------------------------------ + partition_split_schema | sales_feb2022 | sales_feb2022_sales_date_idx | | CREATE INDEX sales_feb2022_sales_date_idx ON partition_split_schema.sales_feb2022 USING btree (sales_date) +(1 row) + +SELECT * FROM pg_indexes WHERE tablename = 'sales_mar2022' and schemaname = 'partition_split_schema' ORDER BY indexname; + schemaname | tablename | indexname | tablespace | indexdef +------------------------+---------------+------------------------------+------------+------------------------------------------------------------------------------------------------------------ + partition_split_schema | sales_mar2022 | sales_mar2022_sales_date_idx | | CREATE INDEX sales_mar2022_sales_date_idx ON partition_split_schema.sales_mar2022 USING btree (sales_date) +(1 row) + +SELECT * FROM pg_indexes WHERE tablename = 'sales_apr2022' and schemaname = 'partition_split_schema' ORDER BY indexname; + schemaname | tablename | indexname | tablespace | indexdef +------------------------+---------------+------------------------------+------------+------------------------------------------------------------------------------------------------------------ + partition_split_schema | sales_apr2022 | sales_apr2022_sales_date_idx | | CREATE INDEX sales_apr2022_sales_date_idx ON partition_split_schema.sales_apr2022 USING btree (sales_date) +(1 row) + +SELECT * FROM pg_indexes WHERE tablename = 'sales_others' and schemaname = 'partition_split_schema' ORDER BY indexname; + schemaname | tablename | indexname | tablespace | indexdef +------------------------+--------------+------------------------------+------------+----------------------------------------------------------------------------------------------------------- + partition_split_schema | sales_others | sales_others_sales_date_idx1 | | CREATE INDEX sales_others_sales_date_idx1 ON partition_split_schema.sales_others USING btree (sales_date) +(1 row) + +DROP TABLE sales_range CASCADE; +-- +-- Test: some cases for splitting DEFAULT partition (different bounds) +-- +CREATE TABLE sales_range (salesperson_id INT, salesperson_name VARCHAR(30), sales_amount INT, sales_date INT) PARTITION BY RANGE (sales_date); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; +-- sales_error intersects with sales_dec2022 (lower bound) +-- ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20211230) TO (20220201), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); +ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +LINE 3: PARTITION sales_error FOR VALUES FROM (20211230) TO (2022... + ^ +-- sales_error intersects with sales_feb2022 (upper bound) +-- ERROR: lower bound of partition "sales_feb2022" conflicts with upper bound of previous partition "sales_error" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20220101) TO (20220202), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); +ERROR: lower bound of partition "sales_feb2022" conflicts with upper bound of previous partition "sales_error" +LINE 4: PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20... + ^ +-- sales_error intersects with sales_dec2022 (inside bound) +-- ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20211210) TO (20211220), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); +ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +LINE 3: PARTITION sales_error FOR VALUES FROM (20211210) TO (2021... + ^ +-- sales_error intersects with sales_dec2022 (exactly the same bounds) +-- ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); +ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +LINE 3: PARTITION sales_error FOR VALUES FROM (20211201) TO (2022... + ^ +-- ERROR: all partitions in the list should be DEFAULT because split partition is DEFAULT +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_jan2022 FOR VALUES FROM (20220101) TO (20220201), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301)); +ERROR: one partition in the list should be DEFAULT because split partition is DEFAULT +LINE 2: (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20... + ^ +-- no error: bounds of sales_noerror are between sales_dec2022 and sales_feb2022 +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_noerror FOR VALUES FROM (20220110) TO (20220120), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); +DROP TABLE sales_range; +CREATE TABLE sales_range (salesperson_id INT, salesperson_name VARCHAR(30), sales_amount INT, sales_date INT) PARTITION BY RANGE (sales_date); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; +-- no error: bounds of sales_noerror are equal to lower and upper bounds of sales_dec2022 and sales_feb2022 +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_noerror FOR VALUES FROM (20210101) TO (20210201), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); +DROP TABLE sales_range; +-- +-- Test: split partition with CHECK and FOREIGN KEY CONSTRAINTs on partitioned table +-- +CREATE TABLE salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)); +INSERT INTO salespeople VALUES (1, 'Poirot'); +CREATE TABLE sales_range ( +salesperson_id INT REFERENCES salespeople(salesperson_id), +sales_amount INT CHECK (sales_amount > 1), +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_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_feb_mar_apr2022'::regclass::oid ORDER BY conname; + pg_get_constraintdef | conname | conkey +---------------------------------------------------------------------+---------------------------------+-------- + CHECK ((sales_amount > 1)) | sales_range_sales_amount_check | {2} + FOREIGN KEY (salesperson_id) REFERENCES salespeople(salesperson_id) | sales_range_salesperson_id_fkey | {1} +(2 rows) + +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +-- We should see the same CONSTRAINTs as on sales_feb_mar_apr2022 partition +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_feb2022'::regclass::oid ORDER BY conname;; + pg_get_constraintdef | conname | conkey +---------------------------------------------------------------------+---------------------------------+-------- + CHECK ((sales_amount > 1)) | sales_range_sales_amount_check | {2} + FOREIGN KEY (salesperson_id) REFERENCES salespeople(salesperson_id) | sales_range_salesperson_id_fkey | {1} +(2 rows) + +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_mar2022'::regclass::oid ORDER BY conname;; + pg_get_constraintdef | conname | conkey +---------------------------------------------------------------------+---------------------------------+-------- + CHECK ((sales_amount > 1)) | sales_range_sales_amount_check | {2} + FOREIGN KEY (salesperson_id) REFERENCES salespeople(salesperson_id) | sales_range_salesperson_id_fkey | {1} +(2 rows) + +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_apr2022'::regclass::oid ORDER BY conname;; + pg_get_constraintdef | conname | conkey +---------------------------------------------------------------------+---------------------------------+-------- + CHECK ((sales_amount > 1)) | sales_range_sales_amount_check | {2} + FOREIGN KEY (salesperson_id) REFERENCES salespeople(salesperson_id) | sales_range_salesperson_id_fkey | {1} +(2 rows) + +-- ERROR: new row for relation "sales_mar2022" violates check constraint "sales_range_sales_amount_check" +INSERT INTO sales_range VALUES (1, 0, '2022-03-11'); +ERROR: new row for relation "sales_mar2022" violates check constraint "sales_range_sales_amount_check" +DETAIL: Failing row contains (1, 0, 03-11-2022). +-- ERROR: insert or update on table "sales_mar2022" violates foreign key constraint "sales_range_salesperson_id_fkey" +INSERT INTO sales_range VALUES (-1, 10, '2022-03-11'); +ERROR: insert or update on table "sales_mar2022" violates foreign key constraint "sales_range_salesperson_id_fkey" +DETAIL: Key (salesperson_id)=(-1) is not present in table "salespeople". +-- ok +INSERT INTO sales_range VALUES (1, 10, '2022-03-11'); +DROP TABLE sales_range CASCADE; +DROP TABLE salespeople CASCADE; +-- +-- Test: split partition on partitioned table in case of existing FOREIGN KEY reference from another table +-- +CREATE TABLE salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); +CREATE TABLE sales (salesperson_id INT REFERENCES salespeople(salesperson_id), sales_amount INT, sales_date DATE); +CREATE TABLE salespeople01_10 PARTITION OF salespeople FOR VALUES FROM (1) TO (10); +CREATE TABLE salespeople10_40 PARTITION OF salespeople FOR VALUES FROM (10) TO (40); +INSERT INTO salespeople VALUES (1, 'Poirot'); +INSERT INTO salespeople VALUES (10, 'May'); +INSERT INTO salespeople VALUES (19, 'Ivanov'); +INSERT INTO salespeople VALUES (20, 'Smirnoff'); +INSERT INTO salespeople VALUES (30, 'Ford'); +INSERT INTO sales VALUES (1, 100, '2022-03-01'); +INSERT INTO sales VALUES (1, 110, '2022-03-02'); +INSERT INTO sales VALUES (10, 150, '2022-03-01'); +INSERT INTO sales VALUES (10, 90, '2022-03-03'); +INSERT INTO sales VALUES (19, 200, '2022-03-04'); +INSERT INTO sales VALUES (20, 50, '2022-03-12'); +INSERT INTO sales VALUES (20, 170, '2022-03-02'); +INSERT INTO sales VALUES (30, 30, '2022-03-04'); +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) + +ALTER TABLE salespeople SPLIT PARTITION salespeople10_40 INTO + (PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20), + PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30), + PARTITION salespeople30_40 FOR VALUES FROM (30) TO (40)); +SELECT * FROM salespeople01_10; + salesperson_id | salesperson_name +----------------+------------------ + 1 | Poirot +(1 row) + +SELECT * FROM salespeople10_20; + salesperson_id | salesperson_name +----------------+------------------ + 10 | May + 19 | Ivanov +(2 rows) + +SELECT * FROM salespeople20_30; + salesperson_id | salesperson_name +----------------+------------------ + 20 | Smirnoff +(1 row) + +SELECT * FROM salespeople30_40; + salesperson_id | salesperson_name +----------------+------------------ + 30 | Ford +(1 row) + +-- ERROR: insert or update on table "sales" violates foreign key constraint "sales_salesperson_id_fkey" +INSERT INTO sales VALUES (40, 50, '2022-03-04'); +ERROR: insert or update on table "sales" violates foreign key constraint "sales_salesperson_id_fkey" +DETAIL: Key (salesperson_id)=(40) is not present in table "salespeople". +-- ok +INSERT INTO sales VALUES (30, 50, '2022-03-04'); +DROP TABLE sales CASCADE; +DROP TABLE salespeople CASCADE; +-- +-- Test: split partition 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_40 PARTITION OF salespeople FOR VALUES FROM (10) 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_40 VALUES (19, 'Ivanov'); +NOTICE: trigger(salespeople) called: action = INSERT, when = AFTER, level = ROW +ALTER TABLE salespeople SPLIT PARTITION salespeople10_40 INTO + (PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20), + PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30), + PARTITION salespeople30_40 FOR VALUES FROM (30) TO (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 salespeople30_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_20; + salesperson_id | salesperson_name +----------------+------------------ + 10 | May + 19 | Ivanov +(2 rows) + +SELECT * FROM salespeople20_30; + salesperson_id | salesperson_name +----------------+------------------ + 20 | Smirnoff +(1 row) + +SELECT * FROM salespeople30_40; + salesperson_id | salesperson_name +----------------+------------------ + 30 | Ford +(1 row) + +DROP TABLE salespeople CASCADE; +DROP FUNCTION after_insert_row_trigger(); +-- +-- Test: split partition witch identity column +-- If split partition column is identity column, columns of new partitions are identity columns too. +-- +CREATE TABLE salespeople(salesperson_id INT GENERATED ALWAYS AS IDENTITY PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); +CREATE TABLE salespeople1_2 PARTITION OF salespeople FOR VALUES FROM (1) TO (2); +-- Create new partition with identity column: +CREATE TABLE salespeople2_5(salesperson_id INT NOT NULL, salesperson_name VARCHAR(30)); +ALTER TABLE salespeople ATTACH PARTITION salespeople2_5 FOR VALUES FROM (2) TO (5); +INSERT INTO salespeople (salesperson_name) VALUES ('Poirot'); +INSERT INTO salespeople (salesperson_name) VALUES ('Ivanov'); +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople1_2'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +-- Split partition has identity column: +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople2_5'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +ALTER TABLE salespeople SPLIT PARTITION salespeople2_5 INTO + (PARTITION salespeople2_3 FOR VALUES FROM (2) TO (3), + PARTITION salespeople3_4 FOR VALUES FROM (3) TO (4), + PARTITION salespeople4_5 FOR VALUES FROM (4) TO (5)); +INSERT INTO salespeople (salesperson_name) VALUES ('May'); +INSERT INTO salespeople (salesperson_name) VALUES ('Ford'); +SELECT * FROM salespeople1_2; + salesperson_id | salesperson_name +----------------+------------------ + 1 | Poirot +(1 row) + +SELECT * FROM salespeople2_3; + salesperson_id | salesperson_name +----------------+------------------ + 2 | Ivanov +(1 row) + +SELECT * FROM salespeople3_4; + salesperson_id | salesperson_name +----------------+------------------ + 3 | May +(1 row) + +SELECT * FROM salespeople4_5; + salesperson_id | salesperson_name +----------------+------------------ + 4 | Ford +(1 row) + +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople1_2'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +-- New partitions have identity-columns: +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople2_3'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople3_4'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople4_5'::regclass::oid ORDER BY attnum; + attname | attidentity | attgenerated +------------------+-------------+-------------- + salesperson_id | a | + salesperson_name | | +(2 rows) + +DROP TABLE salespeople CASCADE; +-- +-- Test: split partition 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 new partition with some deleted columns: +CREATE TABLE salespeople10_40(d1 VARCHAR(30), salesperson_id INT PRIMARY KEY, d2 INT, d3 DATE, salesperson_name VARCHAR(30)); +INSERT INTO salespeople10_40 VALUES ('dummy value 1', 19, 100, now(), 'Ivanov'); +INSERT INTO salespeople10_40 VALUES ('dummy value 2', 20, 101, now(), 'Smirnoff'); +ALTER TABLE salespeople10_40 DROP COLUMN d1; +ALTER TABLE salespeople10_40 DROP COLUMN d2; +ALTER TABLE salespeople10_40 DROP COLUMN d3; +ALTER TABLE salespeople ATTACH PARTITION salespeople10_40 FOR VALUES FROM (10) TO (40); +INSERT INTO salespeople VALUES (1, 'Poirot'); +INSERT INTO salespeople VALUES (10, 'May'); +INSERT INTO salespeople VALUES (30, 'Ford'); +ALTER TABLE salespeople SPLIT PARTITION salespeople10_40 INTO + (PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20), + PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30), + PARTITION salespeople30_40 FOR VALUES FROM (30) TO (40)); +select * from salespeople01_10; + salesperson_id | salesperson_name +----------------+------------------ + 1 | Poirot +(1 row) + +select * from salespeople10_20; + salesperson_id | salesperson_name +----------------+------------------ + 19 | Ivanov + 10 | May +(2 rows) + +select * from salespeople20_30; + salesperson_id | salesperson_name +----------------+------------------ + 20 | Smirnoff +(1 row) + +select * from salespeople30_40; + salesperson_id | salesperson_name +----------------+------------------ + 30 | Ford +(1 row) + +DROP TABLE salespeople CASCADE; +-- +-- Test: split sub-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 (salesperson_id INT, salesperson_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date); +CREATE TABLE sales_apr_all PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-01') 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 + 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_apr2022; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 5 | Deev | 250 | 04-07-2022 + 11 | Trump | 380 | 04-06-2022 +(4 rows) + +ALTER TABLE sales_apr2022 SPLIT PARTITION sales_apr_all INTO + (PARTITION sales_apr2022_01_10 FOR VALUES FROM ('2022-04-01') TO ('2022-04-10'), + PARTITION sales_apr2022_10_20 FOR VALUES FROM ('2022-04-10') TO ('2022-04-20'), + PARTITION sales_apr2022_20_30 FOR VALUES FROM ('2022-04-20') TO ('2022-05-01')); +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) + +DROP TABLE sales_range; +-- +-- BY LIST partitioning +-- +-- +-- Test: specific errors for BY LIST partitioning +-- +CREATE TABLE sales_list +(salesperson_id INT, + 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_all PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Lisbon', 'New York', 'Madrid', 'Bejing', 'Berlin', 'Delhi', 'Kyiv', 'Vladivostok'); +CREATE TABLE sales_others PARTITION OF sales_list DEFAULT; +-- ERROR: new partition "sales_east" would overlap with another (not split) partition "sales_nord" +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok', 'Helsinki'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); +ERROR: new partition "sales_east" would overlap with another (not split) partition "sales_nord" +LINE 3: ... FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok', 'Helsinki'... + ^ +-- ERROR: new partition "sales_west" would overlap with another new partition "sales_central" +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Lisbon', 'Kyiv')); +ERROR: new partition "sales_west" would overlap with another new partition "sales_central" +LINE 2: (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York',... + ^ +-- ERROR: new partition "sales_west" cannot have NULL value because split partition does not have +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid', NULL), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); +ERROR: new partition "sales_west" cannot have NULL value because split partition does not have +LINE 2: ...s_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid', NULL), + ^ +DROP TABLE sales_list; +-- +-- Test: two specific errors for BY LIST partitioning: +-- * new partitions do not have NULL value, which split partition has. +-- * new partitions do not have a value that split partition has. +-- +CREATE TABLE sales_list +(salesperson_id INT, + 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 ('Helsinki', 'St. Petersburg', 'Oslo'); +CREATE TABLE sales_all PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Lisbon', 'New York', 'Madrid', 'Bejing', 'Berlin', 'Delhi', 'Kyiv', 'Vladivostok', NULL); +-- ERROR: new partitions do not have value NULL but split partition does +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); +ERROR: new partitions do not have value NULL but split partition does +-- ERROR: new partitions do not have value 'Kyiv' but split partition does +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', NULL)); +ERROR: new partitions do not have value 'Kyiv' but split partition does +DROP TABLE sales_list; +-- +-- Test: BY LIST partitioning, SPLIT PARTITION with data +-- +CREATE TABLE sales_list +(salesperson_id SERIAL, + 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 ('Helsinki', 'St. Petersburg', 'Oslo'); +CREATE TABLE sales_all PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Lisbon', 'New York', 'Madrid', 'Bejing', 'Berlin', 'Delhi', 'Kyiv', 'Vladivostok'); +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', 'Oslo', 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', 'Oslo', 1200, '2022-03-11'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Halder', 'Helsinki', 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'); +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); +SELECT * FROM sales_list; + salesperson_id | salesperson_name | sales_state | sales_amount | sales_date +----------------+------------------+----------------+--------------+------------ + 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 | Oslo | 1200 | 03-06-2022 + 9 | May | Oslo | 1200 | 03-11-2022 + 10 | Halder | Helsinki | 800 | 03-02-2022 + 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 +(14 rows) + +SELECT * FROM sales_west; + 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 +(4 rows) + +SELECT * FROM sales_east; + salesperson_id | salesperson_name | sales_state | sales_amount | sales_date +----------------+------------------+-------------+--------------+------------ + 1 | Trump | Bejing | 1000 | 03-01-2022 + 8 | Li | Vladivostok | 1150 | 03-09-2022 +(2 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 | Oslo | 1200 | 03-06-2022 + 9 | May | Oslo | 1200 | 03-11-2022 + 10 | Halder | Helsinki | 800 | 03-02-2022 +(4 rows) + +SELECT * FROM sales_central; + salesperson_id | salesperson_name | sales_state | sales_amount | sales_date +----------------+------------------+-------------+--------------+------------ + 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 +(4 rows) + +-- Use indexscan for testing indexes after splitting partition +SET enable_indexscan = ON; +SET enable_seqscan = OFF; +SELECT * FROM sales_central 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) + +SET enable_indexscan = ON; +SET enable_seqscan = ON; +DROP TABLE sales_list; +-- +-- Test for: +-- * split DEFAULT partition to partitions with spaces between bounds; +-- * random order of partitions in SPLIT PARTITION command. +-- +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_others PARTITION OF sales_range DEFAULT; +INSERT INTO sales_range VALUES (1, 'May', 1000, '2022-01-31'); +INSERT INTO sales_range VALUES (2, 'Smirnoff', 500, '2022-02-09'); +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-07'); +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'); +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_others DEFAULT, + PARTITION sales_mar2022_1decade FOR VALUES FROM ('2022-03-01') TO ('2022-03-10'), + PARTITION sales_jan2022_1decade FOR VALUES FROM ('2022-01-01') TO ('2022-01-10'), + PARTITION sales_feb2022_1decade FOR VALUES FROM ('2022-02-01') TO ('2022-02-10'), + PARTITION sales_apr2022_1decade FOR VALUES FROM ('2022-04-01') TO ('2022-04-10')); +SELECT * FROM sales_jan2022_1decade; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 13 | Gandi | 377 | 01-09-2022 +(1 row) + +SELECT * FROM sales_feb2022_1decade; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 2 | Smirnoff | 500 | 02-09-2022 + 6 | Poirot | 150 | 02-07-2022 +(2 rows) + +SELECT * FROM sales_mar2022_1decade; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 7 | Li | 175 | 03-08-2022 +(1 row) + +SELECT * FROM sales_apr2022_1decade; + 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_others; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 1 | May | 1000 | 01-31-2022 + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 8 | Ericsson | 185 | 02-23-2022 + 9 | Muller | 250 | 03-11-2022 + 10 | Halder | 350 | 01-28-2022 + 12 | Plato | 350 | 03-19-2022 + 14 | Smith | 510 | 05-04-2022 +(8 rows) + +DROP TABLE sales_range; +-- +-- Test for: +-- * split non-DEFAULT partition to partitions with spaces between bounds; +-- * random order of partitions in SPLIT PARTITION command. +-- +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_all PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; +INSERT INTO sales_range VALUES (1, 'May', 1000, '2022-01-31'); +INSERT INTO sales_range VALUES (2, 'Smirnoff', 500, '2022-02-09'); +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-07'); +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'); +ALTER TABLE sales_range SPLIT PARTITION sales_all INTO + (PARTITION sales_mar2022_1decade FOR VALUES FROM ('2022-03-01') TO ('2022-03-10'), + PARTITION sales_jan2022_1decade FOR VALUES FROM ('2022-01-01') TO ('2022-01-10'), + PARTITION sales_feb2022_1decade FOR VALUES FROM ('2022-02-01') TO ('2022-02-10'), + PARTITION sales_apr2022_1decade FOR VALUES FROM ('2022-04-01') TO ('2022-04-10')); +SELECT * FROM sales_jan2022_1decade; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 13 | Gandi | 377 | 01-09-2022 +(1 row) + +SELECT * FROM sales_feb2022_1decade; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 2 | Smirnoff | 500 | 02-09-2022 + 6 | Poirot | 150 | 02-07-2022 +(2 rows) + +SELECT * FROM sales_mar2022_1decade; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 7 | Li | 175 | 03-08-2022 +(1 row) + +SELECT * FROM sales_apr2022_1decade; + 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_others; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 14 | Smith | 510 | 05-04-2022 + 1 | May | 1000 | 01-31-2022 + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 8 | Ericsson | 185 | 02-23-2022 + 9 | Muller | 250 | 03-11-2022 + 10 | Halder | 350 | 01-28-2022 + 12 | Plato | 350 | 03-19-2022 +(8 rows) + +DROP TABLE sales_range; +-- +-- Test for split non-DEFAULT partition to DEFAULT partition + partitions +-- with spaces between bounds. +-- +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_all PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +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'); +ALTER TABLE sales_range SPLIT PARTITION sales_all INTO + (PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01'), + PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_others DEFAULT); +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 + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 5 | Deev | 250 | 04-07-2022 + 11 | Trump | 380 | 04-06-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 +(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 sales_feb2022; + 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 +(3 rows) + +SELECT * FROM sales_apr2022; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 3 | Ford | 2000 | 04-30-2022 + 4 | Ivanov | 750 | 04-13-2022 + 5 | Deev | 250 | 04-07-2022 + 11 | Trump | 380 | 04-06-2022 +(4 rows) + +SELECT * FROM sales_others; + salesperson_id | salesperson_name | sales_amount | sales_date +----------------+------------------+--------------+------------ + 7 | Li | 175 | 03-08-2022 + 9 | Muller | 250 | 03-11-2022 + 12 | Plato | 350 | 03-19-2022 + 14 | Smith | 510 | 05-04-2022 +(4 rows) + +DROP TABLE sales_range; +-- +-- Try to SPLIT partition of another table. +-- +CREATE TABLE t1(i int, t text) PARTITION BY LIST (t); +CREATE TABLE t1pa PARTITION OF t1 FOR VALUES IN ('A'); +CREATE TABLE t2 (i int, t text) PARTITION BY RANGE (t); +-- ERROR: relation "t1pa" is not a partition of relation "t2" +ALTER TABLE t2 SPLIT PARTITION t1pa INTO + (PARTITION t2a FOR VALUES FROM ('A') TO ('B'), + PARTITION t2b FOR VALUES FROM ('B') TO ('C')); +ERROR: relation "t1pa" is not a partition of relation "t2" +DROP TABLE t2; +DROP TABLE t1; +-- +-- Try to SPLIT partition of temporary table. +-- +CREATE TEMP TABLE t (i int) PARTITION BY RANGE (i); +CREATE TEMP TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) 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_2 | FOR VALUES FROM (0) TO (2) | t +(1 row) + +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +-- Partitions 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_1 | FOR VALUES FROM (0) TO (1) | t + tp_1_2 | FOR VALUES FROM (1) TO (2) | t +(2 rows) + +DROP TABLE t; +-- Check the new partitions inherit parent's tablespace +CREATE TABLE t (i int PRIMARY KEY USING INDEX TABLESPACE regress_tblspace) + PARTITION BY RANGE (i) TABLESPACE regress_tblspace; +CREATE TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) TO (2); +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +SELECT tablename, tablespace FROM pg_tables + WHERE tablename IN ('t', 'tp_0_1', 'tp_1_2') AND schemaname = 'partition_split_schema' + ORDER BY tablename, tablespace; + tablename | tablespace +-----------+------------------ + t | regress_tblspace + tp_0_1 | regress_tblspace + tp_1_2 | regress_tblspace +(3 rows) + +SELECT tablename, indexname, tablespace FROM pg_indexes + WHERE tablename IN ('t', 'tp_0_1', 'tp_1_2') AND schemaname = 'partition_split_schema' + ORDER BY tablename, indexname, tablespace; + tablename | indexname | tablespace +-----------+-------------+------------------ + t | t_pkey | regress_tblspace + tp_0_1 | tp_0_1_pkey | regress_tblspace + tp_1_2 | tp_1_2_pkey | regress_tblspace +(3 rows) + +DROP TABLE t; +-- Check new partitions inherits parent's table access method +CREATE ACCESS METHOD partition_split_heap TYPE TABLE HANDLER heap_tableam_handler; +CREATE TABLE t (i int) PARTITION BY RANGE (i) USING partition_split_heap; +CREATE TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) TO (2); +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (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_1'::regclass, 'tp_1_2'::regclass) +ORDER BY c.relname; + relname | amname +---------+---------------------- + t | partition_split_heap + tp_0_1 | partition_split_heap + tp_1_2 | partition_split_heap +(3 rows) + +DROP TABLE t; +DROP ACCESS METHOD partition_split_heap; +-- Test permission checks. The user needs to own the parent table and the +-- the partition to split to do the split. +CREATE ROLE regress_partition_split_alice; +CREATE ROLE regress_partition_split_bob; +GRANT ALL ON SCHEMA partition_split_schema TO regress_partition_split_alice; +GRANT ALL ON SCHEMA partition_split_schema TO regress_partition_split_bob; +SET SESSION AUTHORIZATION regress_partition_split_alice; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) TO (2); +SET SESSION AUTHORIZATION regress_partition_split_bob; +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +ERROR: must be owner of table t +RESET SESSION AUTHORIZATION; +ALTER TABLE t OWNER TO regress_partition_split_bob; +SET SESSION AUTHORIZATION regress_partition_split_bob; +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +ERROR: must be owner of table tp_0_2 +RESET SESSION AUTHORIZATION; +ALTER TABLE tp_0_2 OWNER TO regress_partition_split_bob; +SET SESSION AUTHORIZATION regress_partition_split_bob; +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +RESET SESSION AUTHORIZATION; +DROP TABLE t; +REVOKE ALL ON SCHEMA partition_split_schema FROM regress_partition_split_alice; +REVOKE ALL ON SCHEMA partition_split_schema FROM regress_partition_split_bob; +DROP ROLE regress_partition_split_alice; +DROP ROLE regress_partition_split_bob; +-- Split partition of a temporary table when one of the partitions after +-- split has the same name as the partition being split +CREATE TEMP TABLE t (a int) PARTITION BY RANGE (a); +CREATE TEMP TABLE tp_0 PARTITION OF t FOR VALUES FROM (0) TO (2); +ALTER TABLE t SPLIT PARTITION tp_0 INTO + (PARTITION tp_0 FOR VALUES FROM (0) TO (1), + PARTITION tp_1 FOR VALUES FROM (1) TO (2)); +DROP TABLE t; +RESET search_path; +-- +DROP SCHEMA partition_split_schema; +DROP SCHEMA partition_split_schema2; diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule index 0014801684..2429ec2bba 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_merge partition_join partition_prune reloptions hash_part indexing partition_aggregate partition_info tuplesort explain compression memoize stats predicate +test: partition_merge partition_split 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_split.sql b/src/test/regress/sql/partition_split.sql new file mode 100644 index 0000000000..ef5ea07f74 --- /dev/null +++ b/src/test/regress/sql/partition_split.sql @@ -0,0 +1,962 @@ +-- +-- PARTITION_SPLIT +-- Tests for "ALTER TABLE ... SPLIT PARTITION ..." command +-- + +CREATE SCHEMA partition_split_schema; +CREATE SCHEMA partition_split_schema2; +SET search_path = partition_split_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_jan2022 PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-02-01'); +CREATE TABLE sales_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; + +-- ERROR: relation "sales_xxx" does not exist +ALTER TABLE sales_range SPLIT PARTITION sales_xxx INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- ERROR: relation "sales_jan2022" already exists +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_jan2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- ERROR: invalid bound specification for a range partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_jan2022 FOR VALUES IN ('2022-05-01', '2022-06-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- ERROR: empty range bound specified for partition "sales_mar2022" +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-02-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +--ERROR: list of split partitions should contain at least two items +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-10-01')); + +-- ERROR: lower bound of partition "sales_feb2022" is less than lower bound of split partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-01-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- ERROR: name "sales_feb_mar_apr2022" is already used +-- (We can create partition with the same name as split partition, but can't create two partitions with the same name) +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb_mar_apr2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_feb_mar_apr2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- ERROR: name "sales_feb2022" is already used +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_feb2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- ERROR: "sales_feb_mar_apr2022" is not a partitioned table +ALTER TABLE sales_feb_mar_apr2022 SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_jan2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_feb2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- ERROR: upper bound of partition "sales_apr2022" is greater than upper bound of split partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-06-01')); + +-- ERROR: lower bound of partition "sales_mar2022" conflicts with upper bound of previous partition "sales_feb2022" +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-02-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- Tests for spaces between partitions, them should be executed without DEFAULT partition +ALTER TABLE sales_range DETACH PARTITION sales_others; + +-- ERROR: lower bound of partition "sales_feb2022" is not equal to lower bound of split partition +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-02') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- Check the source partition not in the search path +SET search_path = partition_split_schema2, public; +ALTER TABLE partition_split_schema.sales_range +SPLIT PARTITION partition_split_schema.sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +SET search_path = partition_split_schema, public; +\d+ sales_range + +DROP TABLE sales_range; +DROP TABLE sales_others; + +-- +-- Add rows into partitioned table then split 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_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; + +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'); + +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +SELECT * FROM sales_range; +SELECT * FROM sales_jan2022; +SELECT * FROM sales_feb2022; +SELECT * FROM sales_mar2022; +SELECT * FROM sales_apr2022; +SELECT * FROM sales_others; + +DROP TABLE sales_range CASCADE; + +-- +-- Add split partition, then add rows into partitioned table +-- +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_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; + +-- Split partition, also check schema qualification of new partitions +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION partition_split_schema.sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION partition_split_schema2.sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); +\d+ sales_range + +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_jan2022; +SELECT * FROM sales_feb2022; +SELECT * FROM partition_split_schema2.sales_mar2022; +SELECT * FROM sales_apr2022; +SELECT * FROM sales_others; + +DROP TABLE sales_range CASCADE; + +-- +-- 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_jan_feb2022 PARTITION OF sales_date FOR VALUES FROM (2022, 1, 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_jan_feb2022; +SELECT * FROM sales_other; + +ALTER TABLE sales_date SPLIT PARTITION sales_jan_feb2022 INTO + (PARTITION sales_jan2022 FOR VALUES FROM (2022, 1, 1) TO (2022, 2, 1), + PARTITION sales_feb2022 FOR VALUES FROM (2022, 2, 1) TO (2022, 3, 1)); + +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_jan2022; +SELECT * FROM sales_feb2022; +SELECT * FROM sales_other; + +--ERROR: relation "sales_jan_feb2022" does not exist +SELECT * FROM sales_jan_feb2022; + +DROP TABLE sales_date CASCADE; + +-- +-- Test: split DEFAULT partition; use an index on partition key; check index after split +-- +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_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_others; +SELECT * FROM pg_indexes WHERE tablename = 'sales_others' and schemaname = 'partition_split_schema' ORDER BY indexname; + +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01'), + PARTITION sales_others DEFAULT); + +-- Use indexscan for testing indexes +SET enable_indexscan = ON; +SET enable_seqscan = OFF; + +SELECT * FROM sales_feb2022 where sales_date > '2022-01-01'; +SELECT * FROM sales_mar2022 where sales_date > '2022-01-01'; +SELECT * FROM sales_apr2022 where sales_date > '2022-01-01'; +SELECT * FROM sales_others where sales_date > '2022-01-01'; + +SET enable_indexscan = ON; +SET enable_seqscan = ON; + +SELECT * FROM pg_indexes WHERE tablename = 'sales_feb2022' and schemaname = 'partition_split_schema' ORDER BY indexname; +SELECT * FROM pg_indexes WHERE tablename = 'sales_mar2022' and schemaname = 'partition_split_schema' ORDER BY indexname; +SELECT * FROM pg_indexes WHERE tablename = 'sales_apr2022' and schemaname = 'partition_split_schema' ORDER BY indexname; +SELECT * FROM pg_indexes WHERE tablename = 'sales_others' and schemaname = 'partition_split_schema' ORDER BY indexname; + +DROP TABLE sales_range CASCADE; + +-- +-- Test: some cases for splitting DEFAULT partition (different bounds) +-- +CREATE TABLE sales_range (salesperson_id INT, salesperson_name VARCHAR(30), sales_amount INT, sales_date INT) PARTITION BY RANGE (sales_date); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; + +-- sales_error intersects with sales_dec2022 (lower bound) +-- ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20211230) TO (20220201), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); + +-- sales_error intersects with sales_feb2022 (upper bound) +-- ERROR: lower bound of partition "sales_feb2022" conflicts with upper bound of previous partition "sales_error" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20220101) TO (20220202), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); + +-- sales_error intersects with sales_dec2022 (inside bound) +-- ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20211210) TO (20211220), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); + +-- sales_error intersects with sales_dec2022 (exactly the same bounds) +-- ERROR: lower bound of partition "sales_error" conflicts with upper bound of previous partition "sales_dec2022" +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_error FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); + +-- ERROR: all partitions in the list should be DEFAULT because split partition is DEFAULT +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_jan2022 FOR VALUES FROM (20220101) TO (20220201), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301)); + +-- no error: bounds of sales_noerror are between sales_dec2022 and sales_feb2022 +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_noerror FOR VALUES FROM (20220110) TO (20220120), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); + +DROP TABLE sales_range; + +CREATE TABLE sales_range (salesperson_id INT, salesperson_name VARCHAR(30), sales_amount INT, sales_date INT) PARTITION BY RANGE (sales_date); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; + +-- no error: bounds of sales_noerror are equal to lower and upper bounds of sales_dec2022 and sales_feb2022 +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_dec2022 FOR VALUES FROM (20211201) TO (20220101), + PARTITION sales_noerror FOR VALUES FROM (20210101) TO (20210201), + PARTITION sales_feb2022 FOR VALUES FROM (20220201) TO (20220301), + PARTITION sales_others DEFAULT); + +DROP TABLE sales_range; + +-- +-- Test: split partition with CHECK and FOREIGN KEY CONSTRAINTs on partitioned table +-- +CREATE TABLE salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)); +INSERT INTO salespeople VALUES (1, 'Poirot'); + +CREATE TABLE sales_range ( +salesperson_id INT REFERENCES salespeople(salesperson_id), +sales_amount INT CHECK (sales_amount > 1), +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_feb_mar_apr2022 PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; + +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_feb_mar_apr2022'::regclass::oid ORDER BY conname; + +ALTER TABLE sales_range SPLIT PARTITION sales_feb_mar_apr2022 INTO + (PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_mar2022 FOR VALUES FROM ('2022-03-01') TO ('2022-04-01'), + PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01')); + +-- We should see the same CONSTRAINTs as on sales_feb_mar_apr2022 partition +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_feb2022'::regclass::oid ORDER BY conname;; +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_mar2022'::regclass::oid ORDER BY conname;; +SELECT pg_get_constraintdef(oid), conname, conkey FROM pg_constraint WHERE conrelid = 'sales_apr2022'::regclass::oid ORDER BY conname;; + +-- ERROR: new row for relation "sales_mar2022" violates check constraint "sales_range_sales_amount_check" +INSERT INTO sales_range VALUES (1, 0, '2022-03-11'); +-- ERROR: insert or update on table "sales_mar2022" violates foreign key constraint "sales_range_salesperson_id_fkey" +INSERT INTO sales_range VALUES (-1, 10, '2022-03-11'); +-- ok +INSERT INTO sales_range VALUES (1, 10, '2022-03-11'); + +DROP TABLE sales_range CASCADE; +DROP TABLE salespeople CASCADE; + +-- +-- Test: split partition on partitioned table in case of existing FOREIGN KEY reference from another table +-- +CREATE TABLE salespeople(salesperson_id INT PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); +CREATE TABLE sales (salesperson_id INT REFERENCES salespeople(salesperson_id), sales_amount INT, sales_date DATE); + +CREATE TABLE salespeople01_10 PARTITION OF salespeople FOR VALUES FROM (1) TO (10); +CREATE TABLE salespeople10_40 PARTITION OF salespeople FOR VALUES FROM (10) TO (40); + +INSERT INTO salespeople VALUES (1, 'Poirot'); +INSERT INTO salespeople VALUES (10, 'May'); +INSERT INTO salespeople VALUES (19, 'Ivanov'); +INSERT INTO salespeople VALUES (20, 'Smirnoff'); +INSERT INTO salespeople VALUES (30, 'Ford'); + +INSERT INTO sales VALUES (1, 100, '2022-03-01'); +INSERT INTO sales VALUES (1, 110, '2022-03-02'); +INSERT INTO sales VALUES (10, 150, '2022-03-01'); +INSERT INTO sales VALUES (10, 90, '2022-03-03'); +INSERT INTO sales VALUES (19, 200, '2022-03-04'); +INSERT INTO sales VALUES (20, 50, '2022-03-12'); +INSERT INTO sales VALUES (20, 170, '2022-03-02'); +INSERT INTO sales VALUES (30, 30, '2022-03-04'); + +SELECT * FROM salespeople01_10; +SELECT * FROM salespeople10_40; + +ALTER TABLE salespeople SPLIT PARTITION salespeople10_40 INTO + (PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20), + PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30), + PARTITION salespeople30_40 FOR VALUES FROM (30) TO (40)); + +SELECT * FROM salespeople01_10; +SELECT * FROM salespeople10_20; +SELECT * FROM salespeople20_30; +SELECT * FROM salespeople30_40; + +-- ERROR: insert or update on table "sales" violates foreign key constraint "sales_salesperson_id_fkey" +INSERT INTO sales VALUES (40, 50, '2022-03-04'); +-- ok +INSERT INTO sales VALUES (30, 50, '2022-03-04'); + +DROP TABLE sales CASCADE; +DROP TABLE salespeople CASCADE; + +-- +-- Test: split partition 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_40 PARTITION OF salespeople FOR VALUES FROM (10) 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_40 VALUES (19, 'Ivanov'); + +ALTER TABLE salespeople SPLIT PARTITION salespeople10_40 INTO + (PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20), + PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30), + PARTITION salespeople30_40 FOR VALUES FROM (30) TO (40)); + +-- 2 triggers should fire here (row + statement): +INSERT INTO salespeople VALUES (20, 'Smirnoff'); +-- 1 trigger should fire here (row): +INSERT INTO salespeople30_40 VALUES (30, 'Ford'); + +SELECT * FROM salespeople01_10; +SELECT * FROM salespeople10_20; +SELECT * FROM salespeople20_30; +SELECT * FROM salespeople30_40; + +DROP TABLE salespeople CASCADE; +DROP FUNCTION after_insert_row_trigger(); + +-- +-- Test: split partition witch identity column +-- If split partition column is identity column, columns of new partitions are identity columns too. +-- +CREATE TABLE salespeople(salesperson_id INT GENERATED ALWAYS AS IDENTITY PRIMARY KEY, salesperson_name VARCHAR(30)) PARTITION BY RANGE (salesperson_id); + +CREATE TABLE salespeople1_2 PARTITION OF salespeople FOR VALUES FROM (1) TO (2); +-- Create new partition with identity column: +CREATE TABLE salespeople2_5(salesperson_id INT NOT NULL, salesperson_name VARCHAR(30)); +ALTER TABLE salespeople ATTACH PARTITION salespeople2_5 FOR VALUES FROM (2) TO (5); + +INSERT INTO salespeople (salesperson_name) VALUES ('Poirot'); +INSERT INTO salespeople (salesperson_name) VALUES ('Ivanov'); + +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople'::regclass::oid ORDER BY attnum; +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople1_2'::regclass::oid ORDER BY attnum; +-- Split partition has identity column: +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople2_5'::regclass::oid ORDER BY attnum; + +ALTER TABLE salespeople SPLIT PARTITION salespeople2_5 INTO + (PARTITION salespeople2_3 FOR VALUES FROM (2) TO (3), + PARTITION salespeople3_4 FOR VALUES FROM (3) TO (4), + PARTITION salespeople4_5 FOR VALUES FROM (4) TO (5)); + +INSERT INTO salespeople (salesperson_name) VALUES ('May'); +INSERT INTO salespeople (salesperson_name) VALUES ('Ford'); + +SELECT * FROM salespeople1_2; +SELECT * FROM salespeople2_3; +SELECT * FROM salespeople3_4; +SELECT * FROM salespeople4_5; + +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople'::regclass::oid ORDER BY attnum; +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople1_2'::regclass::oid ORDER BY attnum; +-- New partitions have identity-columns: +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople2_3'::regclass::oid ORDER BY attnum; +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople3_4'::regclass::oid ORDER BY attnum; +SELECT attname, attidentity, attgenerated FROM pg_attribute WHERE attnum > 0 AND attrelid = 'salespeople4_5'::regclass::oid ORDER BY attnum; + +DROP TABLE salespeople CASCADE; + +-- +-- Test: split partition 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 new partition with some deleted columns: +CREATE TABLE salespeople10_40(d1 VARCHAR(30), salesperson_id INT PRIMARY KEY, d2 INT, d3 DATE, salesperson_name VARCHAR(30)); + +INSERT INTO salespeople10_40 VALUES ('dummy value 1', 19, 100, now(), 'Ivanov'); +INSERT INTO salespeople10_40 VALUES ('dummy value 2', 20, 101, now(), 'Smirnoff'); + +ALTER TABLE salespeople10_40 DROP COLUMN d1; +ALTER TABLE salespeople10_40 DROP COLUMN d2; +ALTER TABLE salespeople10_40 DROP COLUMN d3; + +ALTER TABLE salespeople ATTACH PARTITION salespeople10_40 FOR VALUES FROM (10) TO (40); + +INSERT INTO salespeople VALUES (1, 'Poirot'); +INSERT INTO salespeople VALUES (10, 'May'); +INSERT INTO salespeople VALUES (30, 'Ford'); + +ALTER TABLE salespeople SPLIT PARTITION salespeople10_40 INTO + (PARTITION salespeople10_20 FOR VALUES FROM (10) TO (20), + PARTITION salespeople20_30 FOR VALUES FROM (20) TO (30), + PARTITION salespeople30_40 FOR VALUES FROM (30) TO (40)); + +select * from salespeople01_10; +select * from salespeople10_20; +select * from salespeople20_30; +select * from salespeople30_40; + +DROP TABLE salespeople CASCADE; + +-- +-- Test: split sub-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 (salesperson_id INT, salesperson_name VARCHAR(30), sales_amount INT, sales_date DATE) PARTITION BY RANGE (sales_date); +CREATE TABLE sales_apr_all PARTITION OF sales_apr2022 FOR VALUES FROM ('2022-04-01') 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; + +ALTER TABLE sales_apr2022 SPLIT PARTITION sales_apr_all INTO + (PARTITION sales_apr2022_01_10 FOR VALUES FROM ('2022-04-01') TO ('2022-04-10'), + PARTITION sales_apr2022_10_20 FOR VALUES FROM ('2022-04-10') TO ('2022-04-20'), + PARTITION sales_apr2022_20_30 FOR VALUES FROM ('2022-04-20') TO ('2022-05-01')); + +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; + +DROP TABLE sales_range; + +-- +-- BY LIST partitioning +-- + +-- +-- Test: specific errors for BY LIST partitioning +-- +CREATE TABLE sales_list +(salesperson_id INT, + 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_all PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Lisbon', 'New York', 'Madrid', 'Bejing', 'Berlin', 'Delhi', 'Kyiv', 'Vladivostok'); +CREATE TABLE sales_others PARTITION OF sales_list DEFAULT; + +-- ERROR: new partition "sales_east" would overlap with another (not split) partition "sales_nord" +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok', 'Helsinki'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); + +-- ERROR: new partition "sales_west" would overlap with another new partition "sales_central" +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Lisbon', 'Kyiv')); + +-- ERROR: new partition "sales_west" cannot have NULL value because split partition does not have +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid', NULL), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); + +DROP TABLE sales_list; + +-- +-- Test: two specific errors for BY LIST partitioning: +-- * new partitions do not have NULL value, which split partition has. +-- * new partitions do not have a value that split partition has. +-- +CREATE TABLE sales_list +(salesperson_id INT, + 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 ('Helsinki', 'St. Petersburg', 'Oslo'); +CREATE TABLE sales_all PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Lisbon', 'New York', 'Madrid', 'Bejing', 'Berlin', 'Delhi', 'Kyiv', 'Vladivostok', NULL); + +-- ERROR: new partitions do not have value NULL but split partition does +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); + +-- ERROR: new partitions do not have value 'Kyiv' but split partition does +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', NULL)); + +DROP TABLE sales_list; + +-- +-- Test: BY LIST partitioning, SPLIT PARTITION with data +-- +CREATE TABLE sales_list +(salesperson_id SERIAL, + 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 ('Helsinki', 'St. Petersburg', 'Oslo'); +CREATE TABLE sales_all PARTITION OF sales_list FOR VALUES IN ('Warsaw', 'Lisbon', 'New York', 'Madrid', 'Bejing', 'Berlin', 'Delhi', 'Kyiv', 'Vladivostok'); +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', 'Oslo', 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', 'Oslo', 1200, '2022-03-11'); +INSERT INTO sales_list (salesperson_name, sales_state, sales_amount, sales_date) VALUES ('Halder', 'Helsinki', 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'); + +ALTER TABLE sales_list SPLIT PARTITION sales_all INTO + (PARTITION sales_west FOR VALUES IN ('Lisbon', 'New York', 'Madrid'), + PARTITION sales_east FOR VALUES IN ('Bejing', 'Delhi', 'Vladivostok'), + PARTITION sales_central FOR VALUES IN ('Warsaw', 'Berlin', 'Kyiv')); + +SELECT * FROM sales_list; +SELECT * FROM sales_west; +SELECT * FROM sales_east; +SELECT * FROM sales_nord; +SELECT * FROM sales_central; + +-- Use indexscan for testing indexes after splitting partition +SET enable_indexscan = ON; +SET enable_seqscan = OFF; + +SELECT * FROM sales_central WHERE sales_state = 'Warsaw'; +SELECT * FROM sales_list WHERE sales_state = 'Warsaw'; +SELECT * FROM sales_list WHERE salesperson_name = 'Ivanov'; + +SET enable_indexscan = ON; +SET enable_seqscan = ON; + +DROP TABLE sales_list; + +-- +-- Test for: +-- * split DEFAULT partition to partitions with spaces between bounds; +-- * random order of partitions in SPLIT PARTITION command. +-- +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_others PARTITION OF sales_range DEFAULT; + +INSERT INTO sales_range VALUES (1, 'May', 1000, '2022-01-31'); +INSERT INTO sales_range VALUES (2, 'Smirnoff', 500, '2022-02-09'); +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-07'); +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'); + +ALTER TABLE sales_range SPLIT PARTITION sales_others INTO + (PARTITION sales_others DEFAULT, + PARTITION sales_mar2022_1decade FOR VALUES FROM ('2022-03-01') TO ('2022-03-10'), + PARTITION sales_jan2022_1decade FOR VALUES FROM ('2022-01-01') TO ('2022-01-10'), + PARTITION sales_feb2022_1decade FOR VALUES FROM ('2022-02-01') TO ('2022-02-10'), + PARTITION sales_apr2022_1decade FOR VALUES FROM ('2022-04-01') TO ('2022-04-10')); + +SELECT * FROM sales_jan2022_1decade; +SELECT * FROM sales_feb2022_1decade; +SELECT * FROM sales_mar2022_1decade; +SELECT * FROM sales_apr2022_1decade; +SELECT * FROM sales_others; + +DROP TABLE sales_range; + +-- +-- Test for: +-- * split non-DEFAULT partition to partitions with spaces between bounds; +-- * random order of partitions in SPLIT PARTITION command. +-- +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_all PARTITION OF sales_range FOR VALUES FROM ('2022-01-01') TO ('2022-05-01'); +CREATE TABLE sales_others PARTITION OF sales_range DEFAULT; + +INSERT INTO sales_range VALUES (1, 'May', 1000, '2022-01-31'); +INSERT INTO sales_range VALUES (2, 'Smirnoff', 500, '2022-02-09'); +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-07'); +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'); + +ALTER TABLE sales_range SPLIT PARTITION sales_all INTO + (PARTITION sales_mar2022_1decade FOR VALUES FROM ('2022-03-01') TO ('2022-03-10'), + PARTITION sales_jan2022_1decade FOR VALUES FROM ('2022-01-01') TO ('2022-01-10'), + PARTITION sales_feb2022_1decade FOR VALUES FROM ('2022-02-01') TO ('2022-02-10'), + PARTITION sales_apr2022_1decade FOR VALUES FROM ('2022-04-01') TO ('2022-04-10')); + +SELECT * FROM sales_jan2022_1decade; +SELECT * FROM sales_feb2022_1decade; +SELECT * FROM sales_mar2022_1decade; +SELECT * FROM sales_apr2022_1decade; +SELECT * FROM sales_others; + +DROP TABLE sales_range; + +-- +-- Test for split non-DEFAULT partition to DEFAULT partition + partitions +-- with spaces between bounds. +-- +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_all PARTITION OF sales_range FOR VALUES FROM ('2022-02-01') TO ('2022-05-01'); + +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'); + +ALTER TABLE sales_range SPLIT PARTITION sales_all INTO + (PARTITION sales_apr2022 FOR VALUES FROM ('2022-04-01') TO ('2022-05-01'), + PARTITION sales_feb2022 FOR VALUES FROM ('2022-02-01') TO ('2022-03-01'), + PARTITION sales_others DEFAULT); + +INSERT INTO sales_range VALUES (14, 'Smith', 510, '2022-05-04'); + +SELECT * FROM sales_range; +SELECT * FROM sales_jan2022; +SELECT * FROM sales_feb2022; +SELECT * FROM sales_apr2022; +SELECT * FROM sales_others; + +DROP TABLE sales_range; + +-- +-- Try to SPLIT partition of another table. +-- +CREATE TABLE t1(i int, t text) PARTITION BY LIST (t); +CREATE TABLE t1pa PARTITION OF t1 FOR VALUES IN ('A'); +CREATE TABLE t2 (i int, t text) PARTITION BY RANGE (t); + +-- ERROR: relation "t1pa" is not a partition of relation "t2" +ALTER TABLE t2 SPLIT PARTITION t1pa INTO + (PARTITION t2a FOR VALUES FROM ('A') TO ('B'), + PARTITION t2b FOR VALUES FROM ('B') TO ('C')); + +DROP TABLE t2; +DROP TABLE t1; + +-- +-- Try to SPLIT partition of temporary table. +-- +CREATE TEMP TABLE t (i int) PARTITION BY RANGE (i); +CREATE TEMP TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) 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 SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); + +-- Partitions 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 new partitions inherit parent's tablespace +CREATE TABLE t (i int PRIMARY KEY USING INDEX TABLESPACE regress_tblspace) + PARTITION BY RANGE (i) TABLESPACE regress_tblspace; +CREATE TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) TO (2); +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +SELECT tablename, tablespace FROM pg_tables + WHERE tablename IN ('t', 'tp_0_1', 'tp_1_2') AND schemaname = 'partition_split_schema' + ORDER BY tablename, tablespace; +SELECT tablename, indexname, tablespace FROM pg_indexes + WHERE tablename IN ('t', 'tp_0_1', 'tp_1_2') AND schemaname = 'partition_split_schema' + ORDER BY tablename, indexname, tablespace; +DROP TABLE t; + +-- Check new partitions inherits parent's table access method +CREATE ACCESS METHOD partition_split_heap TYPE TABLE HANDLER heap_tableam_handler; +CREATE TABLE t (i int) PARTITION BY RANGE (i) USING partition_split_heap; +CREATE TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) TO (2); +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (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_1'::regclass, 'tp_1_2'::regclass) +ORDER BY c.relname; +DROP TABLE t; +DROP ACCESS METHOD partition_split_heap; + +-- Test permission checks. The user needs to own the parent table and the +-- the partition to split to do the split. +CREATE ROLE regress_partition_split_alice; +CREATE ROLE regress_partition_split_bob; +GRANT ALL ON SCHEMA partition_split_schema TO regress_partition_split_alice; +GRANT ALL ON SCHEMA partition_split_schema TO regress_partition_split_bob; + +SET SESSION AUTHORIZATION regress_partition_split_alice; +CREATE TABLE t (i int) PARTITION BY RANGE (i); +CREATE TABLE tp_0_2 PARTITION OF t FOR VALUES FROM (0) TO (2); + +SET SESSION AUTHORIZATION regress_partition_split_bob; +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +RESET SESSION AUTHORIZATION; + +ALTER TABLE t OWNER TO regress_partition_split_bob; +SET SESSION AUTHORIZATION regress_partition_split_bob; +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +RESET SESSION AUTHORIZATION; + +ALTER TABLE tp_0_2 OWNER TO regress_partition_split_bob; +SET SESSION AUTHORIZATION regress_partition_split_bob; +ALTER TABLE t SPLIT PARTITION tp_0_2 INTO + (PARTITION tp_0_1 FOR VALUES FROM (0) TO (1), + PARTITION tp_1_2 FOR VALUES FROM (1) TO (2)); +RESET SESSION AUTHORIZATION; + +DROP TABLE t; +REVOKE ALL ON SCHEMA partition_split_schema FROM regress_partition_split_alice; +REVOKE ALL ON SCHEMA partition_split_schema FROM regress_partition_split_bob; +DROP ROLE regress_partition_split_alice; +DROP ROLE regress_partition_split_bob; + +-- Split partition of a temporary table when one of the partitions after +-- split has the same name as the partition being split +CREATE TEMP TABLE t (a int) PARTITION BY RANGE (a); +CREATE TEMP TABLE tp_0 PARTITION OF t FOR VALUES FROM (0) TO (2); +ALTER TABLE t SPLIT PARTITION tp_0 INTO + (PARTITION tp_0 FOR VALUES FROM (0) TO (1), + PARTITION tp_1 FOR VALUES FROM (1) TO (2)); +DROP TABLE t; + +RESET search_path; + +-- +DROP SCHEMA partition_split_schema; +DROP SCHEMA partition_split_schema2; diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list index ddc4c78fb9..16dcd62335 100644 --- a/src/tools/pgindent/typedefs.list +++ b/src/tools/pgindent/typedefs.list @@ -2723,6 +2723,7 @@ SpecialJoinInfo SpinDelayStatus SplitInterval SplitLR +SplitPartitionContext SplitPageLayout SplitPoint SplitTextOutputData -- 2.40.1.windows.1