mirror of
https://github.com/postgres/postgres.git
synced 2025-07-30 11:03:19 +03:00
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
This commit is contained in:
@ -670,6 +670,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);
|
||||
|
||||
@ -4740,6 +4743,10 @@ AlterTableGetLockLevel(List *cmds)
|
||||
cmd_lockmode = ShareUpdateExclusiveLock;
|
||||
break;
|
||||
|
||||
case AT_SplitPartition:
|
||||
cmd_lockmode = AccessExclusiveLock;
|
||||
break;
|
||||
|
||||
case AT_MergePartitions:
|
||||
cmd_lockmode = AccessExclusiveLock;
|
||||
break;
|
||||
@ -5163,6 +5170,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 */
|
||||
@ -5565,6 +5577,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);
|
||||
@ -6567,6 +6587,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:
|
||||
@ -20852,6 +20874,260 @@ GetAttributeStorage(Oid atttypid, const char *storagemode)
|
||||
return cstorage;
|
||||
}
|
||||
|
||||
/*
|
||||
* Struct with context of new partition for insert rows from splited partition
|
||||
*/
|
||||
typedef struct SplitPartitionContext
|
||||
{
|
||||
ExprState *partqualstate; /* expression for check slot for partition
|
||||
* (NULL for DEFAULT partition) */
|
||||
BulkInsertState bistate; /* state of bulk inserts for partition */
|
||||
TupleTableSlot *dstslot; /* slot for insert 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 Relation's.
|
||||
* 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;
|
||||
bool insert_indexes;
|
||||
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 for 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. We ignore the
|
||||
* 'insert_indexes' flag since newPartRel has no indexes anyway.
|
||||
*/
|
||||
(void) table_tuple_insert(pc->partRel, insertslot, mycid,
|
||||
ti_options, pc->bistate, &insert_indexes);
|
||||
|
||||
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 new partition with given name
|
||||
* (newPartName) like table (modelRelName)
|
||||
@ -20906,6 +21182,141 @@ createPartitionTable(RangeVar *newPartName, RangeVar *modelRelName,
|
||||
NULL);
|
||||
}
|
||||
|
||||
/*
|
||||
* ALTER TABLE <name> SPLIT PARTITION <partition-name> INTO <partition-list>
|
||||
*/
|
||||
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;
|
||||
RangeVar *parentName;
|
||||
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 prevent DML-queries to the partition.
|
||||
*/
|
||||
splitRel = table_openrv(cmd->name, AccessExclusiveLock);
|
||||
|
||||
if (splitRel->rd_rel->relkind != RELKIND_RELATION)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
|
||||
errmsg("cannot split non-table partition \"%s\"",
|
||||
RelationGetRelationName(splitRel))));
|
||||
|
||||
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.
|
||||
*/
|
||||
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 reuse name.
|
||||
*/
|
||||
if (isSameName)
|
||||
{
|
||||
/*
|
||||
* We must bump the command counter to make the split partition tuple
|
||||
* visible for rename.
|
||||
*/
|
||||
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 rename.
|
||||
*/
|
||||
CommandCounterIncrement();
|
||||
}
|
||||
|
||||
/* Create new partitions (like split partition), without indexes. */
|
||||
parentName = makeRangeVar(get_namespace_name(RelationGetNamespace(rel)),
|
||||
RelationGetRelationName(rel), -1);
|
||||
foreach(listptr, cmd->partlist)
|
||||
{
|
||||
SinglePartitionSpec *sps = (SinglePartitionSpec *) lfirst(listptr);
|
||||
Relation newPartRel;
|
||||
|
||||
createPartitionTable(sps->name, parentName, context);
|
||||
|
||||
/* Open the new partition and acquire exclusive lock on it. */
|
||||
newPartRel = table_openrv(sps->name, AccessExclusiveLock);
|
||||
|
||||
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 need. */
|
||||
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
|
||||
|
@ -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;
|
||||
@ -646,6 +647,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
|
||||
%type <partelem> part_elem
|
||||
%type <list> part_params
|
||||
%type <partboundspec> PartitionBoundSpec
|
||||
%type <singlepartspec> SinglePartitionSpec
|
||||
%type <list> partitions_list
|
||||
%type <list> hash_partbound
|
||||
%type <defelt> hash_partbound_elem
|
||||
|
||||
@ -777,7 +780,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
|
||||
|
||||
@ -2307,6 +2310,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 <name> ATTACH PARTITION <table_name> FOR VALUES */
|
||||
ATTACH PARTITION qualified_name PartitionBoundSpec
|
||||
@ -2351,6 +2371,20 @@ partition_cmd:
|
||||
n->def = (Node *) cmd;
|
||||
$$ = (Node *) n;
|
||||
}
|
||||
/* ALTER TABLE <name> SPLIT PARTITION <partition_name> 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 <name> MERGE PARTITIONS () INTO <partition_name> */
|
||||
| MERGE PARTITIONS '(' qualified_name_list ')' INTO qualified_name
|
||||
{
|
||||
@ -17756,6 +17790,7 @@ unreserved_keyword:
|
||||
| SKIP
|
||||
| SNAPSHOT
|
||||
| SOURCE
|
||||
| SPLIT
|
||||
| SQL_P
|
||||
| STABLE
|
||||
| STANDALONE_P
|
||||
@ -18393,6 +18428,7 @@ bare_label_keyword:
|
||||
| SNAPSHOT
|
||||
| SOME
|
||||
| SOURCE
|
||||
| SPLIT
|
||||
| SQL_P
|
||||
| STABLE
|
||||
| STANDALONE_P
|
||||
|
@ -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);
|
||||
@ -3415,6 +3415,43 @@ transformRuleStmt(RuleStmt *stmt, const char *queryString,
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* transformPartitionCmdForSplit
|
||||
* Analyze the ALTER TABLLE ... 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);
|
||||
|
||||
/* Then we should check partitions with transformed bounds. */
|
||||
check_partitions_for_split(parent, splitPartOid, partcmd->name, partcmd->partlist, cxt->pstate);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* transformPartitionCmdForMerge
|
||||
* Analyze the ALTER TABLLE ... MERGE PARTITIONS command
|
||||
@ -3751,7 +3788,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;
|
||||
}
|
||||
@ -3759,6 +3796,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
|
||||
newcmds = lappend(newcmds, cmd);
|
||||
break;
|
||||
|
||||
case AT_SplitPartition:
|
||||
case AT_MergePartitions:
|
||||
{
|
||||
PartitionCmd *partcmd = (PartitionCmd *) cmd->def;
|
||||
@ -3767,7 +3805,11 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
|
||||
errmsg("list of new partitions should contains at least two items")));
|
||||
transformPartitionCmdForMerge(&cxt, partcmd);
|
||||
|
||||
if (cmd->subtype == AT_SplitPartition)
|
||||
transformPartitionCmdForSplit(&cxt, partcmd);
|
||||
else
|
||||
transformPartitionCmdForMerge(&cxt, partcmd);
|
||||
newcmds = lappend(newcmds, cmd);
|
||||
break;
|
||||
}
|
||||
@ -4172,13 +4214,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;
|
||||
|
||||
@ -4187,9 +4229,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:
|
||||
|
||||
@ -4197,7 +4239,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",
|
||||
|
@ -5040,10 +5040,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)
|
||||
@ -5076,6 +5135,604 @@ 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" is 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 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);
|
||||
bool overlap = false;
|
||||
|
||||
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 && cmpval) || (defaultPart && cmpval < 0))
|
||||
overlap = true;
|
||||
}
|
||||
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 of "spec" should be equal (or less than or equal
|
||||
* in case defaultPart=true) to upper bound of split partition.
|
||||
*/
|
||||
if ((!defaultPart && cmpval) || (defaultPart && cmpval > 0))
|
||||
overlap = true;
|
||||
}
|
||||
|
||||
if (overlap)
|
||||
{
|
||||
PartitionRangeDatum *datum;
|
||||
|
||||
datum = list_nth(first ? spec->lowerdatums : spec->upperdatums, abs(cmpval) - 1);
|
||||
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
|
||||
errmsg("%s bound of partition \"%s\" is %s %s bound of split partition",
|
||||
first ? "lower" : "upper",
|
||||
relname,
|
||||
defaultPart ? (first ? "less than" : "greater than") : "not equals to",
|
||||
first ? "lower" : "upper"),
|
||||
parser_errposition(pstate, datum->location)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* check_partition_bounds_for_split_list
|
||||
*
|
||||
* (function for BY LIST partitioning)
|
||||
*
|
||||
* Checks that bounds of new partition is 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 it contains
|
||||
* split partition (partOid).
|
||||
*/
|
||||
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
|
||||
* partitionde 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 get 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 not have value %s but split partition has",
|
||||
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("any 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\" 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
|
||||
*
|
||||
|
@ -13241,3 +13241,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;
|
||||
}
|
||||
|
Reference in New Issue
Block a user