aboutsummaryrefslogtreecommitdiff
path: root/src/backend
diff options
context:
space:
mode:
authorAlexander Korotkov <akorotkov@postgresql.org>2024-08-24 18:48:48 +0300
committerAlexander Korotkov <akorotkov@postgresql.org>2024-08-24 18:48:48 +0300
commit3890d90c1508125729ed20038d90513694fc3a7b (patch)
tree5c0ed62151685aed09e62e3d48524d346c5080a7 /src/backend
parent6e8a0317b4c062d4d524b916e10ca7f351ed0793 (diff)
downloadpostgresql-3890d90c1508125729ed20038d90513694fc3a7b.tar.gz
postgresql-3890d90c1508125729ed20038d90513694fc3a7b.zip
Revert support for ALTER TABLE ... MERGE/SPLIT PARTITION(S) commands
This commit reverts 1adf16b8fb, 87c21bb941, and subsequent fixes and improvements including df64c81ca9, c99ef1811a, 9dfcac8e15, 885742b9f8, 842c9b2705, fcf80c5d5f, 96c7381c4c, f4fc7cb54b, 60ae37a8bc, 259c96fa8f, 449cdcd486, 3ca43dbbb6, 2a679ae94e, 3a82c689fd, fbd4321fd5, d53a4286d7, c086896625, 4e5d6c4091, 04158e7fa3. The reason for reverting is security issues related to repeatable name lookups (CVE-2014-0062). Even though 04158e7fa3 solved part of the problem, there are still remaining issues, which aren't feasible to even carefully analyze before the RC deadline. Reported-by: Noah Misch, Robert Haas Discussion: https://postgr.es/m/20240808171351.a9.nmisch%40google.com Backpatch-through: 17
Diffstat (limited to 'src/backend')
-rw-r--r--src/backend/commands/tablecmds.c819
-rw-r--r--src/backend/parser/gram.y61
-rw-r--r--src/backend/parser/parse_utilcmd.c193
-rw-r--r--src/backend/partitioning/partbounds.c901
-rw-r--r--src/backend/tcop/utility.c6
-rw-r--r--src/backend/utils/adt/ruleutils.c18
6 files changed, 30 insertions, 1968 deletions
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 52ce6b0c92a..dac39df83ac 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -657,11 +657,6 @@ 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);
/* ----------------------------------------------------------------
* DefineRelation
@@ -4672,14 +4667,6 @@ AlterTableGetLockLevel(List *cmds)
cmd_lockmode = ShareUpdateExclusiveLock;
break;
- case AT_SplitPartition:
- cmd_lockmode = AccessExclusiveLock;
- break;
-
- case AT_MergePartitions:
- cmd_lockmode = AccessExclusiveLock;
- break;
-
case AT_CheckNotNull:
/*
@@ -5106,16 +5093,6 @@ 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 */
- pass = AT_PASS_MISC;
- break;
default: /* oops */
elog(ERROR, "unrecognized alter table type: %d",
(int) cmd->subtype);
@@ -5512,22 +5489,6 @@ 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);
- 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);
@@ -6516,10 +6477,6 @@ 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:
return "ALTER COLUMN ... ADD IDENTITY";
case AT_SetIdentity:
@@ -18329,37 +18286,6 @@ QueuePartitionConstraintValidation(List **wqueue, Relation scanrel,
}
/*
- * attachPartitionTable: attach a new partition to the partitioned table
- *
- * wqueue: the ALTER TABLE work queue; can be NULL when not running as part
- * of an ALTER TABLE sequence.
- * rel: partitioned relation;
- * attachrel: relation of attached partition;
- * bound: bounds of attached relation.
- */
-static void
-attachPartitionTable(List **wqueue, Relation rel, Relation attachrel, PartitionBoundSpec *bound)
-{
- /* OK to create inheritance. Rest of the checks performed there */
- CreateInheritance(attachrel, rel, true);
-
- /* Update the pg_class entry. */
- StorePartitionBound(attachrel, rel, bound);
-
- /* Ensure there exists a correct set of indexes in the partition. */
- AttachPartitionEnsureIndexes(wqueue, rel, attachrel);
-
- /* and triggers */
- CloneRowTriggersToPartition(rel, attachrel);
-
- /*
- * Clone foreign key constraints. Callee is responsible for setting up
- * for phase 3 constraint verification.
- */
- CloneForeignKeyConstraints(wqueue, rel, attachrel);
-}
-
-/*
* ALTER TABLE <name> ATTACH PARTITION <partition-name> FOR VALUES
*
* Return the address of the newly attached partition.
@@ -18561,8 +18487,23 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd,
check_new_partition_bound(RelationGetRelationName(attachrel), rel,
cmd->bound, pstate);
- /* Attach a new partition to the partitioned table. */
- attachPartitionTable(wqueue, rel, attachrel, cmd->bound);
+ /* 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);
/*
* Generate partition constraint from the partition bound specification.
@@ -20077,729 +20018,3 @@ 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)
- *
- * Emulates command: CREATE [TEMP] TABLE <newPartName> (LIKE <modelRel's name>
- * INCLUDING ALL EXCLUDING INDEXES EXCLUDING IDENTITY EXCLUDING STATISTICS)
- *
- * Also, this function sets the new partition access method same as parent
- * table access methods (similarly to CREATE TABLE ... PARTITION OF). It
- * checks that parent and child tables have compatible persistence.
- *
- * Function returns the created relation (locked in AccessExclusiveLock mode).
- */
-static Relation
-createPartitionTable(RangeVar *newPartName, Relation modelRel,
- AlterTableUtilityContext *context)
-{
- CreateStmt *createStmt;
- TableLikeClause *tlc;
- PlannedStmt *wrapper;
- Relation newRel;
-
- /* If existing rel is temp, it must belong to this session */
- if (modelRel->rd_rel->relpersistence == RELPERSISTENCE_TEMP &&
- !modelRel->rd_islocaltemp)
- ereport(ERROR,
- (errcode(ERRCODE_WRONG_OBJECT_TYPE),
- errmsg("cannot create as partition of temporary relation of another session")));
-
- /* New partition should have the same persistence as modelRel */
- newPartName->relpersistence = modelRel->rd_rel->relpersistence;
-
- createStmt = makeNode(CreateStmt);
- createStmt->relation = newPartName;
- createStmt->tableElts = NIL;
- createStmt->inhRelations = NIL;
- createStmt->constraints = NIL;
- createStmt->options = NIL;
- createStmt->oncommit = ONCOMMIT_NOOP;
- createStmt->tablespacename = get_tablespace_name(modelRel->rd_rel->reltablespace);
- createStmt->if_not_exists = false;
- createStmt->accessMethod = get_am_name(modelRel->rd_rel->relam);
-
- tlc = makeNode(TableLikeClause);
- tlc->relation = makeRangeVar(get_namespace_name(RelationGetNamespace(modelRel)),
- RelationGetRelationName(modelRel), -1);
-
- /*
- * Indexes will be inherited on "attach new partitions" stage, after data
- * moving. We also don't copy the extended statistics for consistency
- * with CREATE TABLE PARTITION OF.
- */
- tlc->options = CREATE_TABLE_LIKE_ALL &
- ~(CREATE_TABLE_LIKE_INDEXES | CREATE_TABLE_LIKE_IDENTITY | CREATE_TABLE_LIKE_STATISTICS);
- tlc->relationOid = InvalidOid;
- tlc->newRelationOid = InvalidOid;
- createStmt->tableElts = lappend(createStmt->tableElts, tlc);
-
- /* Need to make a wrapper PlannedStmt. */
- wrapper = makeNode(PlannedStmt);
- wrapper->commandType = CMD_UTILITY;
- wrapper->canSetTag = false;
- wrapper->utilityStmt = (Node *) createStmt;
- wrapper->stmt_location = context->pstmt->stmt_location;
- wrapper->stmt_len = context->pstmt->stmt_len;
-
- ProcessUtility(wrapper,
- context->queryString,
- false,
- PROCESS_UTILITY_SUBCOMMAND,
- NULL,
- NULL,
- None_Receiver,
- NULL);
-
- /*
- * Open the new partition with no lock, because we already have
- * AccessExclusiveLock placed there after creation.
- */
- newRel = table_open(tlc->newRelationOid, NoLock);
-
- /*
- * We intended to create the partition with the same persistence as the
- * parent table, but we still need to recheck because that might be
- * affected by the search_path. If the parent is permanent, so must be
- * all of its partitions.
- */
- if (modelRel->rd_rel->relpersistence != RELPERSISTENCE_TEMP &&
- newRel->rd_rel->relpersistence == RELPERSISTENCE_TEMP)
- ereport(ERROR,
- (errcode(ERRCODE_WRONG_OBJECT_TYPE),
- errmsg("cannot create a temporary relation as partition of permanent relation \"%s\"",
- RelationGetRelationName(modelRel))));
-
- /* Permanent rels cannot be partitions belonging to temporary parent */
- if (newRel->rd_rel->relpersistence != RELPERSISTENCE_TEMP &&
- modelRel->rd_rel->relpersistence == RELPERSISTENCE_TEMP)
- ereport(ERROR,
- (errcode(ERRCODE_WRONG_OBJECT_TYPE),
- errmsg("cannot create a permanent relation as partition of temporary relation \"%s\"",
- RelationGetRelationName(modelRel))));
-
- return newRel;
-}
-
-/*
- * 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;
- 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
- * (newPartRel).
- */
-static void
-moveMergedTablesRows(Relation rel, List *mergingPartitionsList,
- Relation newPartRel)
-{
- CommandId mycid;
-
- /* The FSM is empty, so don't bother using it. */
- int ti_options = TABLE_INSERT_SKIP_FSM;
- ListCell *listptr;
- BulkInsertState bistate; /* state of bulk inserts for partition */
- TupleTableSlot *dstslot;
-
- mycid = GetCurrentCommandId(true);
-
- /* Prepare a BulkInsertState for table_tuple_insert. */
- bistate = GetBulkInsertState();
-
- /* Create necessary tuple slot. */
- dstslot = MakeSingleTupleTableSlot(RelationGetDescr(newPartRel),
- table_slot_callbacks(newPartRel));
- ExecStoreAllNullTuple(dstslot);
-
- foreach(listptr, mergingPartitionsList)
- {
- Relation mergingPartition = (Relation) lfirst(listptr);
- TupleTableSlot *srcslot;
- TupleConversionMap *tuple_map;
- TableScanDesc scan;
- Snapshot snapshot;
-
- /* Create tuple slot for new partition. */
- srcslot = MakeSingleTupleTableSlot(RelationGetDescr(mergingPartition),
- table_slot_callbacks(mergingPartition));
-
- /*
- * Map computing for moving attributes of merged partition to new
- * partition.
- */
- tuple_map = convert_tuples_by_name(RelationGetDescr(mergingPartition),
- RelationGetDescr(newPartRel));
-
- /* Scan through the rows. */
- snapshot = RegisterSnapshot(GetLatestSnapshot());
- scan = table_beginscan(mergingPartition, snapshot, 0, NULL);
-
- while (table_scan_getnextslot(scan, ForwardScanDirection, srcslot))
- {
- TupleTableSlot *insertslot;
-
- /* Extract data from old tuple. */
- slot_getallattrs(srcslot);
-
- if (tuple_map)
- {
- /* Need to use map to copy attributes. */
- insertslot = execute_attr_map_slot(tuple_map->attrMap, srcslot, dstslot);
- }
- else
- {
- /* Copy attributes directly. */
- insertslot = dstslot;
-
- ExecClearTuple(insertslot);
-
- memcpy(insertslot->tts_values, srcslot->tts_values,
- sizeof(Datum) * srcslot->tts_nvalid);
- memcpy(insertslot->tts_isnull, srcslot->tts_isnull,
- sizeof(bool) * srcslot->tts_nvalid);
-
- ExecStoreVirtualTuple(insertslot);
- }
-
- /* Write the tuple out to the new relation. */
- table_tuple_insert(newPartRel, insertslot, mycid,
- ti_options, bistate);
-
- CHECK_FOR_INTERRUPTS();
- }
-
- table_endscan(scan);
- UnregisterSnapshot(snapshot);
-
- if (tuple_map)
- free_conversion_map(tuple_map);
-
- ExecDropSingleTupleTableSlot(srcslot);
- }
-
- ExecDropSingleTupleTableSlot(dstslot);
- FreeBulkInsertState(bistate);
-
- table_finish_bulk_insert(newPartRel, ti_options);
-}
-
-/*
- * ALTER TABLE <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;
- Oid namespaceId;
- Oid existingRelid;
-
- /*
- * Lock all merged partitions, check them and create list with partitions
- * contexts.
- */
- foreach(listptr, cmd->partlist)
- {
- RangeVar *name = (RangeVar *) lfirst(listptr);
- Relation mergingPartition;
-
- /*
- * We are going to detach and remove this partition: need to use
- * exclusive lock for preventing DML-queries to the partition.
- */
- mergingPartition = table_openrv(name, AccessExclusiveLock);
-
- /* Store a next merging partition into the list. */
- mergingPartitionsList = lappend(mergingPartitionsList,
- mergingPartition);
- }
-
- /*
- * Look up the namespace in which we are supposed to create the partition,
- * check we have permission to create there, lock it against concurrent
- * drop, and mark stmt->relation as RELPERSISTENCE_TEMP if a temporary
- * namespace is selected.
- */
- cmd->name->relpersistence = rel->rd_rel->relpersistence;
- namespaceId =
- RangeVarGetAndCheckCreationNamespace(cmd->name, NoLock, NULL);
-
- /*
- * Check if this name is already taken. This helps us to detect the
- * situation when one of the merging partitions has the same name as the
- * new partition. Otherwise, this would fail later on anyway but catching
- * this here allows us to emit a nicer error message.
- */
- existingRelid = get_relname_relid(cmd->name->relname, namespaceId);
-
- if (OidIsValid(existingRelid))
- {
- Relation sameNamePartition = NULL;
-
- foreach_ptr(RelationData, mergingPartition, mergingPartitionsList)
- {
- if (RelationGetRelid(mergingPartition) == existingRelid)
- {
- sameNamePartition = mergingPartition;
- break;
- }
- }
-
- if (sameNamePartition)
- {
- /*
- * The new partition has the same name as one of merging
- * partitions.
- */
- char tmpRelName[NAMEDATALEN];
-
- /* Generate temporary name. */
- sprintf(tmpRelName, "merge-%u-%X-tmp", RelationGetRelid(rel), MyProcPid);
-
- /*
- * Rename the existing partition with a temporary name, leaving it
- * free for the new partition. We don't need to care about this
- * in the future because we're going to eventually drop the
- * existing partition anyway.
- */
- RenameRelationInternal(RelationGetRelid(sameNamePartition),
- tmpRelName, false, false);
-
- /*
- * We must bump the command counter to make the new partition
- * tuple visible for rename.
- */
- CommandCounterIncrement();
- }
- else
- {
- ereport(ERROR,
- (errcode(ERRCODE_DUPLICATE_TABLE),
- errmsg("relation \"%s\" already exists", cmd->name->relname)));
- }
- }
-
- /* Detach all merged partitions. */
- defaultPartOid =
- get_default_oid_from_partdesc(RelationGetPartitionDesc(rel, true));
- foreach(listptr, mergingPartitionsList)
- {
- Relation mergingPartition = (Relation) lfirst(listptr);
-
- /* Remove the pg_inherits row first. */
- RemoveInheritance(mergingPartition, rel, false);
- /* Do the final part of detaching. */
- DetachPartitionFinalize(rel, mergingPartition, false, defaultPartOid);
- }
-
- /* Create table for new partition, use partitioned table as model. */
- newPartRel = createPartitionTable(cmd->name, rel, context);
-
- /* Copy data from merged partitions to new partition. */
- moveMergedTablesRows(rel, mergingPartitionsList, newPartRel);
-
- /* Drop the current partitions before attaching the new one. */
- foreach(listptr, mergingPartitionsList)
- {
- ObjectAddress object;
- Relation mergingPartition = (Relation) lfirst(listptr);
-
- /* Get relation id before table_close() call. */
- object.objectId = RelationGetRelid(mergingPartition);
- object.classId = RelationRelationId;
- object.objectSubId = 0;
-
- /* Keep the lock until commit. */
- table_close(mergingPartition, NoLock);
-
- performDeletion(&object, DROP_RESTRICT, 0);
- }
- list_free(mergingPartitionsList);
-
- /*
- * Attach a new partition to the partitioned table. wqueue = NULL:
- * verification for each cloned constraint is not needed.
- */
- attachPartitionTable(NULL, rel, newPartRel, cmd->bound);
-
- /* Keep the lock until commit. */
- table_close(newPartRel, NoLock);
-}
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index b7d98eb9f02..84cef57a707 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -269,7 +269,6 @@ 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,8 +645,6 @@ 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
@@ -763,7 +760,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
ORDER ORDINALITY OTHERS OUT_P OUTER_P
OVER OVERLAPS OVERLAY OVERRIDING OWNED OWNER
- PARALLEL PARAMETER PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PATH
+ PARALLEL PARAMETER PARSER PARTIAL PARTITION PASSING PASSWORD PATH
PLACING PLAN PLANS POLICY
POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY
PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROCEDURES PROGRAM PUBLICATION
@@ -778,7 +775,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 SPLIT SOURCE SQL_P STABLE STANDALONE_P
+ SIMILAR SIMPLE SKIP SMALLINT SNAPSHOT SOME 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
@@ -2311,23 +2308,6 @@ 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
@@ -2338,7 +2318,6 @@ partition_cmd:
n->subtype = AT_AttachPartition;
cmd->name = $3;
cmd->bound = $4;
- cmd->partlist = NULL;
cmd->concurrent = false;
n->def = (Node *) cmd;
@@ -2353,7 +2332,6 @@ partition_cmd:
n->subtype = AT_DetachPartition;
cmd->name = $3;
cmd->bound = NULL;
- cmd->partlist = NULL;
cmd->concurrent = $4;
n->def = (Node *) cmd;
@@ -2367,35 +2345,6 @@ 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> 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
- {
- 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;
@@ -2412,7 +2361,6 @@ index_partition_cmd:
n->subtype = AT_AttachPartition;
cmd->name = $3;
cmd->bound = NULL;
- cmd->partlist = NULL;
cmd->concurrent = false;
n->def = (Node *) cmd;
@@ -4138,7 +4086,6 @@ TableLikeClause:
n->relation = $2;
n->options = $3;
n->relationOid = InvalidOid;
- n->newRelationOid = InvalidOid;
$$ = (Node *) n;
}
;
@@ -17744,7 +17691,6 @@ unreserved_keyword:
| PARSER
| PARTIAL
| PARTITION
- | PARTITIONS
| PASSING
| PASSWORD
| PATH
@@ -17812,7 +17758,6 @@ unreserved_keyword:
| SKIP
| SNAPSHOT
| SOURCE
- | SPLIT
| SQL_P
| STABLE
| STANDALONE_P
@@ -18369,7 +18314,6 @@ bare_label_keyword:
| PARSER
| PARTIAL
| PARTITION
- | PARTITIONS
| PASSING
| PASSWORD
| PATH
@@ -18449,7 +18393,6 @@ 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 d5c2b2ff0b0..79cad4ab30c 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -32,7 +32,6 @@
#include "catalog/heap.h"
#include "catalog/index.h"
#include "catalog/namespace.h"
-#include "catalog/partition.h"
#include "catalog/pg_am.h"
#include "catalog/pg_collation.h"
#include "catalog/pg_constraint.h"
@@ -59,8 +58,6 @@
#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"
@@ -136,7 +133,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, PartitionBoundSpec *bound);
+static void transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd);
static List *transformPartitionRangeBounds(ParseState *pstate, List *blist,
Relation parent);
static void validateInfiniteBounds(ParseState *pstate, List *blist);
@@ -3233,160 +3230,6 @@ transformRuleStmt(RuleStmt *stmt, const char *queryString,
/*
- * checkPartition
- * Check that partRelOid is an oid of partition of the parent table rel
- */
-static void
-checkPartition(Relation rel, Oid partRelOid)
-{
- Relation partRel;
-
- partRel = relation_open(partRelOid, AccessShareLock);
-
- if (partRel->rd_rel->relkind != RELKIND_RELATION)
- ereport(ERROR,
- (errcode(ERRCODE_WRONG_OBJECT_TYPE),
- errmsg("\"%s\" is not a table",
- RelationGetRelationName(partRel))));
-
- if (!partRel->rd_rel->relispartition)
- ereport(ERROR,
- (errcode(ERRCODE_WRONG_OBJECT_TYPE),
- errmsg("\"%s\" is not a partition",
- RelationGetRelationName(partRel))));
-
- if (get_partition_parent(partRelOid, false) != RelationGetRelid(rel))
- ereport(ERROR,
- (errcode(ERRCODE_UNDEFINED_TABLE),
- errmsg("relation \"%s\" is not a partition of relation \"%s\"",
- RelationGetRelationName(partRel),
- RelationGetRelationName(rel))));
-
- /* Permissions checks */
- if (!object_ownercheck(RelationRelationId, RelationGetRelid(partRel), GetUserId()))
- aclcheck_error(ACLCHECK_NOT_OWNER, get_relkind_objtype(partRel->rd_rel->relkind),
- RelationGetRelationName(partRel));
-
- relation_close(partRel, AccessShareLock);
-}
-
-/*
- * 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
- *
- * Does simple checks for merged partitions. Calculates bound of resulting
- * partition.
- */
-static void
-transformPartitionCmdForMerge(CreateStmtContext *cxt, PartitionCmd *partcmd)
-{
- Oid defaultPartOid;
- Oid partOid;
- Relation parent = cxt->rel;
- PartitionKey key;
- char strategy;
- ListCell *listptr,
- *listptr2;
- bool isDefaultPart = false;
- List *partOids = NIL;
-
- if (parent->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
- errmsg("\"%s\" is not a partitioned table", RelationGetRelationName(parent))));
-
- key = RelationGetPartitionKey(parent);
- strategy = get_partition_strategy(key);
-
- if (strategy == PARTITION_STRATEGY_HASH)
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- errmsg("partition of hash-partitioned table cannot be merged")));
-
- /* Is current partition a DEFAULT partition? */
- defaultPartOid = get_default_oid_from_partdesc(
- RelationGetPartitionDesc(parent, true));
-
- foreach(listptr, partcmd->partlist)
- {
- RangeVar *name = (RangeVar *) lfirst(listptr);
-
- /* Partitions in the list should have different names. */
- for_each_cell(listptr2, partcmd->partlist, lnext(partcmd->partlist, listptr))
- {
- RangeVar *name2 = (RangeVar *) lfirst(listptr2);
-
- if (equal(name, name2))
- ereport(ERROR,
- (errcode(ERRCODE_DUPLICATE_TABLE),
- errmsg("partition with name \"%s\" is already used", name->relname)),
- parser_errposition(cxt->pstate, name2->location));
- }
-
- /* Search DEFAULT partition in the list. */
- partOid = RangeVarGetRelid(name, NoLock, false);
- if (partOid == defaultPartOid)
- isDefaultPart = true;
-
- checkPartition(parent, partOid);
-
- partOids = lappend_oid(partOids, partOid);
- }
-
- /* Allocate bound of resulting partition. */
- Assert(partcmd->bound == NULL);
- partcmd->bound = makeNode(PartitionBoundSpec);
-
- /* Fill partition bound. */
- partcmd->bound->strategy = strategy;
- partcmd->bound->location = -1;
- partcmd->bound->is_default = isDefaultPart;
- if (!isDefaultPart)
- calculate_partition_bound_for_merge(parent, partcmd->partlist,
- partOids, partcmd->bound,
- cxt->pstate);
-}
-
-/*
* transformAlterTableStmt -
* parse analysis for ALTER TABLE
*
@@ -3654,7 +3497,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
{
PartitionCmd *partcmd = (PartitionCmd *) cmd->def;
- transformPartitionCmd(&cxt, partcmd->bound);
+ transformPartitionCmd(&cxt, partcmd);
/* assign transformed value of the partition bound */
partcmd->bound = cxt.partbound;
}
@@ -3662,24 +3505,6 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
newcmds = lappend(newcmds, cmd);
break;
- case AT_SplitPartition:
- case AT_MergePartitions:
- {
- PartitionCmd *partcmd = (PartitionCmd *) cmd->def;
-
- if (list_length(partcmd->partlist) < 2)
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
- errmsg("list of new partitions should contain at least two items")));
-
- if (cmd->subtype == AT_SplitPartition)
- transformPartitionCmdForSplit(&cxt, partcmd);
- else
- transformPartitionCmdForMerge(&cxt, partcmd);
- newcmds = lappend(newcmds, cmd);
- break;
- }
-
default:
/*
@@ -4070,13 +3895,13 @@ setSchemaName(const char *context_schema, char **stmt_schema_name)
/*
* transformPartitionCmd
- * Analyze the ATTACH/DETACH/SPLIT PARTITION command
+ * Analyze the ATTACH/DETACH PARTITION command
*
- * In case of the ATTACH/SPLIT PARTITION command, cxt->partbound is set to the
- * transformed value of bound.
+ * In case of the ATTACH PARTITION command, cxt->partbound is set to the
+ * transformed value of cmd->bound.
*/
static void
-transformPartitionCmd(CreateStmtContext *cxt, PartitionBoundSpec *bound)
+transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd)
{
Relation parentRel = cxt->rel;
@@ -4085,9 +3910,9 @@ transformPartitionCmd(CreateStmtContext *cxt, PartitionBoundSpec *bound)
case RELKIND_PARTITIONED_TABLE:
/* transform the partition bound, if any */
Assert(RelationGetPartitionKey(parentRel) != NULL);
- if (bound != NULL)
+ if (cmd->bound != NULL)
cxt->partbound = transformPartitionBound(cxt->pstate, parentRel,
- bound);
+ cmd->bound);
break;
case RELKIND_PARTITIONED_INDEX:
@@ -4095,7 +3920,7 @@ transformPartitionCmd(CreateStmtContext *cxt, PartitionBoundSpec *bound)
* A partitioned index cannot have a partition bound set. ALTER
* INDEX prevents that with its grammar, but not ALTER TABLE.
*/
- if (bound != NULL)
+ if (cmd->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 4eda59767ce..c28639d2e3f 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -3214,9 +3214,8 @@ check_new_partition_bound(char *relname, Relation parent,
PartitionRangeDatum *datum;
/*
- * Point to problematic key in the list of lower
- * datums; if we have equality, point to the first
- * one.
+ * Point to problematic key in the lower datums list;
+ * if we have equality, point to the first one.
*/
datum = cmpval == 0 ? linitial(spec->lowerdatums) :
list_nth(spec->lowerdatums, abs(cmpval) - 1);
@@ -4978,899 +4977,3 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
PG_RETURN_BOOL(rowHash % modulus == remainder);
}
-
-/*
- * check_two_partitions_bounds_range
- *
- * (function for BY RANGE partitioning)
- *
- * This is a helper function for check_partitions_for_split() and
- * calculate_partition_bound_for_merge().
- * This function compares upper bound of first_bound and lower bound of
- * second_bound. These bounds should be equal except when
- * "defaultPart == true" (this means that one of split partitions is DEFAULT).
- * In this case upper bound of first_bound can be less than lower bound of
- * second_bound because space between these bounds will be included in
- * DEFAULT partition.
- *
- * parent: partitioned table
- * first_name: name of first partition
- * first_bound: bound of first partition
- * second_name: name of second partition
- * second_bound: bound of second partition
- * defaultPart: true if one of split partitions is DEFAULT
- * pstate: pointer to ParseState struct for determining error position
- */
-static void
-check_two_partitions_bounds_range(Relation parent,
- RangeVar *first_name,
- PartitionBoundSpec *first_bound,
- RangeVar *second_name,
- PartitionBoundSpec *second_bound,
- bool defaultPart,
- ParseState *pstate)
-{
- PartitionKey key = RelationGetPartitionKey(parent);
- PartitionRangeBound *first_upper;
- PartitionRangeBound *second_lower;
- int cmpval;
-
- Assert(key->strategy == PARTITION_STRATEGY_RANGE);
-
- first_upper = make_one_partition_rbound(key, -1, first_bound->upperdatums, false);
- second_lower = make_one_partition_rbound(key, -1, second_bound->lowerdatums, true);
-
- /*
- * lower1=false (the second to last argument) for correct comparison of
- * lower and upper bounds.
- */
- cmpval = partition_rbound_cmp(key->partnatts,
- key->partsupfunc,
- key->partcollation,
- second_lower->datums, second_lower->kind,
- false, first_upper);
- if ((!defaultPart && cmpval) || (defaultPart && cmpval < 0))
- {
- PartitionRangeDatum *datum = linitial(second_bound->lowerdatums);
-
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
- errmsg("lower bound of partition \"%s\" conflicts with upper bound of previous partition \"%s\"",
- second_name->relname, first_name->relname),
- parser_errposition(pstate, datum->location)));
- }
-}
-
-/*
- * 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)
-{
- 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;
-}
-
-/*
- * 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
- *
- * Calculates the bound of merged partition "spec" by using the bounds of
- * partitions to be merged.
- *
- * parent: partitioned table
- * partNames: names of partitions to be merged
- * partOids: Oids of partitions to be merged
- * spec (out): bounds specification of the merged partition
- * pstate: pointer to ParseState struct for determine error position
- */
-void
-calculate_partition_bound_for_merge(Relation parent,
- List *partNames,
- List *partOids,
- PartitionBoundSpec *spec,
- ParseState *pstate)
-{
- PartitionKey key = RelationGetPartitionKey(parent);
- PartitionBoundSpec *bound;
-
- Assert(!spec->is_default);
-
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_RANGE:
- {
- int i;
- PartitionRangeBound **lower_bounds;
- int nparts = list_length(partOids);
- List *bounds = NIL;
-
- lower_bounds = (PartitionRangeBound **)
- palloc0(nparts * sizeof(PartitionRangeBound *));
-
- /*
- * Create array of lower bounds and list of
- * PartitionBoundSpec.
- */
- for (i = 0; i < nparts; i++)
- {
- bound = get_partition_bound_spec(list_nth_oid(partOids, i),
- (RangeVar *) list_nth(partNames, i));
-
- lower_bounds[i] = make_one_partition_rbound(key, i, bound->lowerdatums, true);
- bounds = lappend(bounds, bound);
- }
-
- /* Sort array of lower bounds. */
- qsort_arg(lower_bounds, nparts, sizeof(PartitionRangeBound *),
- qsort_partition_rbound_cmp, (void *) key);
-
- /* Ranges of partitions should not overlap. */
- for (i = 1; i < nparts; i++)
- {
- int index = lower_bounds[i]->index;
- int prev_index = lower_bounds[i - 1]->index;
-
- check_two_partitions_bounds_range(parent,
- (RangeVar *) list_nth(partNames, prev_index),
- (PartitionBoundSpec *) list_nth(bounds, prev_index),
- (RangeVar *) list_nth(partNames, index),
- (PartitionBoundSpec *) list_nth(bounds, index),
- false, pstate);
- }
-
- /*
- * Lower bound of first partition is the lower bound of merged
- * partition.
- */
- spec->lowerdatums =
- ((PartitionBoundSpec *) list_nth(bounds, lower_bounds[0]->index))->lowerdatums;
-
- /*
- * Upper bound of last partition is the upper bound of merged
- * partition.
- */
- spec->upperdatums =
- ((PartitionBoundSpec *) list_nth(bounds, lower_bounds[nparts - 1]->index))->upperdatums;
-
- pfree(lower_bounds);
- list_free(bounds);
- break;
- }
-
- case PARTITION_STRATEGY_LIST:
- {
- ListCell *listptr,
- *listptr2;
-
- /* Consolidate bounds for all partitions in the list. */
- forboth(listptr, partOids, listptr2, partNames)
- {
- RangeVar *name = (RangeVar *) lfirst(listptr2);
- Oid curOid = lfirst_oid(listptr);
-
- bound = get_partition_bound_spec(curOid, name);
- spec->listdatums = list_concat(spec->listdatums, bound->listdatums);
- }
- break;
- }
-
- default:
- elog(ERROR, "unexpected partition strategy: %d",
- (int) key->strategy);
- }
-}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index b385175e7a2..b2ea8125c92 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -1225,12 +1225,6 @@ ProcessUtilitySlow(ParseState *pstate,
morestmts = expandTableLikeClause(table_rv, like);
stmts = list_concat(morestmts, stmts);
-
- /*
- * Store the OID of newly created relation to the
- * TableLikeClause for the caller to use it.
- */
- like->newRelationOid = address.objectId;
}
else
{
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 4039ee0df73..00eda1b34c0 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -13296,21 +13296,3 @@ 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;
-}