mirror of
https://github.com/postgres/postgres.git
synced 2025-07-30 11:03:19 +03:00
Implement ALTER TABLE ... MERGE PARTITIONS ... command
This new DDL command merges several partitions into the one partition of the target table. The target partition is created using new createPartitionTable() function with parent partition as the template. This commit comprises quite naive implementation which works in single process and holds the ACCESS EXCLUSIVE LOCK on the parent table during all the operations including the tuple routing. This is why this new DDL command can't be recommended for large partitioned tables under a high load. However, this implementation come in handy in certain cases even as is. Also, it could be used as a foundation for future implementations with lesser locking and possibly parallel. Discussion: https://postgr.es/m/c73a1746-0cd0-6bdd-6b23-3ae0b7c0c582%40postgrespro.ru Author: Dmitry Koval Reviewed-by: Matthias van de Meent, Laurenz Albe, Zhihong Yu, Justin Pryzby Reviewed-by: Alvaro Herrera, Robert Haas, Stephane Tachoires
This commit is contained in:
@ -670,6 +670,8 @@ static void ATDetachCheckNoForeignKeyRefs(Relation partition);
|
||||
static char GetAttributeCompression(Oid atttypid, const char *compression);
|
||||
static char GetAttributeStorage(Oid atttypid, const char *storagemode);
|
||||
|
||||
static void ATExecMergePartitions(List **wqueue, AlteredTableInfo *tab, Relation rel,
|
||||
PartitionCmd *cmd, AlterTableUtilityContext *context);
|
||||
|
||||
/* ----------------------------------------------------------------
|
||||
* DefineRelation
|
||||
@ -4738,6 +4740,10 @@ AlterTableGetLockLevel(List *cmds)
|
||||
cmd_lockmode = ShareUpdateExclusiveLock;
|
||||
break;
|
||||
|
||||
case AT_MergePartitions:
|
||||
cmd_lockmode = AccessExclusiveLock;
|
||||
break;
|
||||
|
||||
default: /* oops */
|
||||
elog(ERROR, "unrecognized alter table type: %d",
|
||||
(int) cmd->subtype);
|
||||
@ -5157,6 +5163,11 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
|
||||
/* No command-specific prep needed */
|
||||
pass = AT_PASS_MISC;
|
||||
break;
|
||||
case AT_MergePartitions:
|
||||
ATSimplePermissions(cmd->subtype, rel, ATT_TABLE);
|
||||
/* No command-specific prep needed */
|
||||
pass = AT_PASS_MISC;
|
||||
break;
|
||||
default: /* oops */
|
||||
elog(ERROR, "unrecognized alter table type: %d",
|
||||
(int) cmd->subtype);
|
||||
@ -5554,6 +5565,14 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab,
|
||||
case AT_DetachPartitionFinalize:
|
||||
address = ATExecDetachPartitionFinalize(rel, ((PartitionCmd *) cmd->def)->name);
|
||||
break;
|
||||
case AT_MergePartitions:
|
||||
cmd = ATParseTransformCmd(wqueue, tab, rel, cmd, false, lockmode,
|
||||
cur_pass, context);
|
||||
Assert(cmd != NULL);
|
||||
Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
|
||||
ATExecMergePartitions(wqueue, tab, rel, (PartitionCmd *) cmd->def,
|
||||
context);
|
||||
break;
|
||||
default: /* oops */
|
||||
elog(ERROR, "unrecognized alter table type: %d",
|
||||
(int) cmd->subtype);
|
||||
@ -6548,6 +6567,8 @@ alter_table_type_to_string(AlterTableType cmdtype)
|
||||
return "DETACH PARTITION";
|
||||
case AT_DetachPartitionFinalize:
|
||||
return "DETACH PARTITION ... FINALIZE";
|
||||
case AT_MergePartitions:
|
||||
return "MERGE PARTITIONS";
|
||||
case AT_AddIdentity:
|
||||
return "ALTER COLUMN ... ADD IDENTITY";
|
||||
case AT_SetIdentity:
|
||||
@ -19042,6 +19063,37 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* attachPartitionTable: attach new partition to partitioned table
|
||||
*
|
||||
* wqueue: the ALTER TABLE work queue; can be NULL when not running as part
|
||||
* of an ALTER TABLE sequence.
|
||||
* rel: partitioned relation;
|
||||
* attachrel: relation of attached partition;
|
||||
* bound: bounds of attached relation.
|
||||
*/
|
||||
static void
|
||||
attachPartitionTable(List **wqueue, Relation rel, Relation attachrel, PartitionBoundSpec *bound)
|
||||
{
|
||||
/* OK to create inheritance. Rest of the checks performed there */
|
||||
CreateInheritance(attachrel, rel, true);
|
||||
|
||||
/* Update the pg_class entry. */
|
||||
StorePartitionBound(attachrel, rel, bound);
|
||||
|
||||
/* Ensure there exists a correct set of indexes in the partition. */
|
||||
AttachPartitionEnsureIndexes(wqueue, rel, attachrel);
|
||||
|
||||
/* and triggers */
|
||||
CloneRowTriggersToPartition(rel, attachrel);
|
||||
|
||||
/*
|
||||
* Clone foreign key constraints. Callee is responsible for setting up
|
||||
* for phase 3 constraint verification.
|
||||
*/
|
||||
CloneForeignKeyConstraints(wqueue, rel, attachrel);
|
||||
}
|
||||
|
||||
/*
|
||||
* ALTER TABLE <name> ATTACH PARTITION <partition-name> FOR VALUES
|
||||
*
|
||||
@ -19244,23 +19296,8 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
|
||||
check_new_partition_bound(RelationGetRelationName(attachrel), rel,
|
||||
cmd->bound, pstate);
|
||||
|
||||
/* OK to create inheritance. Rest of the checks performed there */
|
||||
CreateInheritance(attachrel, rel, true);
|
||||
|
||||
/* Update the pg_class entry. */
|
||||
StorePartitionBound(attachrel, rel, cmd->bound);
|
||||
|
||||
/* Ensure there exists a correct set of indexes in the partition. */
|
||||
AttachPartitionEnsureIndexes(wqueue, rel, attachrel);
|
||||
|
||||
/* and triggers */
|
||||
CloneRowTriggersToPartition(rel, attachrel);
|
||||
|
||||
/*
|
||||
* Clone foreign key constraints. Callee is responsible for setting up
|
||||
* for phase 3 constraint verification.
|
||||
*/
|
||||
CloneForeignKeyConstraints(wqueue, rel, attachrel);
|
||||
/* Attach a new partition to the partitioned table. */
|
||||
attachPartitionTable(wqueue, rel, attachrel, cmd->bound);
|
||||
|
||||
/*
|
||||
* Generate partition constraint from the partition bound specification.
|
||||
@ -20814,3 +20851,286 @@ GetAttributeStorage(Oid atttypid, const char *storagemode)
|
||||
|
||||
return cstorage;
|
||||
}
|
||||
|
||||
/*
|
||||
* createPartitionTable: create table for new partition with given name
|
||||
* (newPartName) like table (modelRelName)
|
||||
*
|
||||
* Emulates command: CREATE TABLE <newPartName> (LIKE <modelRelName>
|
||||
* INCLUDING ALL EXCLUDING INDEXES EXCLUDING IDENTITY)
|
||||
*/
|
||||
static void
|
||||
createPartitionTable(RangeVar *newPartName, RangeVar *modelRelName,
|
||||
AlterTableUtilityContext *context)
|
||||
{
|
||||
CreateStmt *createStmt;
|
||||
TableLikeClause *tlc;
|
||||
PlannedStmt *wrapper;
|
||||
|
||||
createStmt = makeNode(CreateStmt);
|
||||
createStmt->relation = newPartName;
|
||||
createStmt->tableElts = NIL;
|
||||
createStmt->inhRelations = NIL;
|
||||
createStmt->constraints = NIL;
|
||||
createStmt->options = NIL;
|
||||
createStmt->oncommit = ONCOMMIT_NOOP;
|
||||
createStmt->tablespacename = NULL;
|
||||
createStmt->if_not_exists = false;
|
||||
|
||||
tlc = makeNode(TableLikeClause);
|
||||
tlc->relation = modelRelName;
|
||||
|
||||
/*
|
||||
* Indexes will be inherited on "attach new partitions" stage, after data
|
||||
* moving.
|
||||
*/
|
||||
tlc->options = CREATE_TABLE_LIKE_ALL & ~(CREATE_TABLE_LIKE_INDEXES | CREATE_TABLE_LIKE_IDENTITY);
|
||||
tlc->relationOid = InvalidOid;
|
||||
createStmt->tableElts = lappend(createStmt->tableElts, tlc);
|
||||
|
||||
/* Need to make a wrapper PlannedStmt. */
|
||||
wrapper = makeNode(PlannedStmt);
|
||||
wrapper->commandType = CMD_UTILITY;
|
||||
wrapper->canSetTag = false;
|
||||
wrapper->utilityStmt = (Node *) createStmt;
|
||||
wrapper->stmt_location = context->pstmt->stmt_location;
|
||||
wrapper->stmt_len = context->pstmt->stmt_len;
|
||||
|
||||
ProcessUtility(wrapper,
|
||||
context->queryString,
|
||||
false,
|
||||
PROCESS_UTILITY_SUBCOMMAND,
|
||||
NULL,
|
||||
NULL,
|
||||
None_Receiver,
|
||||
NULL);
|
||||
}
|
||||
|
||||
/*
|
||||
* moveMergedTablesRows: scan partitions to be merged (mergingPartitionsList)
|
||||
* of the partitioned table (rel) and move rows into the new partition
|
||||
* (newPartRel).
|
||||
*/
|
||||
static void
|
||||
moveMergedTablesRows(Relation rel, List *mergingPartitionsList,
|
||||
Relation newPartRel)
|
||||
{
|
||||
CommandId mycid;
|
||||
|
||||
/* The FSM is empty, so don't bother using it. */
|
||||
int ti_options = TABLE_INSERT_SKIP_FSM;
|
||||
ListCell *listptr;
|
||||
BulkInsertState bistate; /* state of bulk inserts for partition */
|
||||
TupleTableSlot *dstslot;
|
||||
|
||||
mycid = GetCurrentCommandId(true);
|
||||
|
||||
/* Prepare a BulkInsertState for table_tuple_insert. */
|
||||
bistate = GetBulkInsertState();
|
||||
|
||||
/* Create necessary tuple slot. */
|
||||
dstslot = MakeSingleTupleTableSlot(RelationGetDescr(newPartRel),
|
||||
table_slot_callbacks(newPartRel));
|
||||
ExecStoreAllNullTuple(dstslot);
|
||||
|
||||
foreach(listptr, mergingPartitionsList)
|
||||
{
|
||||
Relation mergingPartition = (Relation) lfirst(listptr);
|
||||
TupleTableSlot *srcslot;
|
||||
TupleConversionMap *tuple_map;
|
||||
TableScanDesc scan;
|
||||
Snapshot snapshot;
|
||||
|
||||
/* Create tuple slot for new partition. */
|
||||
srcslot = MakeSingleTupleTableSlot(RelationGetDescr(mergingPartition),
|
||||
table_slot_callbacks(mergingPartition));
|
||||
|
||||
/*
|
||||
* Map computing for moving attributes of merged partition to new
|
||||
* partition.
|
||||
*/
|
||||
tuple_map = convert_tuples_by_name(RelationGetDescr(mergingPartition),
|
||||
RelationGetDescr(newPartRel));
|
||||
|
||||
/* Scan through the rows. */
|
||||
snapshot = RegisterSnapshot(GetLatestSnapshot());
|
||||
scan = table_beginscan(mergingPartition, snapshot, 0, NULL);
|
||||
|
||||
while (table_scan_getnextslot(scan, ForwardScanDirection, srcslot))
|
||||
{
|
||||
TupleTableSlot *insertslot;
|
||||
bool insert_indexes;
|
||||
|
||||
/* Extract data from old tuple. */
|
||||
slot_getallattrs(srcslot);
|
||||
|
||||
if (tuple_map)
|
||||
{
|
||||
/* Need to use map for copy attributes. */
|
||||
insertslot = execute_attr_map_slot(tuple_map->attrMap, srcslot, dstslot);
|
||||
}
|
||||
else
|
||||
{
|
||||
/* Copy attributes directly. */
|
||||
insertslot = dstslot;
|
||||
|
||||
ExecClearTuple(insertslot);
|
||||
|
||||
memcpy(insertslot->tts_values, srcslot->tts_values,
|
||||
sizeof(Datum) * srcslot->tts_nvalid);
|
||||
memcpy(insertslot->tts_isnull, srcslot->tts_isnull,
|
||||
sizeof(bool) * srcslot->tts_nvalid);
|
||||
|
||||
ExecStoreVirtualTuple(insertslot);
|
||||
}
|
||||
|
||||
/*
|
||||
* Write the tuple out to the new relation. We ignore the
|
||||
* 'insert_indexes' flag since newPartRel has no indexes anyway.
|
||||
*/
|
||||
(void) table_tuple_insert(newPartRel, insertslot, mycid,
|
||||
ti_options, bistate, &insert_indexes);
|
||||
|
||||
CHECK_FOR_INTERRUPTS();
|
||||
}
|
||||
|
||||
table_endscan(scan);
|
||||
UnregisterSnapshot(snapshot);
|
||||
|
||||
if (tuple_map)
|
||||
free_conversion_map(tuple_map);
|
||||
|
||||
ExecDropSingleTupleTableSlot(srcslot);
|
||||
}
|
||||
|
||||
ExecDropSingleTupleTableSlot(dstslot);
|
||||
FreeBulkInsertState(bistate);
|
||||
|
||||
table_finish_bulk_insert(newPartRel, ti_options);
|
||||
}
|
||||
|
||||
/*
|
||||
* ALTER TABLE <name> MERGE PARTITIONS <partition-list> INTO <partition-name>
|
||||
*/
|
||||
static void
|
||||
ATExecMergePartitions(List **wqueue, AlteredTableInfo *tab, Relation rel,
|
||||
PartitionCmd *cmd, AlterTableUtilityContext *context)
|
||||
{
|
||||
Relation newPartRel;
|
||||
ListCell *listptr;
|
||||
List *mergingPartitionsList = NIL;
|
||||
Oid defaultPartOid;
|
||||
char tmpRelName[NAMEDATALEN];
|
||||
RangeVar *mergePartName = cmd->name;
|
||||
bool isSameName = false;
|
||||
|
||||
/*
|
||||
* Lock all merged partitions, check them and create list with partitions
|
||||
* contexts.
|
||||
*/
|
||||
foreach(listptr, cmd->partlist)
|
||||
{
|
||||
RangeVar *name = (RangeVar *) lfirst(listptr);
|
||||
Relation mergingPartition;
|
||||
|
||||
/*
|
||||
* We are going to detach and remove this partition: need to use
|
||||
* exclusive lock for prevent DML-queries to the partition.
|
||||
*/
|
||||
mergingPartition = table_openrv(name, AccessExclusiveLock);
|
||||
|
||||
if (mergingPartition->rd_rel->relkind != RELKIND_RELATION)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
|
||||
errmsg("cannot merge non-table partition \"%s\"",
|
||||
RelationGetRelationName(mergingPartition))));
|
||||
|
||||
/*
|
||||
* Checking that two partitions have the same name was before, in
|
||||
* function transformPartitionCmdForMerge().
|
||||
*/
|
||||
if (equal(name, cmd->name))
|
||||
/* One new partition can have the same name as merged partition. */
|
||||
isSameName = true;
|
||||
|
||||
/* Store a next merging partition into the list. */
|
||||
mergingPartitionsList = lappend(mergingPartitionsList,
|
||||
mergingPartition);
|
||||
}
|
||||
|
||||
/* Detach all merged partitions. */
|
||||
defaultPartOid =
|
||||
get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, true));
|
||||
foreach(listptr, mergingPartitionsList)
|
||||
{
|
||||
Relation mergingPartition = (Relation) lfirst(listptr);
|
||||
|
||||
/* Remove the pg_inherits row first. */
|
||||
RemoveInheritance(mergingPartition, rel, false);
|
||||
/* Do the final part of detaching. */
|
||||
DetachPartitionFinalize(rel, mergingPartition, false, defaultPartOid);
|
||||
}
|
||||
|
||||
/* Create table for new partition, use partitioned table as model. */
|
||||
if (isSameName)
|
||||
{
|
||||
/* Create partition table with generated temparary name. */
|
||||
sprintf(tmpRelName, "merge-%u-%X-tmp", RelationGetRelid(rel), MyProcPid);
|
||||
mergePartName = makeRangeVar(get_namespace_name(RelationGetNamespace(rel)),
|
||||
tmpRelName, -1);
|
||||
}
|
||||
createPartitionTable(mergePartName,
|
||||
makeRangeVar(get_namespace_name(RelationGetNamespace(rel)),
|
||||
RelationGetRelationName(rel), -1),
|
||||
context);
|
||||
|
||||
/*
|
||||
* Open the new partition and acquire exclusive lock on it. This will
|
||||
* stop all the operations with partitioned table. This might seem
|
||||
* excessive, but this is the way we make sure nobody is planning queries
|
||||
* involving merging partitions.
|
||||
*/
|
||||
newPartRel = table_openrv(mergePartName, AccessExclusiveLock);
|
||||
|
||||
/* Copy data from merged partitions to new partition. */
|
||||
moveMergedTablesRows(rel, mergingPartitionsList, newPartRel);
|
||||
|
||||
/*
|
||||
* Attach a new partition to the partitioned table. wqueue = NULL:
|
||||
* verification for each cloned constraint is not need.
|
||||
*/
|
||||
attachPartitionTable(NULL, rel, newPartRel, cmd->bound);
|
||||
|
||||
/* Unlock and drop merged partitions. */
|
||||
foreach(listptr, mergingPartitionsList)
|
||||
{
|
||||
ObjectAddress object;
|
||||
Relation mergingPartition = (Relation) lfirst(listptr);
|
||||
|
||||
/* Get relation id before table_close() call. */
|
||||
object.objectId = RelationGetRelid(mergingPartition);
|
||||
object.classId = RelationRelationId;
|
||||
object.objectSubId = 0;
|
||||
|
||||
/* Keep the lock until commit. */
|
||||
table_close(mergingPartition, NoLock);
|
||||
|
||||
performDeletion(&object, DROP_RESTRICT, 0);
|
||||
}
|
||||
list_free(mergingPartitionsList);
|
||||
|
||||
/* Rename new partition if it is needed. */
|
||||
if (isSameName)
|
||||
{
|
||||
/*
|
||||
* We must bump the command counter to make the new partition tuple
|
||||
* visible for rename.
|
||||
*/
|
||||
CommandCounterIncrement();
|
||||
/* Rename partition. */
|
||||
RenameRelationInternal(RelationGetRelid(newPartRel),
|
||||
cmd->name->relname, false, false);
|
||||
}
|
||||
/* Keep the lock until commit. */
|
||||
table_close(newPartRel, NoLock);
|
||||
}
|
||||
|
@ -761,7 +761,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
|
||||
ORDER ORDINALITY OTHERS OUT_P OUTER_P
|
||||
OVER OVERLAPS OVERLAY OVERRIDING OWNED OWNER
|
||||
|
||||
PARALLEL PARAMETER PARSER PARTIAL PARTITION PASSING PASSWORD PATH
|
||||
PARALLEL PARAMETER PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PATH
|
||||
PERIOD PLACING PLAN PLANS POLICY
|
||||
|
||||
POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY
|
||||
@ -2317,6 +2317,7 @@ partition_cmd:
|
||||
n->subtype = AT_AttachPartition;
|
||||
cmd->name = $3;
|
||||
cmd->bound = $4;
|
||||
cmd->partlist = NULL;
|
||||
cmd->concurrent = false;
|
||||
n->def = (Node *) cmd;
|
||||
|
||||
@ -2331,6 +2332,7 @@ partition_cmd:
|
||||
n->subtype = AT_DetachPartition;
|
||||
cmd->name = $3;
|
||||
cmd->bound = NULL;
|
||||
cmd->partlist = NULL;
|
||||
cmd->concurrent = $4;
|
||||
n->def = (Node *) cmd;
|
||||
|
||||
@ -2344,6 +2346,21 @@ partition_cmd:
|
||||
n->subtype = AT_DetachPartitionFinalize;
|
||||
cmd->name = $3;
|
||||
cmd->bound = NULL;
|
||||
cmd->partlist = NULL;
|
||||
cmd->concurrent = false;
|
||||
n->def = (Node *) cmd;
|
||||
$$ = (Node *) n;
|
||||
}
|
||||
/* ALTER TABLE <name> MERGE PARTITIONS () INTO <partition_name> */
|
||||
| MERGE PARTITIONS '(' qualified_name_list ')' INTO qualified_name
|
||||
{
|
||||
AlterTableCmd *n = makeNode(AlterTableCmd);
|
||||
PartitionCmd *cmd = makeNode(PartitionCmd);
|
||||
|
||||
n->subtype = AT_MergePartitions;
|
||||
cmd->name = $7;
|
||||
cmd->bound = NULL;
|
||||
cmd->partlist = $4;
|
||||
cmd->concurrent = false;
|
||||
n->def = (Node *) cmd;
|
||||
$$ = (Node *) n;
|
||||
@ -2360,6 +2377,7 @@ index_partition_cmd:
|
||||
n->subtype = AT_AttachPartition;
|
||||
cmd->name = $3;
|
||||
cmd->bound = NULL;
|
||||
cmd->partlist = NULL;
|
||||
cmd->concurrent = false;
|
||||
n->def = (Node *) cmd;
|
||||
|
||||
@ -17668,6 +17686,7 @@ unreserved_keyword:
|
||||
| PARSER
|
||||
| PARTIAL
|
||||
| PARTITION
|
||||
| PARTITIONS
|
||||
| PASSING
|
||||
| PASSWORD
|
||||
| PATH
|
||||
@ -18292,6 +18311,7 @@ bare_label_keyword:
|
||||
| PARSER
|
||||
| PARTIAL
|
||||
| PARTITION
|
||||
| PARTITIONS
|
||||
| PASSING
|
||||
| PASSWORD
|
||||
| PATH
|
||||
|
@ -58,6 +58,8 @@
|
||||
#include "parser/parse_type.h"
|
||||
#include "parser/parse_utilcmd.h"
|
||||
#include "parser/parser.h"
|
||||
#include "partitioning/partdesc.h"
|
||||
#include "partitioning/partbounds.h"
|
||||
#include "rewrite/rewriteManip.h"
|
||||
#include "utils/acl.h"
|
||||
#include "utils/builtins.h"
|
||||
@ -3413,6 +3415,80 @@ transformRuleStmt(RuleStmt *stmt, const char *queryString,
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* transformPartitionCmdForMerge
|
||||
* Analyze the ALTER TABLLE ... MERGE PARTITIONS command
|
||||
*
|
||||
* Does simple checks for merged partitions. Calculates bound of result
|
||||
* partition.
|
||||
*/
|
||||
static void
|
||||
transformPartitionCmdForMerge(CreateStmtContext *cxt, PartitionCmd *partcmd)
|
||||
{
|
||||
Oid defaultPartOid;
|
||||
Oid partOid;
|
||||
Relation parent = cxt->rel;
|
||||
PartitionKey key;
|
||||
char strategy;
|
||||
ListCell *listptr,
|
||||
*listptr2;
|
||||
bool isDefaultPart = false;
|
||||
List *partOids = NIL;
|
||||
|
||||
if (parent->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
|
||||
errmsg("\"%s\" is not a partitioned table", RelationGetRelationName(parent))));
|
||||
|
||||
key = RelationGetPartitionKey(parent);
|
||||
strategy = get_partition_strategy(key);
|
||||
|
||||
if (strategy == PARTITION_STRATEGY_HASH)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("partition of hash-partitioned table cannot be merged")));
|
||||
|
||||
/* Is current partition a DEFAULT partition? */
|
||||
defaultPartOid = get_default_oid_from_partdesc(
|
||||
RelationGetPartitionDesc(parent, true));
|
||||
|
||||
foreach(listptr, partcmd->partlist)
|
||||
{
|
||||
RangeVar *name = (RangeVar *) lfirst(listptr);
|
||||
|
||||
/* Partitions in the list should have different names. */
|
||||
for_each_cell(listptr2, partcmd->partlist, lnext(partcmd->partlist, listptr))
|
||||
{
|
||||
RangeVar *name2 = (RangeVar *) lfirst(listptr2);
|
||||
|
||||
if (equal(name, name2))
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_DUPLICATE_TABLE),
|
||||
errmsg("partition with name \"%s\" already used", name->relname)),
|
||||
parser_errposition(cxt->pstate, name2->location));
|
||||
}
|
||||
|
||||
/* Search DEFAULT partition in the list. */
|
||||
partOid = RangeVarGetRelid(name, NoLock, false);
|
||||
if (partOid == defaultPartOid)
|
||||
isDefaultPart = true;
|
||||
partOids = lappend_oid(partOids, partOid);
|
||||
}
|
||||
|
||||
/* Allocate bound of result partition. */
|
||||
Assert(partcmd->bound == NULL);
|
||||
partcmd->bound = makeNode(PartitionBoundSpec);
|
||||
|
||||
/* Fill partition bound. */
|
||||
partcmd->bound->strategy = strategy;
|
||||
partcmd->bound->location = -1;
|
||||
partcmd->bound->is_default = isDefaultPart;
|
||||
if (!isDefaultPart)
|
||||
calculate_partition_bound_for_merge(parent, partcmd->partlist,
|
||||
partOids, partcmd->bound,
|
||||
cxt->pstate);
|
||||
}
|
||||
|
||||
/*
|
||||
* transformAlterTableStmt -
|
||||
* parse analysis for ALTER TABLE
|
||||
@ -3683,6 +3759,19 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
|
||||
newcmds = lappend(newcmds, cmd);
|
||||
break;
|
||||
|
||||
case AT_MergePartitions:
|
||||
{
|
||||
PartitionCmd *partcmd = (PartitionCmd *) cmd->def;
|
||||
|
||||
if (list_length(partcmd->partlist) < 2)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
|
||||
errmsg("list of new partitions should contains at least two items")));
|
||||
transformPartitionCmdForMerge(&cxt, partcmd);
|
||||
newcmds = lappend(newcmds, cmd);
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
|
||||
/*
|
||||
|
@ -4977,3 +4977,210 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
|
||||
|
||||
PG_RETURN_BOOL(rowHash % modulus == remainder);
|
||||
}
|
||||
|
||||
/*
|
||||
* check_two_partitions_bounds_range
|
||||
*
|
||||
* (function for BY RANGE partitioning)
|
||||
*
|
||||
* This is a helper function for check_partitions_for_split() and
|
||||
* calculate_partition_bound_for_merge().
|
||||
* This function compares upper bound of first_bound and lower bound of
|
||||
* second_bound. These bounds should be equals except case
|
||||
* "defaultPart == true" (this means that one of split partitions is DEFAULT).
|
||||
* In this case upper bound of first_bound can be less than lower bound of
|
||||
* second_bound because space between of these bounds will be included in
|
||||
* DEFAULT partition.
|
||||
*
|
||||
* parent: partitioned table
|
||||
* first_name: name of first partition
|
||||
* first_bound: bound of first partition
|
||||
* second_name: name of second partition
|
||||
* second_bound: bound of second partition
|
||||
* defaultPart: true if one of split partitions is DEFAULT
|
||||
* pstate: pointer to ParseState struct for determine error position
|
||||
*/
|
||||
static void
|
||||
check_two_partitions_bounds_range(Relation parent,
|
||||
RangeVar *first_name,
|
||||
PartitionBoundSpec *first_bound,
|
||||
RangeVar *second_name,
|
||||
PartitionBoundSpec *second_bound,
|
||||
bool defaultPart,
|
||||
ParseState *pstate)
|
||||
{
|
||||
PartitionKey key = RelationGetPartitionKey(parent);
|
||||
PartitionRangeBound *first_upper;
|
||||
PartitionRangeBound *second_lower;
|
||||
int cmpval;
|
||||
|
||||
Assert(key->strategy == PARTITION_STRATEGY_RANGE);
|
||||
|
||||
first_upper = make_one_partition_rbound(key, -1, first_bound->upperdatums, false);
|
||||
second_lower = make_one_partition_rbound(key, -1, second_bound->lowerdatums, true);
|
||||
|
||||
/*
|
||||
* lower1=false (the second to last argument) for correct comparison lower
|
||||
* and upper bounds.
|
||||
*/
|
||||
cmpval = partition_rbound_cmp(key->partnatts,
|
||||
key->partsupfunc,
|
||||
key->partcollation,
|
||||
second_lower->datums, second_lower->kind,
|
||||
false, first_upper);
|
||||
if ((!defaultPart && cmpval) || (defaultPart && cmpval < 0))
|
||||
{
|
||||
PartitionRangeDatum *datum = linitial(second_bound->lowerdatums);
|
||||
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
|
||||
errmsg("lower bound of partition \"%s\" conflicts with upper bound of previous partition \"%s\"",
|
||||
second_name->relname, first_name->relname),
|
||||
parser_errposition(pstate, datum->location)));
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* get_partition_bound_spec
|
||||
*
|
||||
* Returns description of partition with Oid "partOid" and name "name".
|
||||
*/
|
||||
static PartitionBoundSpec *
|
||||
get_partition_bound_spec(Oid partOid, RangeVar *name)
|
||||
{
|
||||
HeapTuple tuple;
|
||||
Datum datum;
|
||||
bool isnull;
|
||||
PartitionBoundSpec *boundspec = NULL;
|
||||
|
||||
/* Try fetching the tuple from the catcache, for speed. */
|
||||
tuple = SearchSysCache1(RELOID, partOid);
|
||||
if (!HeapTupleIsValid(tuple))
|
||||
elog(ERROR, "cache lookup failed for relation \"%s\"",
|
||||
name->relname);
|
||||
|
||||
datum = SysCacheGetAttr(RELOID, tuple,
|
||||
Anum_pg_class_relpartbound,
|
||||
&isnull);
|
||||
if (isnull)
|
||||
elog(ERROR, "partition bound for relation \"%s\" is null",
|
||||
name->relname);
|
||||
|
||||
boundspec = stringToNode(TextDatumGetCString(datum));
|
||||
|
||||
if (!IsA(boundspec, PartitionBoundSpec))
|
||||
elog(ERROR, "expected PartitionBoundSpec for relation \"%s\"",
|
||||
name->relname);
|
||||
|
||||
ReleaseSysCache(tuple);
|
||||
return boundspec;
|
||||
}
|
||||
|
||||
/*
|
||||
* calculate_partition_bound_for_merge
|
||||
*
|
||||
* Calculates the bound of merged partition "spec" by using the bounds of
|
||||
* partitions to be merged.
|
||||
*
|
||||
* parent: partitioned table
|
||||
* partNames: names of partitions to be merged
|
||||
* partOids: Oids of partitions to be merged
|
||||
* spec (out): bounds specification of the merged partition
|
||||
* pstate: pointer to ParseState struct for determine error position
|
||||
*/
|
||||
void
|
||||
calculate_partition_bound_for_merge(Relation parent,
|
||||
List *partNames,
|
||||
List *partOids,
|
||||
PartitionBoundSpec *spec,
|
||||
ParseState *pstate)
|
||||
{
|
||||
PartitionKey key = RelationGetPartitionKey(parent);
|
||||
PartitionBoundSpec *bound;
|
||||
|
||||
Assert(!spec->is_default);
|
||||
|
||||
switch (key->strategy)
|
||||
{
|
||||
case PARTITION_STRATEGY_RANGE:
|
||||
{
|
||||
int i;
|
||||
PartitionRangeBound **lower_bounds;
|
||||
int nparts = list_length(partOids);
|
||||
List *bounds = NIL;
|
||||
|
||||
lower_bounds = (PartitionRangeBound **)
|
||||
palloc0(nparts * sizeof(PartitionRangeBound *));
|
||||
|
||||
/*
|
||||
* Create array of lower bounds and list of
|
||||
* PartitionBoundSpec.
|
||||
*/
|
||||
for (i = 0; i < nparts; i++)
|
||||
{
|
||||
bound = get_partition_bound_spec(list_nth_oid(partOids, i),
|
||||
(RangeVar *) list_nth(partNames, i));
|
||||
|
||||
lower_bounds[i] = make_one_partition_rbound(key, i, bound->lowerdatums, true);
|
||||
bounds = lappend(bounds, bound);
|
||||
}
|
||||
|
||||
/* Sort array of lower bounds. */
|
||||
qsort_arg(lower_bounds, nparts, sizeof(PartitionRangeBound *),
|
||||
qsort_partition_rbound_cmp, (void *) key);
|
||||
|
||||
/* Ranges of partitions should not overlap. */
|
||||
for (i = 1; i < nparts; i++)
|
||||
{
|
||||
int index = lower_bounds[i]->index;
|
||||
int prev_index = lower_bounds[i - 1]->index;
|
||||
|
||||
check_two_partitions_bounds_range(parent,
|
||||
(RangeVar *) list_nth(partNames, prev_index),
|
||||
(PartitionBoundSpec *) list_nth(bounds, prev_index),
|
||||
(RangeVar *) list_nth(partNames, index),
|
||||
(PartitionBoundSpec *) list_nth(bounds, index),
|
||||
false, pstate);
|
||||
}
|
||||
|
||||
/*
|
||||
* Lower bound of first partition is a lower bound of merged
|
||||
* partition.
|
||||
*/
|
||||
spec->lowerdatums =
|
||||
((PartitionBoundSpec *) list_nth(bounds, lower_bounds[0]->index))->lowerdatums;
|
||||
|
||||
/*
|
||||
* Upper bound of last partition is a upper bound of merged
|
||||
* partition.
|
||||
*/
|
||||
spec->upperdatums =
|
||||
((PartitionBoundSpec *) list_nth(bounds, lower_bounds[nparts - 1]->index))->upperdatums;
|
||||
|
||||
pfree(lower_bounds);
|
||||
list_free(bounds);
|
||||
break;
|
||||
}
|
||||
|
||||
case PARTITION_STRATEGY_LIST:
|
||||
{
|
||||
ListCell *listptr,
|
||||
*listptr2;
|
||||
|
||||
/* Consolidate bounds for all partitions in the list. */
|
||||
forboth(listptr, partOids, listptr2, partNames)
|
||||
{
|
||||
RangeVar *name = (RangeVar *) lfirst(listptr2);
|
||||
Oid curOid = lfirst_oid(listptr);
|
||||
|
||||
bound = get_partition_bound_spec(curOid, name);
|
||||
spec->listdatums = list_concat(spec->listdatums, bound->listdatums);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
elog(ERROR, "unexpected partition strategy: %d",
|
||||
(int) key->strategy);
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user