Implement ALTER TABLE ADD CONSTRAINT command

pull/1292/head
velioglu 2017-03-23 14:21:57 +03:00
parent fbb6a47adf
commit 24d24db25c
15 changed files with 1404 additions and 591 deletions

View File

@ -20,9 +20,7 @@
#include "access/xact.h"
#include "catalog/dependency.h"
#include "catalog/index.h"
#include "catalog/indexing.h"
#include "catalog/pg_am.h"
#include "catalog/pg_constraint.h"
#if (PG_VERSION_NUM >= 90600)
#include "catalog/pg_constraint_fn.h"
#endif
@ -30,7 +28,6 @@
#include "catalog/pg_extension.h"
#include "catalog/pg_opclass.h"
#include "catalog/pg_trigger.h"
#include "catalog/pg_type.h"
#include "commands/defrem.h"
#include "commands/extension.h"
#include "commands/trigger.h"
@ -42,6 +39,7 @@
#include "distributed/metadata_sync.h"
#include "distributed/multi_copy.h"
#include "distributed/multi_logical_planner.h"
#include "distributed/multi_utility.h"
#include "distributed/pg_dist_colocation.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/reference_table_utils.h"
@ -59,7 +57,6 @@
#include "tcop/pquery.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/rel.h"
@ -81,12 +78,6 @@ static char LookupDistributionMethod(Oid distributionMethodOid);
static Oid SupportFunctionForColumn(Var *partitionColumn, Oid accessMethodId,
int16 supportFunctionNumber);
static bool LocalTableEmpty(Oid tableId);
static void ErrorIfNotSupportedConstraint(Relation relation, char distributionMethod,
Var *distributionColumn, uint32 colocationId);
static void ErrorIfNotSupportedForeignConstraint(Relation relation,
char distributionMethod,
Var *distributionColumn,
uint32 colocationId);
static void CreateHashDistributedTable(Oid relationId, char *distributionColumnName,
char *colocateWithTableName,
int shardCount, int replicationFactor);
@ -401,8 +392,8 @@ ConvertToDistributedTable(Oid relationId, char *distributionColumnName,
}
}
ErrorIfNotSupportedConstraint(relation, distributionMethod, distributionColumn,
colocationId);
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn,
colocationId);
InsertIntoPgDistPartition(relationId, distributionMethod, distributionColumn,
colocationId, replicationModel);
@ -420,348 +411,6 @@ ConvertToDistributedTable(Oid relationId, char *distributionColumnName,
}
/*
* ErrorIfNotSupportedConstraint run checks related to unique index / exclude
* constraints.
*
* The function skips the uniqeness checks for reference tables (i.e., distribution
* method is 'none').
*
* Forbid UNIQUE, PRIMARY KEY, or EXCLUDE constraints on append partitioned
* tables, since currently there is no way of enforcing uniqueness for
* overlapping shards.
*
* Similarly, do not allow such constraints if they do not include partition
* column. This check is important for two reasons:
* i. First, currently Citus does not enforce uniqueness constraint on multiple
* shards.
* ii. Second, INSERT INTO .. ON CONFLICT (i.e., UPSERT) queries can be executed
* with no further check for constraints.
*/
static void
ErrorIfNotSupportedConstraint(Relation relation, char distributionMethod,
Var *distributionColumn, uint32 colocationId)
{
char *relationName = NULL;
List *indexOidList = NULL;
ListCell *indexOidCell = NULL;
/*
* We first perform check for foreign constraints. It is important to do this check
* before next check, because other types of constraints are allowed on reference
* tables and we return early for those constraints thanks to next check. Therefore,
* for reference tables, we first check for foreing constraints and if they are OK,
* we do not error out for other types of constraints.
*/
ErrorIfNotSupportedForeignConstraint(relation, distributionMethod, distributionColumn,
colocationId);
/*
* Citus supports any kind of uniqueness constraints for reference tables
* given that they only consist of a single shard and we can simply rely on
* Postgres.
*/
if (distributionMethod == DISTRIBUTE_BY_NONE)
{
return;
}
relationName = RelationGetRelationName(relation);
indexOidList = RelationGetIndexList(relation);
foreach(indexOidCell, indexOidList)
{
Oid indexOid = lfirst_oid(indexOidCell);
Relation indexDesc = index_open(indexOid, RowExclusiveLock);
IndexInfo *indexInfo = NULL;
AttrNumber *attributeNumberArray = NULL;
bool hasDistributionColumn = false;
int attributeCount = 0;
int attributeIndex = 0;
/* extract index key information from the index's pg_index info */
indexInfo = BuildIndexInfo(indexDesc);
/* only check unique indexes and exclusion constraints. */
if (indexInfo->ii_Unique == false && indexInfo->ii_ExclusionOps == NULL)
{
index_close(indexDesc, NoLock);
continue;
}
/*
* Citus cannot enforce uniqueness/exclusion constraints with overlapping shards.
* Thus, emit a warning for unique indexes and exclusion constraints on
* append partitioned tables.
*/
if (distributionMethod == DISTRIBUTE_BY_APPEND)
{
ereport(WARNING, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("table \"%s\" has a UNIQUE or EXCLUDE constraint",
relationName),
errdetail("UNIQUE constraints, EXCLUDE constraints, "
"and PRIMARY KEYs on "
"append-partitioned tables cannot be enforced."),
errhint("Consider using hash partitioning.")));
}
attributeCount = indexInfo->ii_NumIndexAttrs;
attributeNumberArray = indexInfo->ii_KeyAttrNumbers;
for (attributeIndex = 0; attributeIndex < attributeCount; attributeIndex++)
{
AttrNumber attributeNumber = attributeNumberArray[attributeIndex];
bool uniqueConstraint = false;
bool exclusionConstraintWithEquality = false;
if (distributionColumn->varattno != attributeNumber)
{
continue;
}
uniqueConstraint = indexInfo->ii_Unique;
exclusionConstraintWithEquality = (indexInfo->ii_ExclusionOps != NULL &&
OperatorImplementsEquality(
indexInfo->ii_ExclusionOps[
attributeIndex]));
if (uniqueConstraint || exclusionConstraintWithEquality)
{
hasDistributionColumn = true;
break;
}
}
if (!hasDistributionColumn)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot distribute relation: \"%s\"",
relationName),
errdetail("Distributed relations cannot have UNIQUE, "
"EXCLUDE, or PRIMARY KEY constraints that do not "
"include the partition column (with an equality "
"operator if EXCLUDE).")));
}
index_close(indexDesc, NoLock);
}
}
/*
* ErrorIfNotSupportedForeignConstraint runs checks related to foreign constraints and
* errors out if it is not possible to create one of the foreign constraint in distributed
* environment.
*
* To support foreign constraints, we require that;
* - Referencing and referenced tables are hash distributed.
* - Referencing and referenced tables are co-located.
* - Foreign constraint is defined over distribution column.
* - ON DELETE/UPDATE SET NULL, ON DELETE/UPDATE SET DEFAULT and ON UPDATE CASCADE options
* are not used.
* - Replication factors of referencing and referenced table are 1.
*/
static void
ErrorIfNotSupportedForeignConstraint(Relation relation, char distributionMethod,
Var *distributionColumn, uint32 colocationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
HeapTuple heapTuple = NULL;
Oid referencedTableId = InvalidOid;
uint32 referencedTableColocationId = INVALID_COLOCATION_ID;
Var *referencedTablePartitionColumn = NULL;
Datum referencingColumnsDatum;
Datum *referencingColumnArray;
int referencingColumnCount = 0;
Datum referencedColumnsDatum;
Datum *referencedColumnArray;
int referencedColumnCount = 0;
bool isNull = false;
int attrIdx = 0;
bool foreignConstraintOnPartitionColumn = false;
bool selfReferencingTable = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
relation->rd_id);
scanDescriptor = systable_beginscan(pgConstraint, ConstraintRelidIndexId, true, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
if (constraintForm->contype != CONSTRAINT_FOREIGN)
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
referencedTableId = constraintForm->confrelid;
selfReferencingTable = relation->rd_id == referencedTableId;
/*
* We do not support foreign keys for reference tables. Here we skip the second
* part of check if the table is a self referencing table because;
* - PartitionMethod only works for distributed tables and this table is not
* distributed yet.
* - Since referencing and referenced tables are same, it is OK to not checking
* distribution method twice.
*/
if (distributionMethod == DISTRIBUTE_BY_NONE ||
(!selfReferencingTable &&
PartitionMethod(referencedTableId) == DISTRIBUTE_BY_NONE))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Foreign key constraints are not allowed from or "
"to reference tables.")));
}
/*
* ON DELETE SET NULL and ON DELETE SET DEFAULT is not supported. Because we do
* not want to set partition column to NULL or default value.
*/
if (constraintForm->confdeltype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confdeltype == FKCONSTR_ACTION_SETDEFAULT)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL or SET DEFAULT is not supported"
" in ON DELETE operation.")));
}
/*
* ON UPDATE SET NULL, ON UPDATE SET DEFAULT and UPDATE CASCADE is not supported.
* Because we do not want to set partition column to NULL or default value. Also
* cascading update operation would require re-partitioning. Updating partition
* column value is not allowed anyway even outside of foreign key concept.
*/
if (constraintForm->confupdtype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confupdtype == FKCONSTR_ACTION_SETDEFAULT ||
constraintForm->confupdtype == FKCONSTR_ACTION_CASCADE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL, SET DEFAULT or CASCADE is not"
" supported in ON UPDATE operation.")));
}
/*
* Some checks are not meaningful if foreign key references the table itself.
* Therefore we will skip those checks.
*/
if (!selfReferencingTable)
{
if (!IsDistributedTable(referencedTableId))
{
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("cannot create foreign key constraint"),
errdetail("Referenced table must be a distributed "
"table.")));
}
/* to enforce foreign constraints, tables must be co-located */
referencedTableColocationId = TableColocationId(referencedTableId);
if (relation->rd_id != referencedTableId &&
(colocationId == INVALID_COLOCATION_ID ||
colocationId != referencedTableColocationId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Foreign key constraint can only be created"
" on co-located tables.")));
}
/*
* Partition column must exist in both referencing and referenced side of the
* foreign key constraint. They also must be in same ordinal.
*/
referencedTablePartitionColumn = PartitionKey(referencedTableId);
}
else
{
/*
* Partition column must exist in both referencing and referenced side of the
* foreign key constraint. They also must be in same ordinal.
*/
referencedTablePartitionColumn = distributionColumn;
}
/*
* Column attributes are not available in Form_pg_constraint, therefore we need
* to find them in the system catalog. After finding them, we iterate over column
* attributes together because partition column must be at the same place in both
* referencing and referenced side of the foreign key constraint
*/
referencingColumnsDatum = SysCacheGetAttr(CONSTROID, heapTuple,
Anum_pg_constraint_conkey, &isNull);
referencedColumnsDatum = SysCacheGetAttr(CONSTROID, heapTuple,
Anum_pg_constraint_confkey, &isNull);
deconstruct_array(DatumGetArrayTypeP(referencingColumnsDatum), INT2OID, 2, true,
's', &referencingColumnArray, NULL, &referencingColumnCount);
deconstruct_array(DatumGetArrayTypeP(referencedColumnsDatum), INT2OID, 2, true,
's', &referencedColumnArray, NULL, &referencedColumnCount);
Assert(referencingColumnCount == referencedColumnCount);
for (attrIdx = 0; attrIdx < referencingColumnCount; ++attrIdx)
{
AttrNumber referencingAttrNo = DatumGetInt16(referencingColumnArray[attrIdx]);
AttrNumber referencedAttrNo = DatumGetInt16(referencedColumnArray[attrIdx]);
if (distributionColumn->varattno == referencingAttrNo &&
referencedTablePartitionColumn->varattno == referencedAttrNo)
{
foreignConstraintOnPartitionColumn = true;
}
}
if (!foreignConstraintOnPartitionColumn)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Partition column must exist both "
"referencing and referenced side of the "
"foreign constraint statement and it must "
"be in the same ordinal in both sides.")));
}
/*
* We do not allow to create foreign constraints if shard replication factor is
* greater than 1. Because in our current design, multiple replicas may cause
* locking problems and inconsistent shard contents.
*/
if (ShardReplicationFactor > 1 || (referencedTableId != relation->rd_id &&
!SingleReplicatedTable(referencedTableId)))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Citus Community Edition currently supports "
"foreign key constraints only for "
"\"citus.shard_replication_factor = 1\"."),
errhint("Please change \"citus.shard_replication_factor to "
"1\". To learn more about using foreign keys with "
"other replication factors, please contact us at "
"https://citusdata.com/about/contact_us.")));
}
heapTuple = systable_getnext(scanDescriptor);
}
/* clean up scan and close system catalog */
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
}
/*
* LookupDistributionMethod maps the oids of citus.distribution_type enum
* values to pg_dist_partition.partmethod values.

View File

@ -29,6 +29,8 @@
#include "catalog/pg_attribute.h"
#include "catalog/pg_class.h"
#include "citus_version.h"
#include "catalog/pg_constraint.h"
#include "catalog/pg_type.h"
#include "commands/defrem.h"
#include "commands/tablecmds.h"
#include "commands/prepare.h"
@ -70,6 +72,7 @@
#include "utils/builtins.h"
#include "utils/elog.h"
#include "utils/errcodes.h"
#include "utils/fmgroids.h"
#include "utils/guc.h"
#include "utils/hsearch.h"
#include "utils/inval.h"
@ -134,6 +137,11 @@ static void ErrorIfDistributedAlterSeqOwnedBy(AlterSeqStmt *alterSeqStmt);
static void ErrorIfUnsupportedTruncateStmt(TruncateStmt *truncateStatement);
static bool OptionsSpecifyOwnedBy(List *optionList, Oid *ownedByTableId);
static void ErrorIfUnsupportedRenameStmt(RenameStmt *renameStmt);
static void ErrorIfUnsupportedAlterAddConstraintStmt(AlterTableStmt *alterTableStatement);
static void ErrorIfUnsupportedForeignConstraint(Relation relation,
char distributionMethod,
Var *distributionColumn,
uint32 colocationId);
/* Local functions forward declarations for helper functions */
static char * ExtractNewExtensionVersion(Node *parsetree);
@ -388,6 +396,33 @@ multi_ProcessUtility(Node *parsetree,
{
ListCell *ddlJobCell = NULL;
/*
* At this point, ALTER TABLE command has already run on the master, so we
* are checking constraints over the table with constraints already defined
* (to make the constraint check process same for ALTER TABLE and CREATE
* TABLE). If constraints do not fulfill the rules we defined, they will
* be removed and the table will return back to the state before the ALTER
* TABLE command.
*/
if (IsA(parsetree, AlterTableStmt))
{
AlterTableStmt *alterTableStatement = (AlterTableStmt *) parsetree;
List *commandList = alterTableStatement->cmds;
ListCell *commandCell = NULL;
foreach(commandCell, commandList)
{
AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell);
AlterTableType alterTableType = command->subtype;
if (alterTableType == AT_AddConstraint)
{
Assert(list_length(commandList) == 1);
ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement);
}
}
}
foreach(ddlJobCell, ddlJobs)
{
DDLJob *ddlJob = (DDLJob *) lfirst(ddlJobCell);
@ -938,7 +973,7 @@ PlanAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCo
* only subcommand of ALTER TABLE. It was already checked in
* ErrorIfUnsupportedAlterTableStmt.
*/
Assert(commandList->length <= 1);
Assert(list_length(commandList) <= 1);
rightRelationId = RangeVarGetRelid(constraint->pktable, lockmode,
alterTableStatement->missing_ok);
@ -1584,7 +1619,7 @@ ErrorIfUnsupportedDropIndexStmt(DropStmt *dropIndexStatement)
* ALTER TABLE ALTER COLUMN SET DATA TYPE
* ALTER TABLE SET|DROP NOT NULL
* ALTER TABLE SET|DROP DEFAULT
* ALTER TABLE ADD|DROP CONSTRAINT FOREIGN
* ALTER TABLE ADD|DROP CONSTRAINT
*/
static void
ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
@ -1635,7 +1670,6 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
case AT_DropColumn:
case AT_ColumnDefault:
case AT_AlterColumnType:
case AT_SetNotNull:
case AT_DropNotNull:
{
/* error out if the alter table command is on the partition column */
@ -1675,83 +1709,16 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
case AT_AddConstraint:
{
Constraint *constraint = (Constraint *) command->def;
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid referencingTableId = InvalidOid;
Oid referencedTableId = InvalidOid;
Var *referencingTablePartitionColumn = NULL;
Var *referencedTablePartitionColumn = NULL;
ListCell *referencingTableAttr = NULL;
ListCell *referencedTableAttr = NULL;
bool foreignConstraintOnPartitionColumn = false;
/* we only allow adding foreign constraints with ALTER TABLE */
if (constraint->contype != CONSTR_FOREIGN)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create constraint"),
errdetail("Citus cannot execute ADD CONSTRAINT "
"command other than ADD CONSTRAINT FOREIGN "
"KEY.")));
}
/* we only allow foreign constraints if they are only subcommand */
/* we only allow constraints if they are only subcommand */
if (commandList->length > 1)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Citus cannot execute ADD CONSTRAINT "
"FOREIGN KEY command together with other "
"subcommands."),
errmsg("cannot execute ADD CONSTRAINT command with "
"other subcommands"),
errhint("You can issue each subcommand separately")));
}
referencingTableId = RangeVarGetRelid(alterTableStatement->relation,
lockmode,
alterTableStatement->missing_ok);
referencedTableId = RangeVarGetRelid(constraint->pktable, lockmode,
alterTableStatement->missing_ok);
/* we do not support foreign keys for reference tables */
if (PartitionMethod(referencingTableId) == DISTRIBUTE_BY_NONE ||
PartitionMethod(referencedTableId) == DISTRIBUTE_BY_NONE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail(
"Foreign key constraints are not allowed from or "
"to reference tables.")));
}
/*
* ON DELETE SET NULL and ON DELETE SET DEFAULT is not supported. Because
* we do not want to set partition column to NULL or default value.
*/
if (constraint->fk_del_action == FKCONSTR_ACTION_SETNULL ||
constraint->fk_del_action == FKCONSTR_ACTION_SETDEFAULT)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL or SET DEFAULT is not supported"
" in ON DELETE operation.")));
}
/*
* ON UPDATE SET NULL, ON UPDATE SET DEFAULT and UPDATE CASCADE is not
* supported. Because we do not want to set partition column to NULL or
* default value. Also cascading update operation would require
* re-partitioning. Updating partition column value is not allowed anyway
* even outside of foreign key concept.
*/
if (constraint->fk_upd_action == FKCONSTR_ACTION_SETNULL ||
constraint->fk_upd_action == FKCONSTR_ACTION_SETDEFAULT ||
constraint->fk_upd_action == FKCONSTR_ACTION_CASCADE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL, SET DEFAULT or CASCADE is not"
" supported in ON UPDATE operation.")));
}
/*
* We will use constraint name in each placement by extending it at
* workers. Therefore we require it to be exist.
@ -1759,114 +1726,23 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
if (constraint->conname == NULL)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Creating foreign constraint without a "
"name on a distributed table is currently "
"not supported.")));
}
/* to enforce foreign constraints, tables must be co-located */
if (!TablesColocated(referencingTableId, referencedTableId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Foreign key constraint can only be created"
" on co-located tables.")));
}
/*
* The following logic requires the referenced columns to exists in
* the statement. Otherwise, we cannot apply some of the checks.
*/
if (constraint->pk_attrs == NULL)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint "
"because referenced column list is empty"),
errhint("Add column names to \"REFERENCES\" part of "
"the statement.")));
}
/*
* Referencing column's list length should be equal to referenced columns
* list length.
*/
if (constraint->fk_attrs->length != constraint->pk_attrs->length)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Referencing column list and referenced "
"column list must be in same size.")));
}
/*
* Partition column must exist in both referencing and referenced side
* of the foreign key constraint. They also must be in same ordinal.
*/
referencingTablePartitionColumn = PartitionKey(referencingTableId);
referencedTablePartitionColumn = PartitionKey(referencedTableId);
/*
* We iterate over fk_attrs and pk_attrs together because partition
* column must be at the same place in both referencing and referenced
* side of the foreign key constraint
*/
forboth(referencingTableAttr, constraint->fk_attrs,
referencedTableAttr, constraint->pk_attrs)
{
char *referencingAttrName = strVal(lfirst(referencingTableAttr));
char *referencedAttrName = strVal(lfirst(referencedTableAttr));
AttrNumber referencingAttrNo = get_attnum(referencingTableId,
referencingAttrName);
AttrNumber referencedAttrNo = get_attnum(referencedTableId,
referencedAttrName);
if (referencingTablePartitionColumn->varattno == referencingAttrNo &&
referencedTablePartitionColumn->varattno == referencedAttrNo)
{
foreignConstraintOnPartitionColumn = true;
}
}
if (!foreignConstraintOnPartitionColumn)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Partition column must exist both "
"referencing and referenced side of the "
"foreign constraint statement and it must "
"be in the same ordinal in both sides.")));
}
/*
* We do not allow to create foreign constraints if shard replication
* factor is greater than 1. Because in our current design, multiple
* replicas may cause locking problems and inconsistent shard contents.
*/
if (!SingleReplicatedTable(referencingTableId) ||
!SingleReplicatedTable(referencedTableId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Citus Community Edition currently "
"supports foreign key constraints only for "
"\"citus.shard_replication_factor = 1\"."),
errhint("Please change "
"\"citus.shard_replication_factor to 1\". To "
"learn more about using foreign keys with "
"other replication factors, please contact"
" us at "
"https://citusdata.com/about/contact_us.")));
errmsg("cannot create constraint without a name on a "
"distributed table")));
}
break;
}
case AT_SetNotNull:
case AT_DropConstraint:
case AT_EnableTrigAll:
case AT_DisableTrigAll:
{
/* we will not perform any special checks for these ALTER TABLE types */
/*
* We will not perform any special check for ALTER TABLE DROP CONSTRAINT
* , ALTER TABLE .. ALTER COLUMN .. SET NOT NULL and ALTER TABLE ENABLE/
* DISABLE TRIGGER ALL
*/
break;
}
@ -1875,14 +1751,394 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("alter table command is currently unsupported"),
errdetail("Only ADD|DROP COLUMN, SET|DROP NOT NULL,"
" SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN"
" KEY and TYPE subcommands are supported.")));
" SET|DROP DEFAULT, ADD|DROP CONSTRAINT and "
"TYPE subcommands are supported.")));
}
}
}
}
/*
* ErrorIfUnsopprtedAlterAddConstraintStmt runs the constraint checks on distributed
* table using the same logic with create_distributed_table.
*/
static void
ErrorIfUnsupportedAlterAddConstraintStmt(AlterTableStmt *alterTableStatement)
{
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
char distributionMethod = PartitionMethod(relationId);
Var *distributionColumn = PartitionKey(relationId);
uint32 colocationId = TableColocationId(relationId);
Relation relation = relation_open(relationId, ExclusiveLock);
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn,
colocationId);
relation_close(relation, NoLock);
}
/*
* ErrorIfUnsupportedConstraint run checks related to unique index / exclude
* constraints.
*
* The function skips the uniqeness checks for reference tables (i.e., distribution
* method is 'none').
*
* Forbid UNIQUE, PRIMARY KEY, or EXCLUDE constraints on append partitioned
* tables, since currently there is no way of enforcing uniqueness for
* overlapping shards.
*
* Similarly, do not allow such constraints if they do not include partition
* column. This check is important for two reasons:
* i. First, currently Citus does not enforce uniqueness constraint on multiple
* shards.
* ii. Second, INSERT INTO .. ON CONFLICT (i.e., UPSERT) queries can be executed
* with no further check for constraints.
*/
void
ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
Var *distributionColumn, uint32 colocationId)
{
char *relationName = NULL;
List *indexOidList = NULL;
ListCell *indexOidCell = NULL;
/*
* We first perform check for foreign constraints. It is important to do this check
* before next check, because other types of constraints are allowed on reference
* tables and we return early for those constraints thanks to next check. Therefore,
* for reference tables, we first check for foreing constraints and if they are OK,
* we do not error out for other types of constraints.
*/
ErrorIfUnsupportedForeignConstraint(relation, distributionMethod, distributionColumn,
colocationId);
/*
* Citus supports any kind of uniqueness constraints for reference tables
* given that they only consist of a single shard and we can simply rely on
* Postgres.
*/
if (distributionMethod == DISTRIBUTE_BY_NONE)
{
return;
}
relationName = RelationGetRelationName(relation);
indexOidList = RelationGetIndexList(relation);
foreach(indexOidCell, indexOidList)
{
Oid indexOid = lfirst_oid(indexOidCell);
Relation indexDesc = index_open(indexOid, RowExclusiveLock);
IndexInfo *indexInfo = NULL;
AttrNumber *attributeNumberArray = NULL;
bool hasDistributionColumn = false;
int attributeCount = 0;
int attributeIndex = 0;
/* extract index key information from the index's pg_index info */
indexInfo = BuildIndexInfo(indexDesc);
/* only check unique indexes and exclusion constraints. */
if (indexInfo->ii_Unique == false && indexInfo->ii_ExclusionOps == NULL)
{
index_close(indexDesc, NoLock);
continue;
}
/*
* Citus cannot enforce uniqueness/exclusion constraints with overlapping shards.
* Thus, emit a warning for unique indexes and exclusion constraints on
* append partitioned tables.
*/
if (distributionMethod == DISTRIBUTE_BY_APPEND)
{
ereport(WARNING, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("table \"%s\" has a UNIQUE or EXCLUDE constraint",
relationName),
errdetail("UNIQUE constraints, EXCLUDE constraints, "
"and PRIMARY KEYs on "
"append-partitioned tables cannot be enforced."),
errhint("Consider using hash partitioning.")));
}
attributeCount = indexInfo->ii_NumIndexAttrs;
attributeNumberArray = indexInfo->ii_KeyAttrNumbers;
for (attributeIndex = 0; attributeIndex < attributeCount; attributeIndex++)
{
AttrNumber attributeNumber = attributeNumberArray[attributeIndex];
bool uniqueConstraint = false;
bool exclusionConstraintWithEquality = false;
if (distributionColumn->varattno != attributeNumber)
{
continue;
}
uniqueConstraint = indexInfo->ii_Unique;
exclusionConstraintWithEquality = (indexInfo->ii_ExclusionOps != NULL &&
OperatorImplementsEquality(
indexInfo->ii_ExclusionOps[
attributeIndex]));
if (uniqueConstraint || exclusionConstraintWithEquality)
{
hasDistributionColumn = true;
break;
}
}
if (!hasDistributionColumn)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create constraint on \"%s\"",
relationName),
errdetail("Distributed relations cannot have UNIQUE, "
"EXCLUDE, or PRIMARY KEY constraints that do not "
"include the partition column (with an equality "
"operator if EXCLUDE).")));
}
index_close(indexDesc, NoLock);
}
}
/*
* ErrorIfUnsupportedForeignConstraint runs checks related to foreign constraints and
* errors out if it is not possible to create one of the foreign constraint in distributed
* environment.
*
* To support foreign constraints, we require that;
* - Referencing and referenced tables are hash distributed.
* - Referencing and referenced tables are co-located.
* - Foreign constraint is defined over distribution column.
* - ON DELETE/UPDATE SET NULL, ON DELETE/UPDATE SET DEFAULT and ON UPDATE CASCADE options
* are not used.
* - Replication factors of referencing and referenced table are 1.
*/
static void
ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
Var *distributionColumn, uint32 colocationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
HeapTuple heapTuple = NULL;
Oid referencingTableId = relation->rd_id;
Oid referencedTableId = InvalidOid;
uint32 referencedTableColocationId = INVALID_COLOCATION_ID;
Var *referencedTablePartitionColumn = NULL;
Datum referencingColumnsDatum;
Datum *referencingColumnArray;
int referencingColumnCount = 0;
Datum referencedColumnsDatum;
Datum *referencedColumnArray;
int referencedColumnCount = 0;
bool isNull = false;
int attrIdx = 0;
bool foreignConstraintOnPartitionColumn = false;
bool selfReferencingTable = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
relation->rd_id);
scanDescriptor = systable_beginscan(pgConstraint, ConstraintRelidIndexId, true, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
bool singleReplicatedTable = true;
if (constraintForm->contype != CONSTRAINT_FOREIGN)
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
referencedTableId = constraintForm->confrelid;
selfReferencingTable = referencingTableId == referencedTableId;
/*
* We do not support foreign keys for reference tables. Here we skip the second
* part of check if the table is a self referencing table because;
* - PartitionMethod only works for distributed tables and this table may not be
* distributed yet.
* - Since referencing and referenced tables are same, it is OK to not checking
* distribution method twice.
*/
if (distributionMethod == DISTRIBUTE_BY_NONE ||
(!selfReferencingTable &&
PartitionMethod(referencedTableId) == DISTRIBUTE_BY_NONE))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint from or to "
"reference tables")));
}
/*
* ON DELETE SET NULL and ON DELETE SET DEFAULT is not supported. Because we do
* not want to set partition column to NULL or default value.
*/
if (constraintForm->confdeltype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confdeltype == FKCONSTR_ACTION_SETDEFAULT)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL or SET DEFAULT is not supported"
" in ON DELETE operation.")));
}
/*
* ON UPDATE SET NULL, ON UPDATE SET DEFAULT and UPDATE CASCADE is not supported.
* Because we do not want to set partition column to NULL or default value. Also
* cascading update operation would require re-partitioning. Updating partition
* column value is not allowed anyway even outside of foreign key concept.
*/
if (constraintForm->confupdtype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confupdtype == FKCONSTR_ACTION_SETDEFAULT ||
constraintForm->confupdtype == FKCONSTR_ACTION_CASCADE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL, SET DEFAULT or CASCADE is not"
" supported in ON UPDATE operation.")));
}
/*
* Some checks are not meaningful if foreign key references the table itself.
* Therefore we will skip those checks.
*/
if (!selfReferencingTable)
{
if (!IsDistributedTable(referencedTableId))
{
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("cannot create foreign key constraint"),
errdetail("Referenced table must be a distributed "
"table.")));
}
/* to enforce foreign constraints, tables must be co-located */
referencedTableColocationId = TableColocationId(referencedTableId);
if (colocationId == INVALID_COLOCATION_ID ||
colocationId != referencedTableColocationId)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Foreign key constraint can only be created"
" on co-located tables.")));
}
/*
* Partition column must exist in both referencing and referenced side of the
* foreign key constraint. They also must be in same ordinal.
*/
referencedTablePartitionColumn = PartitionKey(referencedTableId);
}
else
{
/*
* Partition column must exist in both referencing and referenced side of the
* foreign key constraint. They also must be in same ordinal.
*/
referencedTablePartitionColumn = distributionColumn;
}
/*
* Column attributes are not available in Form_pg_constraint, therefore we need
* to find them in the system catalog. After finding them, we iterate over column
* attributes together because partition column must be at the same place in both
* referencing and referenced side of the foreign key constraint
*/
referencingColumnsDatum = SysCacheGetAttr(CONSTROID, heapTuple,
Anum_pg_constraint_conkey, &isNull);
referencedColumnsDatum = SysCacheGetAttr(CONSTROID, heapTuple,
Anum_pg_constraint_confkey, &isNull);
deconstruct_array(DatumGetArrayTypeP(referencingColumnsDatum), INT2OID, 2, true,
's', &referencingColumnArray, NULL, &referencingColumnCount);
deconstruct_array(DatumGetArrayTypeP(referencedColumnsDatum), INT2OID, 2, true,
's', &referencedColumnArray, NULL, &referencedColumnCount);
Assert(referencingColumnCount == referencedColumnCount);
for (attrIdx = 0; attrIdx < referencingColumnCount; ++attrIdx)
{
AttrNumber referencingAttrNo = DatumGetInt16(referencingColumnArray[attrIdx]);
AttrNumber referencedAttrNo = DatumGetInt16(referencedColumnArray[attrIdx]);
if (distributionColumn->varattno == referencingAttrNo &&
referencedTablePartitionColumn->varattno == referencedAttrNo)
{
foreignConstraintOnPartitionColumn = true;
}
}
if (!foreignConstraintOnPartitionColumn)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Partition column must exist both "
"referencing and referenced side of the "
"foreign constraint statement and it must "
"be in the same ordinal in both sides.")));
}
/*
* We do not allow to create foreign constraints if shard replication factor is
* greater than 1. Because in our current design, multiple replicas may cause
* locking problems and inconsistent shard contents. We don't check the referenced
* table, since referenced and referencing tables should be co-located and
* colocation check has been done above.
*/
if (IsDistributedTable(referencingTableId))
{
/* check whether ALTER TABLE command is applied over single replicated table */
if (!SingleReplicatedTable(referencingTableId))
{
singleReplicatedTable = false;
}
}
else
{
/* check whether creating single replicated table with foreign constraint */
if (ShardReplicationFactor > 1)
{
singleReplicatedTable = false;
}
}
if (!singleReplicatedTable)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Citus Community Edition currently supports "
"foreign key constraints only for "
"\"citus.shard_replication_factor = 1\"."),
errhint("Please change \"citus.shard_replication_factor to "
"1\". To learn more about using foreign keys with "
"other replication factors, please contact us at "
"https://citusdata.com/about/contact_us.")));
}
heapTuple = systable_getnext(scanDescriptor);
}
/* clean up scan and close system catalog */
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
}
/*
* ErrorIfUnsupportedSeqStmt errors out if the provided create sequence
* statement specifies a distributed table in its OWNED BY clause.

View File

@ -33,6 +33,8 @@ extern void multi_ProcessUtility(Node *parsetree, const char *queryString,
ProcessUtilityContext context, ParamListInfo params,
DestReceiver *dest, char *completionTag);
extern List * PlanGrantStmt(GrantStmt *grantStmt);
extern void ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
Var *distributionColumn, uint32 colocationId);
#endif /* MULTI_UTILITY_H */

View File

@ -0,0 +1,493 @@
--
-- MULTI_ALTER_TABLE_ADD_CONSTRAINTS
--
-- Test checks whether constraints of distributed tables can be adjusted using
-- the ALTER TABLE ... ADD CONSTRAINT ... command.
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1450000;
ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1450000;
-- Check "PRIMARY KEY CONSTRAINT"
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
create_distributed_table
--------------------------
(1 row)
-- Can only add primary key constraint on distribution column (or group of columns
-- including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(name);
ERROR: cannot create constraint on "products"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(product_no);
NOTICE: using one-phase commit for distributed DDL commands
HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc'
INSERT INTO products VALUES(1, 'product_1', 1);
-- Should error out, since we are trying to add a new row having a value on p_key column
-- conflicting with the existing row.
INSERT INTO products VALUES(1, 'product_1', 1);
ERROR: duplicate key value violates unique constraint "p_key_1450001"
DETAIL: Key (product_no)=(1) already exists.
CONTEXT: while executing command on localhost:57638
ALTER TABLE products DROP CONSTRAINT p_key;
INSERT INTO products VALUES(1, 'product_1', 1);
-- Can not create constraint since it conflicts with the existing data
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(product_no);
ERROR: could not create unique index "p_key_1450001"
DETAIL: Key (product_no)=(1) is duplicated.
CONTEXT: while executing command on localhost:57637
DROP TABLE products;
-- Check "PRIMARY KEY CONSTRAINT" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric
);
SELECT create_reference_table('products_ref');
create_reference_table
------------------------
(1 row)
-- Can add PRIMARY KEY to any column
ALTER TABLE products_ref ADD CONSTRAINT p_key PRIMARY KEY(name);
ALTER TABLE products_ref DROP CONSTRAINT p_key;
ALTER TABLE products_ref ADD CONSTRAINT p_key PRIMARY KEY(product_no);
INSERT INTO products_ref VALUES(1, 'product_1', 1);
-- Should error out, since we are trying to add new row having a value on p_key column
-- conflicting with the existing row.
INSERT INTO products_ref VALUES(1, 'product_1', 1);
ERROR: duplicate key value violates unique constraint "p_key_1450032"
DETAIL: Key (product_no)=(1) already exists.
CONTEXT: while executing command on localhost:57637
DROP TABLE products_ref;
-- Check "PRIMARY KEY CONSTRAINT" on append table
CREATE TABLE products_append (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products_append', 'product_no', 'append');
create_distributed_table
--------------------------
(1 row)
-- Can only add primary key constraint on distribution column (or group
-- of columns including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products_append ADD CONSTRAINT p_key_name PRIMARY KEY(name);
WARNING: table "products_append" has a UNIQUE or EXCLUDE constraint
DETAIL: UNIQUE constraints, EXCLUDE constraints, and PRIMARY KEYs on append-partitioned tables cannot be enforced.
HINT: Consider using hash partitioning.
ERROR: cannot create constraint on "products_append"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
ALTER TABLE products_append ADD CONSTRAINT p_key PRIMARY KEY(product_no);
WARNING: table "products_append" has a UNIQUE or EXCLUDE constraint
DETAIL: UNIQUE constraints, EXCLUDE constraints, and PRIMARY KEYs on append-partitioned tables cannot be enforced.
HINT: Consider using hash partitioning.
--- Error out since first and third rows have the same product_no
\COPY products_append FROM STDIN DELIMITER AS ',';
ERROR: duplicate key value violates unique constraint "p_key_1450033"
DETAIL: Key (product_no)=(1) already exists.
DROP TABLE products_append;
-- Check "UNIQUE CONSTRAINT"
CREATE TABLE unique_test_table(id int, name varchar(20));
SELECT create_distributed_table('unique_test_table', 'id');
create_distributed_table
--------------------------
(1 row)
-- Can only add unique constraint on distribution column (or group
-- of columns including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE unique_test_table ADD CONSTRAINT unn_name UNIQUE(name);
ERROR: cannot create constraint on "unique_test_table"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
ALTER TABLE unique_test_table ADD CONSTRAINT unn_id UNIQUE(id);
-- Error out, since table can not have two rows with same id.
INSERT INTO unique_test_table VALUES(1, 'Ahmet');
INSERT INTO unique_test_table VALUES(1, 'Mehmet');
ERROR: duplicate key value violates unique constraint "unn_id_1450035"
DETAIL: Key (id)=(1) already exists.
CONTEXT: while executing command on localhost:57638
ALTER TABLE unique_test_table DROP CONSTRAINT unn_id;
-- Insert row which will conflict with the next unique constraint command
INSERT INTO unique_test_table VALUES(1, 'Mehmet');
-- Can not create constraint since it conflicts with the existing data
ALTER TABLE unique_test_table ADD CONSTRAINT unn_id UNIQUE(id);
ERROR: could not create unique index "unn_id_1450035"
DETAIL: Key (id)=(1) is duplicated.
CONTEXT: while executing command on localhost:57637
-- Can create unique constraint over multiple columns which must include
-- distribution column
ALTER TABLE unique_test_table ADD CONSTRAINT unn_id_name UNIQUE(id, name);
-- Error out, since tables can not have two rows with same id and name.
INSERT INTO unique_test_table VALUES(1, 'Mehmet');
ERROR: duplicate key value violates unique constraint "unn_id_name_1450035"
DETAIL: Key (id, name)=(1, Mehmet) already exists.
CONTEXT: while executing command on localhost:57638
DROP TABLE unique_test_table;
-- Check "UNIQUE CONSTRAINT" with reference table
CREATE TABLE unique_test_table_ref(id int, name varchar(20));
SELECT create_reference_table('unique_test_table_ref');
create_reference_table
------------------------
(1 row)
-- We can add unique constraint on any column with reference tables
ALTER TABLE unique_test_table_ref ADD CONSTRAINT unn_name UNIQUE(name);
ALTER TABLE unique_test_table_ref ADD CONSTRAINT unn_id UNIQUE(id);
-- Error out. Since the table can not have two rows with the same id.
INSERT INTO unique_test_table_ref VALUES(1, 'Ahmet');
INSERT INTO unique_test_table_ref VALUES(1, 'Mehmet');
ERROR: duplicate key value violates unique constraint "unn_id_1450066"
DETAIL: Key (id)=(1) already exists.
CONTEXT: while executing command on localhost:57637
-- We can add unique constraint with multiple columns
ALTER TABLE unique_test_table_ref DROP CONSTRAINT unn_id;
ALTER TABLE unique_test_table_ref ADD CONSTRAINT unn_id_name UNIQUE(id,name);
-- Error out, since two rows can not have the same id or name.
INSERT INTO unique_test_table_ref VALUES(1, 'Mehmet');
DROP TABLE unique_test_table_ref;
-- Check "UNIQUE CONSTRAINT" with append table
CREATE TABLE unique_test_table_append(id int, name varchar(20));
SELECT create_distributed_table('unique_test_table_append', 'id', 'append');
create_distributed_table
--------------------------
(1 row)
-- Can only add unique constraint on distribution column (or group
-- of columns including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE unique_test_table_append ADD CONSTRAINT unn_name UNIQUE(name);
WARNING: table "unique_test_table_append" has a UNIQUE or EXCLUDE constraint
DETAIL: UNIQUE constraints, EXCLUDE constraints, and PRIMARY KEYs on append-partitioned tables cannot be enforced.
HINT: Consider using hash partitioning.
ERROR: cannot create constraint on "unique_test_table_append"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
ALTER TABLE unique_test_table_append ADD CONSTRAINT unn_id UNIQUE(id);
WARNING: table "unique_test_table_append" has a UNIQUE or EXCLUDE constraint
DETAIL: UNIQUE constraints, EXCLUDE constraints, and PRIMARY KEYs on append-partitioned tables cannot be enforced.
HINT: Consider using hash partitioning.
-- Error out. Table can not have two rows with the same id.
\COPY unique_test_table_append FROM STDIN DELIMITER AS ',';
ERROR: duplicate key value violates unique constraint "unn_id_1450067"
DETAIL: Key (id)=(1) already exists.
DROP TABLE unique_test_table_append;
-- Check "CHECK CONSTRAINT"
CREATE TABLE products (
product_no integer,
name text,
price numeric,
discounted_price numeric
);
SELECT create_distributed_table('products', 'product_no');
create_distributed_table
--------------------------
(1 row)
-- Can add column and table check constraints
ALTER TABLE products ADD CONSTRAINT p_check CHECK(price > 0);
ALTER TABLE products ADD CONSTRAINT p_multi_check CHECK(price > discounted_price);
-- First and third queries will error out, because of conflicts with p_check and
-- p_multi_check, respectively.
INSERT INTO products VALUES(1, 'product_1', -1, -2);
ERROR: new row for relation "products_1450069" violates check constraint "p_check_1450069"
DETAIL: Failing row contains (1, product_1, -1, -2).
CONTEXT: while executing command on localhost:57638
INSERT INTO products VALUES(1, 'product_1', 5, 3);
INSERT INTO products VALUES(1, 'product_1', 2, 3);
ERROR: new row for relation "products_1450069" violates check constraint "p_multi_check_1450069"
DETAIL: Failing row contains (1, product_1, 2, 3).
CONTEXT: while executing command on localhost:57638
DROP TABLE products;
-- Check "CHECK CONSTRAINT" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric,
discounted_price numeric
);
SELECT create_reference_table('products_ref');
create_reference_table
------------------------
(1 row)
-- Can add column and table check constraints
ALTER TABLE products_ref ADD CONSTRAINT p_check CHECK(price > 0);
ALTER TABLE products_ref ADD CONSTRAINT p_multi_check CHECK(price > discounted_price);
-- First and third queries will error out, because of conflicts with p_check and
-- p_multi_check, respectively.
INSERT INTO products_ref VALUES(1, 'product_1', -1, -2);
ERROR: new row for relation "products_ref_1450100" violates check constraint "p_check_1450100"
DETAIL: Failing row contains (1, product_1, -1, -2).
CONTEXT: while executing command on localhost:57637
INSERT INTO products_ref VALUES(1, 'product_1', 5, 3);
INSERT INTO products_ref VALUES(1, 'product_1', 2, 3);
ERROR: new row for relation "products_ref_1450100" violates check constraint "p_multi_check_1450100"
DETAIL: Failing row contains (1, product_1, 2, 3).
CONTEXT: while executing command on localhost:57637
DROP TABLE products_ref;
-- Check "CHECK CONSTRAINT" with append table
CREATE TABLE products_append (
product_no int,
name varchar(20),
price int,
discounted_price int
);
SELECT create_distributed_table('products_append', 'product_no', 'append');
create_distributed_table
--------------------------
(1 row)
-- Can add column and table check constraints
ALTER TABLE products_append ADD CONSTRAINT p_check CHECK(price > 0);
ALTER TABLE products_append ADD CONSTRAINT p_multi_check CHECK(price > discounted_price);
-- Error out,since the third row conflicting with the p_multi_check
\COPY products_append FROM STDIN DELIMITER AS ',';
ERROR: new row for relation "products_append_1450101" violates check constraint "p_multi_check"
DETAIL: Failing row contains (1, Product_3, 8, 10).
DROP TABLE products_append;
-- Check "EXCLUSION CONSTRAINT"
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
create_distributed_table
--------------------------
(1 row)
-- Can only add exclusion constraint on distribution column (or group of columns
-- including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products ADD CONSTRAINT exc_name EXCLUDE USING btree (name with =);
ERROR: cannot create constraint on "products"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
-- We can add composite exclusion
ALTER TABLE products ADD CONSTRAINT exc_pno_name EXCLUDE USING btree (product_no with =, name with =);
-- 4th command will error out since it conflicts with exc_pno_name constraint
INSERT INTO products VALUES(1,'product_1', 5);
INSERT INTO products VALUES(1,'product_2', 10);
INSERT INTO products VALUES(2,'product_2', 5);
INSERT INTO products VALUES(2,'product_2', 5);
ERROR: conflicting key value violates exclusion constraint "exc_pno_name_1450126"
DETAIL: Key (product_no, name)=(2, product_2) conflicts with existing key (product_no, name)=(2, product_2).
CONTEXT: while executing command on localhost:57637
DROP TABLE products;
-- Check "EXCLUSION CONSTRAINT" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric
);
SELECT create_reference_table('products_ref');
create_reference_table
------------------------
(1 row)
-- We can add exclusion constraint on any column
ALTER TABLE products_ref ADD CONSTRAINT exc_name EXCLUDE USING btree (name with =);
-- We can add composite exclusion because none of pair of rows are conflicting
ALTER TABLE products_ref ADD CONSTRAINT exc_pno_name EXCLUDE USING btree (product_no with =, name with =);
-- Third insertion will error out, since it has the same name with second insertion
INSERT INTO products_ref VALUES(1,'product_1', 5);
INSERT INTO products_ref VALUES(1,'product_2', 10);
INSERT INTO products_ref VALUES(2,'product_2', 5);
ERROR: conflicting key value violates exclusion constraint "exc_name_1450134"
DETAIL: Key (name)=(product_2) conflicts with existing key (name)=(product_2).
CONTEXT: while executing command on localhost:57637
DROP TABLE products_ref;
-- Check "EXCLUSION CONSTRAINT" with append table
CREATE TABLE products_append (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products_append', 'product_no','append');
create_distributed_table
--------------------------
(1 row)
-- Can only add exclusion constraint on distribution column (or group of column
-- including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products_append ADD CONSTRAINT exc_name EXCLUDE USING btree (name with =);
WARNING: table "products_append" has a UNIQUE or EXCLUDE constraint
DETAIL: UNIQUE constraints, EXCLUDE constraints, and PRIMARY KEYs on append-partitioned tables cannot be enforced.
HINT: Consider using hash partitioning.
ERROR: cannot create constraint on "products_append"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
ALTER TABLE products_append ADD CONSTRAINT exc_pno_name EXCLUDE USING btree (product_no with =, name with =);
WARNING: table "products_append" has a UNIQUE or EXCLUDE constraint
DETAIL: UNIQUE constraints, EXCLUDE constraints, and PRIMARY KEYs on append-partitioned tables cannot be enforced.
HINT: Consider using hash partitioning.
-- Error out since first and third can not pass the exclusion check.
\COPY products_append FROM STDIN DELIMITER AS ',';
ERROR: conflicting key value violates exclusion constraint "exc_pno_name_1450135"
DETAIL: Key (product_no, name)=(1, Product_1) conflicts with existing key (product_no, name)=(1, Product_1).
DROP TABLE products_append;
-- Check "NOT NULL"
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
create_distributed_table
--------------------------
(1 row)
ALTER TABLE products ALTER COLUMN name SET NOT NULL;
-- Insertions will error out since both product_no and name can not have NULL value
INSERT INTO products VALUES(1,NULL,5);
ERROR: null value in column "name" violates not-null constraint
DETAIL: Failing row contains (1, null, 5).
CONTEXT: while executing command on localhost:57638
INSERT INTO products VALUES(NULL,'product_1', 5);
ERROR: cannot plan INSERT using row with NULL value in partition column
DROP TABLE products;
-- Check "NOT NULL" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric
);
SELECT create_reference_table('products_ref');
create_reference_table
------------------------
(1 row)
ALTER TABLE products_ref ALTER COLUMN name SET NOT NULL;
-- Insertions will error out since both product_no and name can not have NULL value
INSERT INTO products_ref VALUES(1,NULL,5);
ERROR: null value in column "name" violates not-null constraint
DETAIL: Failing row contains (1, null, 5).
CONTEXT: while executing command on localhost:57637
INSERT INTO products_ref VALUES(NULL,'product_1', 5);
DROP TABLE products_ref;
-- Check "NOT NULL" with append table
CREATE TABLE products_append (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products_append', 'product_no', 'append');
create_distributed_table
--------------------------
(1 row)
ALTER TABLE products_append ALTER COLUMN name SET NOT NULL;
-- Error out since name and product_no columns can not handle NULL value.
\COPY products_append FROM STDIN DELIMITER AS ',';
DROP TABLE products_append;
-- Tests for ADD CONSTRAINT is not only subcommand
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
create_distributed_table
--------------------------
(1 row)
-- Should error out since add constraint is not the single subcommand
ALTER TABLE products ADD CONSTRAINT unn_1 UNIQUE(product_no, price), ADD CONSTRAINT unn_2 UNIQUE(product_no, name);
ERROR: cannot execute ADD CONSTRAINT command with other subcommands
HINT: You can issue each subcommand separately
-- Tests for constraints without name
-- Commands below should error out since constraints do not have the name
ALTER TABLE products ADD UNIQUE(product_no);
ERROR: cannot create constraint without a name on a distributed table
ALTER TABLE products ADD PRIMARY KEY(product_no);
ERROR: cannot create constraint without a name on a distributed table
ALTER TABLE products ADD CHECK(product_no <> 0);
ERROR: cannot create constraint without a name on a distributed table
ALTER TABLE products ADD EXCLUDE USING btree (product_no with =);
ERROR: cannot create constraint without a name on a distributed table
DROP TABLE products;
-- Tests with transactions
CREATE TABLE products (
product_no integer,
name text,
price numeric,
discounted_price numeric
);
SELECT create_distributed_table('products', 'product_no');
create_distributed_table
--------------------------
(1 row)
BEGIN;
INSERT INTO products VALUES(1,'product_1', 5);
-- Should error out since conflicts with the above single-shard data modification command.
ALTER TABLE products ADD CONSTRAINT unn_pno UNIQUE(product_no);
ERROR: distributed DDL commands must not appear within transaction blocks containing single-shard data modifications
ROLLBACK;
BEGIN;
-- Add constraints
ALTER TABLE products ADD CONSTRAINT unn_pno UNIQUE(product_no);
ALTER TABLE products ADD CONSTRAINT check_price CHECK(price > discounted_price);
ALTER TABLE products ALTER COLUMN product_no SET NOT NULL;
ALTER TABLE products ADD CONSTRAINT p_key_product PRIMARY KEY(product_no);
-- Single shard DML command can't be located in the same transaction with above commands.
INSERT INTO products VALUES(1,'product_1', 10, 8);
ERROR: single-shard DML commands must not appear in transaction blocks which contain multi-shard data modifications
ROLLBACK;
-- There should be no constraint on master and worker(s)
\d products
Table "public.products"
Column | Type | Modifiers
------------------+---------+-----------
product_no | integer |
name | text |
price | numeric |
discounted_price | numeric |
\c - - - :worker_1_port
\d products_1450199
\c - - - :master_port
-- Tests to check the effect of rollback
BEGIN;
-- Add constraints (which will be rollbacked)
ALTER TABLE products ADD CONSTRAINT unn_pno UNIQUE(product_no);
NOTICE: using one-phase commit for distributed DDL commands
HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc'
ALTER TABLE products ADD CONSTRAINT check_price CHECK(price > discounted_price);
ALTER TABLE products ADD CONSTRAINT p_key_product PRIMARY KEY(product_no);
ROLLBACK;
-- There should be no constraint on master and worker(s)
\d products
Table "public.products"
Column | Type | Modifiers
------------------+---------+-----------
product_no | integer |
name | text |
price | numeric |
discounted_price | numeric |
\c - - - :worker_1_port
\d products_1450199
\c - - - :master_port
DROP TABLE products;

View File

@ -438,21 +438,21 @@ FROM
WHERE
li.l_quantity > 25
ORDER BY
li.l_quantity
li.l_quantity, li.l_partkey, o.o_custkey
LIMIT 10 OFFSET 20;
DEBUG: push down of limit count: 30
l_partkey | o_custkey | l_quantity
-----------+-----------+------------
154380 | 421 | 26.00
103981 | 325 | 26.00
77886 | 817 | 26.00
147369 | 755 | 26.00
78175 | 1075 | 26.00
109784 | 1268 | 26.00
25221 | 656 | 26.00
25373 | 1369 | 26.00
27331 | 571 | 26.00
27699 | 1150 | 26.00
28226 | 913 | 26.00
28635 | 1207 | 26.00
188845 | 554 | 26.00
189398 | 844 | 26.00
71383 | 1397 | 26.00
29101 | 1283 | 26.00
31143 | 640 | 26.00
31239 | 685 | 26.00
33646 | 860 | 26.00
(10 rows)
RESET client_min_messages;

View File

@ -40,7 +40,7 @@ CREATE TABLE pk_on_non_part_col
other_col integer PRIMARY KEY
);
SELECT master_create_distributed_table('pk_on_non_part_col', 'partition_col', 'hash');
ERROR: cannot distribute relation: "pk_on_non_part_col"
ERROR: cannot create constraint on "pk_on_non_part_col"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
CREATE TABLE uq_on_non_part_col
(
@ -48,7 +48,7 @@ CREATE TABLE uq_on_non_part_col
other_col integer UNIQUE
);
SELECT master_create_distributed_table('uq_on_non_part_col', 'partition_col', 'hash');
ERROR: cannot distribute relation: "uq_on_non_part_col"
ERROR: cannot create constraint on "uq_on_non_part_col"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
CREATE TABLE ex_on_non_part_col
(
@ -57,7 +57,7 @@ CREATE TABLE ex_on_non_part_col
EXCLUDE (other_col WITH =)
);
SELECT master_create_distributed_table('ex_on_non_part_col', 'partition_col', 'hash');
ERROR: cannot distribute relation: "ex_on_non_part_col"
ERROR: cannot create constraint on "ex_on_non_part_col"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
-- now show that Citus can distribute unique and EXCLUDE constraints that
-- include the partition column for hash-partitioned tables.
@ -187,7 +187,7 @@ CREATE TABLE ex_wrong_operator
EXCLUDE USING gist (other_col WITH =, partition_col WITH &&)
);
SELECT master_create_distributed_table('ex_wrong_operator', 'partition_col', 'hash');
ERROR: cannot distribute relation: "ex_wrong_operator"
ERROR: cannot create constraint on "ex_wrong_operator"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
CREATE TABLE ex_overlaps
(
@ -344,7 +344,7 @@ CREATE TABLE ex_wrong_operator_named
CONSTRAINT ex_wrong_operator_named_exclude EXCLUDE USING gist (other_col WITH =, partition_col WITH &&)
);
SELECT master_create_distributed_table('ex_wrong_operator_named', 'partition_col', 'hash');
ERROR: cannot distribute relation: "ex_wrong_operator_named"
ERROR: cannot create constraint on "ex_wrong_operator_named"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
CREATE TABLE ex_overlaps_named
(

View File

@ -433,13 +433,11 @@ DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operatio
-- test foreign constraint creation with multiple subcommands
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id),
ADD CONSTRAINT test_constraint FOREIGN KEY(id) REFERENCES referenced_table(test_column);
ERROR: cannot create foreign key constraint
DETAIL: Citus cannot execute ADD CONSTRAINT FOREIGN KEY command together with other subcommands.
ERROR: cannot execute ADD CONSTRAINT command with other subcommands
HINT: You can issue each subcommand separately
-- test foreign constraint creation without giving explicit name
ALTER TABLE referencing_table ADD FOREIGN KEY(ref_id) REFERENCES referenced_table(id);
ERROR: cannot create foreign key constraint
DETAIL: Creating foreign constraint without a name on a distributed table is currently not supported.
ERROR: cannot create constraint without a name on a distributed table
-- test foreign constraint creation on NOT co-located tables
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id);
ERROR: cannot create foreign key constraint
@ -463,8 +461,7 @@ SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
-- columns for the referenced table is empty
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table ON DELETE CASCADE;
ERROR: cannot create foreign key constraint because referenced column list is empty
HINT: Add column names to "REFERENCES" part of the statement.
ERROR: number of referencing and referenced columns for foreign key disagree
-- test foreign constraint creation on non-partition columns
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(id) REFERENCES referenced_table(id);
ERROR: cannot create foreign key constraint
@ -475,8 +472,7 @@ ERROR: cannot create foreign key constraint
DETAIL: Partition column must exist both referencing and referenced side of the foreign constraint statement and it must be in the same ordinal in both sides.
-- test foreign constraint creation while column list are not in same length
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id, test_column);
ERROR: cannot create foreign key constraint
DETAIL: Referencing column list and referenced column list must be in same size.
ERROR: number of referencing and referenced columns for foreign key disagree
-- test foreign constraint creation while existing tables does not satisfy the constraint
INSERT INTO referencing_table VALUES(1, 1);
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id);
@ -794,8 +790,7 @@ SELECT create_distributed_table('referenced_by_reference_table', 'id');
CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_by_reference_table(id));
SELECT create_reference_table('reference_table');
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on CREATE TABLE to reference table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
@ -807,13 +802,11 @@ SELECT create_reference_table('reference_table');
CREATE TABLE references_to_reference_table(id int, referencing_column int REFERENCES reference_table(id));
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on CREATE TABLE from + to reference table
CREATE TABLE reference_table_second(id int, referencing_column int REFERENCES reference_table(id));
SELECT create_reference_table('reference_table_second');
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on CREATE TABLE from reference table to local table
CREATE TABLE referenced_local_table(id int PRIMARY KEY, other_column int);
DROP TABLE reference_table CASCADE;
@ -822,8 +815,7 @@ DETAIL: drop cascades to constraint references_to_reference_table_referencing_c
drop cascades to constraint reference_table_second_referencing_column_fkey on table reference_table_second
CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_local_table(id));
SELECT create_reference_table('reference_table');
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on CREATE TABLE on self referencing reference table
CREATE TABLE self_referencing_reference_table(
id int,
@ -833,8 +825,7 @@ CREATE TABLE self_referencing_reference_table(
FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_reference_table(id, other_column)
);
SELECT create_reference_table('self_referencing_reference_table');
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on ALTER TABLE from reference table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
@ -845,8 +836,7 @@ SELECT create_reference_table('reference_table');
(1 row)
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_by_reference_table(id);
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on ALTER TABLE to reference table
DROP TABLE references_to_reference_table;
CREATE TABLE references_to_reference_table(id int, referencing_column int);
@ -857,8 +847,7 @@ SELECT create_distributed_table('references_to_reference_table', 'referencing_co
(1 row)
ALTER TABLE references_to_reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES reference_table(id);
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on ALTER TABLE from + to reference table
DROP TABLE reference_table_second;
CREATE TABLE reference_table_second(id int, referencing_column int);
@ -869,8 +858,7 @@ SELECT create_reference_table('reference_table_second');
(1 row)
ALTER TABLE reference_table_second ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES reference_table(id);
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on ALTER TABLE from reference table to local table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
@ -881,8 +869,7 @@ SELECT create_reference_table('reference_table');
(1 row)
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_local_table(id);
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: relation referenced_local_table is not distributed
-- test foreign key creation on ALTER TABLE on self referencing reference table
DROP TABLE self_referencing_reference_table;
CREATE TABLE self_referencing_reference_table(
@ -898,7 +885,6 @@ SELECT create_reference_table('self_referencing_reference_table');
(1 row)
ALTER TABLE self_referencing_reference_table ADD CONSTRAINT fk FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_reference_table(id, other_column);
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraints are not allowed from or to reference tables.
ERROR: cannot create foreign key constraint from or to reference tables
-- we no longer need those tables
DROP TABLE referenced_by_reference_table, references_to_reference_table, reference_table, reference_table_second, referenced_local_table, self_referencing_reference_table;

View File

@ -1226,11 +1226,13 @@ FROM
NATURAL JOIN pg_dist_shard
NATURAL JOIN pg_dist_shard_placement
WHERE
logicalrelid = 'mx_ref'::regclass;
logicalrelid = 'mx_ref'::regclass
ORDER BY
nodeport;
logicalrelid | partmethod | repmodel | shardid | placementid | nodename | nodeport
--------------+------------+----------+---------+-------------+-----------+----------
mx_ref | n | t | 1310183 | 100184 | localhost | 57638
mx_ref | n | t | 1310183 | 100183 | localhost | 57637
mx_ref | n | t | 1310183 | 100184 | localhost | 57638
(2 rows)

View File

@ -77,14 +77,11 @@ ALTER TABLE name_lengths ADD COLUMN date_col_12345678901234567890123456789012345
ALTER TABLE name_lengths ADD COLUMN int_col_12345678901234567890123456789012345678901234567890 INTEGER DEFAULT 1;
-- Placeholders for unsupported ALTER TABLE to add constraints with implicit names that are likely too long
ALTER TABLE name_lengths ADD UNIQUE (float_col_12345678901234567890123456789012345678901234567890);
ERROR: cannot create constraint
DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
ERROR: cannot create constraint without a name on a distributed table
ALTER TABLE name_lengths ADD EXCLUDE (int_col_12345678901234567890123456789012345678901234567890 WITH =);
ERROR: cannot create constraint
DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
ERROR: cannot create constraint without a name on a distributed table
ALTER TABLE name_lengths ADD CHECK (date_col_12345678901234567890123456789012345678901234567890 > '2014-01-01'::date);
ERROR: cannot create constraint
DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
ERROR: cannot create constraint without a name on a distributed table
\c - - - :worker_1_port
\d name_lengths_*
Table "public.name_lengths_225002"
@ -112,14 +109,14 @@ Indexes:
\c - - - :master_port
-- Placeholders for unsupported add constraints with EXPLICIT names that are too long
ALTER TABLE name_lengths ADD CONSTRAINT nl_unique_12345678901234567890123456789012345678901234567890 UNIQUE (float_col_12345678901234567890123456789012345678901234567890);
ERROR: cannot create constraint
DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
ERROR: cannot create constraint on "name_lengths"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
ALTER TABLE name_lengths ADD CONSTRAINT nl_exclude_12345678901234567890123456789012345678901234567890 EXCLUDE (int_col_12345678901234567890123456789012345678901234567890 WITH =);
ERROR: cannot create constraint
DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
ERROR: cannot create constraint on "name_lengths"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
ALTER TABLE name_lengths ADD CONSTRAINT nl_checky_12345678901234567890123456789012345678901234567890 CHECK (date_col_12345678901234567890123456789012345678901234567890 >= '2014-01-01'::date);
ERROR: cannot create constraint
DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
NOTICE: using one-phase commit for distributed DDL commands
HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc'
\c - - - :worker_1_port
\d nl_*
\c - - - :master_port

View File

@ -1362,7 +1362,7 @@ ALTER TABLE reference_table_ddl RENAME TO reference_table_ddl_test;
ERROR: renaming distributed tables is currently unsupported
ALTER TABLE reference_table_ddl SET WITH OIDS;
ERROR: alter table command is currently unsupported
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN KEY and TYPE subcommands are supported.
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT and TYPE subcommands are supported.
-- now test reference tables against some helper UDFs that Citus provides
-- cannot delete / drop shards from a reference table
SELECT master_apply_delete_command('DELETE FROM reference_table_ddl');

View File

@ -96,6 +96,7 @@ test: multi_shard_modify
# ----------
test: multi_index_statements
test: multi_alter_table_statements
test: multi_alter_table_add_constraints
# ----------
# multi_create_schema tests creation, loading, and querying of a table in a new

View File

@ -332,7 +332,7 @@ ALTER TABLE lineitem_alter ADD COLUMN int_column1 INTEGER,
ALTER TABLE lineitem_alter ADD COLUMN int_column3 INTEGER,
ALTER COLUMN int_column1 SET STATISTICS 10;
ERROR: alter table command is currently unsupported
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN KEY and TYPE subcommands are supported.
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT and TYPE subcommands are supported.
ALTER TABLE lineitem_alter DROP COLUMN int_column1, DROP COLUMN int_column2;
\d lineitem_alter
Table "public.lineitem_alter"
@ -364,12 +364,12 @@ ERROR: cannot execute ALTER TABLE command involving partition column
-- Verify that we error out on unsupported statement types
ALTER TABLE lineitem_alter ALTER COLUMN l_orderkey SET STATISTICS 100;
ERROR: alter table command is currently unsupported
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN KEY and TYPE subcommands are supported.
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT and TYPE subcommands are supported.
ALTER TABLE lineitem_alter DROP CONSTRAINT IF EXISTS non_existent_contraint;
NOTICE: constraint "non_existent_contraint" of relation "lineitem_alter" does not exist, skipping
ALTER TABLE lineitem_alter SET WITHOUT OIDS;
ERROR: alter table command is currently unsupported
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN KEY and TYPE subcommands are supported.
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT and TYPE subcommands are supported.
-- Verify that we error out in case of postgres errors on supported statement
-- types
ALTER TABLE lineitem_alter ADD COLUMN new_column non_existent_type;

View File

@ -0,0 +1,425 @@
--
-- MULTI_ALTER_TABLE_ADD_CONSTRAINTS
--
-- Test checks whether constraints of distributed tables can be adjusted using
-- the ALTER TABLE ... ADD CONSTRAINT ... command.
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1450000;
ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1450000;
-- Check "PRIMARY KEY CONSTRAINT"
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
-- Can only add primary key constraint on distribution column (or group of columns
-- including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(name);
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(product_no);
INSERT INTO products VALUES(1, 'product_1', 1);
-- Should error out, since we are trying to add a new row having a value on p_key column
-- conflicting with the existing row.
INSERT INTO products VALUES(1, 'product_1', 1);
ALTER TABLE products DROP CONSTRAINT p_key;
INSERT INTO products VALUES(1, 'product_1', 1);
-- Can not create constraint since it conflicts with the existing data
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(product_no);
DROP TABLE products;
-- Check "PRIMARY KEY CONSTRAINT" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric
);
SELECT create_reference_table('products_ref');
-- Can add PRIMARY KEY to any column
ALTER TABLE products_ref ADD CONSTRAINT p_key PRIMARY KEY(name);
ALTER TABLE products_ref DROP CONSTRAINT p_key;
ALTER TABLE products_ref ADD CONSTRAINT p_key PRIMARY KEY(product_no);
INSERT INTO products_ref VALUES(1, 'product_1', 1);
-- Should error out, since we are trying to add new row having a value on p_key column
-- conflicting with the existing row.
INSERT INTO products_ref VALUES(1, 'product_1', 1);
DROP TABLE products_ref;
-- Check "PRIMARY KEY CONSTRAINT" on append table
CREATE TABLE products_append (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products_append', 'product_no', 'append');
-- Can only add primary key constraint on distribution column (or group
-- of columns including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products_append ADD CONSTRAINT p_key_name PRIMARY KEY(name);
ALTER TABLE products_append ADD CONSTRAINT p_key PRIMARY KEY(product_no);
--- Error out since first and third rows have the same product_no
\COPY products_append FROM STDIN DELIMITER AS ',';
1, Product_1, 10
2, Product_2, 15
1, Product_3, 8
\.
DROP TABLE products_append;
-- Check "UNIQUE CONSTRAINT"
CREATE TABLE unique_test_table(id int, name varchar(20));
SELECT create_distributed_table('unique_test_table', 'id');
-- Can only add unique constraint on distribution column (or group
-- of columns including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE unique_test_table ADD CONSTRAINT unn_name UNIQUE(name);
ALTER TABLE unique_test_table ADD CONSTRAINT unn_id UNIQUE(id);
-- Error out, since table can not have two rows with same id.
INSERT INTO unique_test_table VALUES(1, 'Ahmet');
INSERT INTO unique_test_table VALUES(1, 'Mehmet');
ALTER TABLE unique_test_table DROP CONSTRAINT unn_id;
-- Insert row which will conflict with the next unique constraint command
INSERT INTO unique_test_table VALUES(1, 'Mehmet');
-- Can not create constraint since it conflicts with the existing data
ALTER TABLE unique_test_table ADD CONSTRAINT unn_id UNIQUE(id);
-- Can create unique constraint over multiple columns which must include
-- distribution column
ALTER TABLE unique_test_table ADD CONSTRAINT unn_id_name UNIQUE(id, name);
-- Error out, since tables can not have two rows with same id and name.
INSERT INTO unique_test_table VALUES(1, 'Mehmet');
DROP TABLE unique_test_table;
-- Check "UNIQUE CONSTRAINT" with reference table
CREATE TABLE unique_test_table_ref(id int, name varchar(20));
SELECT create_reference_table('unique_test_table_ref');
-- We can add unique constraint on any column with reference tables
ALTER TABLE unique_test_table_ref ADD CONSTRAINT unn_name UNIQUE(name);
ALTER TABLE unique_test_table_ref ADD CONSTRAINT unn_id UNIQUE(id);
-- Error out. Since the table can not have two rows with the same id.
INSERT INTO unique_test_table_ref VALUES(1, 'Ahmet');
INSERT INTO unique_test_table_ref VALUES(1, 'Mehmet');
-- We can add unique constraint with multiple columns
ALTER TABLE unique_test_table_ref DROP CONSTRAINT unn_id;
ALTER TABLE unique_test_table_ref ADD CONSTRAINT unn_id_name UNIQUE(id,name);
-- Error out, since two rows can not have the same id or name.
INSERT INTO unique_test_table_ref VALUES(1, 'Mehmet');
DROP TABLE unique_test_table_ref;
-- Check "UNIQUE CONSTRAINT" with append table
CREATE TABLE unique_test_table_append(id int, name varchar(20));
SELECT create_distributed_table('unique_test_table_append', 'id', 'append');
-- Can only add unique constraint on distribution column (or group
-- of columns including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE unique_test_table_append ADD CONSTRAINT unn_name UNIQUE(name);
ALTER TABLE unique_test_table_append ADD CONSTRAINT unn_id UNIQUE(id);
-- Error out. Table can not have two rows with the same id.
\COPY unique_test_table_append FROM STDIN DELIMITER AS ',';
1, Product_1
2, Product_2
1, Product_3
\.
DROP TABLE unique_test_table_append;
-- Check "CHECK CONSTRAINT"
CREATE TABLE products (
product_no integer,
name text,
price numeric,
discounted_price numeric
);
SELECT create_distributed_table('products', 'product_no');
-- Can add column and table check constraints
ALTER TABLE products ADD CONSTRAINT p_check CHECK(price > 0);
ALTER TABLE products ADD CONSTRAINT p_multi_check CHECK(price > discounted_price);
-- First and third queries will error out, because of conflicts with p_check and
-- p_multi_check, respectively.
INSERT INTO products VALUES(1, 'product_1', -1, -2);
INSERT INTO products VALUES(1, 'product_1', 5, 3);
INSERT INTO products VALUES(1, 'product_1', 2, 3);
DROP TABLE products;
-- Check "CHECK CONSTRAINT" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric,
discounted_price numeric
);
SELECT create_reference_table('products_ref');
-- Can add column and table check constraints
ALTER TABLE products_ref ADD CONSTRAINT p_check CHECK(price > 0);
ALTER TABLE products_ref ADD CONSTRAINT p_multi_check CHECK(price > discounted_price);
-- First and third queries will error out, because of conflicts with p_check and
-- p_multi_check, respectively.
INSERT INTO products_ref VALUES(1, 'product_1', -1, -2);
INSERT INTO products_ref VALUES(1, 'product_1', 5, 3);
INSERT INTO products_ref VALUES(1, 'product_1', 2, 3);
DROP TABLE products_ref;
-- Check "CHECK CONSTRAINT" with append table
CREATE TABLE products_append (
product_no int,
name varchar(20),
price int,
discounted_price int
);
SELECT create_distributed_table('products_append', 'product_no', 'append');
-- Can add column and table check constraints
ALTER TABLE products_append ADD CONSTRAINT p_check CHECK(price > 0);
ALTER TABLE products_append ADD CONSTRAINT p_multi_check CHECK(price > discounted_price);
-- Error out,since the third row conflicting with the p_multi_check
\COPY products_append FROM STDIN DELIMITER AS ',';
1, Product_1, 10, 5
2, Product_2, 15, 8
1, Product_3, 8, 10
\.
DROP TABLE products_append;
-- Check "EXCLUSION CONSTRAINT"
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
-- Can only add exclusion constraint on distribution column (or group of columns
-- including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products ADD CONSTRAINT exc_name EXCLUDE USING btree (name with =);
-- We can add composite exclusion
ALTER TABLE products ADD CONSTRAINT exc_pno_name EXCLUDE USING btree (product_no with =, name with =);
-- 4th command will error out since it conflicts with exc_pno_name constraint
INSERT INTO products VALUES(1,'product_1', 5);
INSERT INTO products VALUES(1,'product_2', 10);
INSERT INTO products VALUES(2,'product_2', 5);
INSERT INTO products VALUES(2,'product_2', 5);
DROP TABLE products;
-- Check "EXCLUSION CONSTRAINT" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric
);
SELECT create_reference_table('products_ref');
-- We can add exclusion constraint on any column
ALTER TABLE products_ref ADD CONSTRAINT exc_name EXCLUDE USING btree (name with =);
-- We can add composite exclusion because none of pair of rows are conflicting
ALTER TABLE products_ref ADD CONSTRAINT exc_pno_name EXCLUDE USING btree (product_no with =, name with =);
-- Third insertion will error out, since it has the same name with second insertion
INSERT INTO products_ref VALUES(1,'product_1', 5);
INSERT INTO products_ref VALUES(1,'product_2', 10);
INSERT INTO products_ref VALUES(2,'product_2', 5);
DROP TABLE products_ref;
-- Check "EXCLUSION CONSTRAINT" with append table
CREATE TABLE products_append (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products_append', 'product_no','append');
-- Can only add exclusion constraint on distribution column (or group of column
-- including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products_append ADD CONSTRAINT exc_name EXCLUDE USING btree (name with =);
ALTER TABLE products_append ADD CONSTRAINT exc_pno_name EXCLUDE USING btree (product_no with =, name with =);
-- Error out since first and third can not pass the exclusion check.
\COPY products_append FROM STDIN DELIMITER AS ',';
1, Product_1, 10
1, Product_2, 15
1, Product_1, 8
\.
DROP TABLE products_append;
-- Check "NOT NULL"
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
ALTER TABLE products ALTER COLUMN name SET NOT NULL;
-- Insertions will error out since both product_no and name can not have NULL value
INSERT INTO products VALUES(1,NULL,5);
INSERT INTO products VALUES(NULL,'product_1', 5);
DROP TABLE products;
-- Check "NOT NULL" with reference table
CREATE TABLE products_ref (
product_no integer,
name text,
price numeric
);
SELECT create_reference_table('products_ref');
ALTER TABLE products_ref ALTER COLUMN name SET NOT NULL;
-- Insertions will error out since both product_no and name can not have NULL value
INSERT INTO products_ref VALUES(1,NULL,5);
INSERT INTO products_ref VALUES(NULL,'product_1', 5);
DROP TABLE products_ref;
-- Check "NOT NULL" with append table
CREATE TABLE products_append (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products_append', 'product_no', 'append');
ALTER TABLE products_append ALTER COLUMN name SET NOT NULL;
-- Error out since name and product_no columns can not handle NULL value.
\COPY products_append FROM STDIN DELIMITER AS ',';
1, \N, 10
\N, Product_2, 15
1, Product_1, 8
\.
DROP TABLE products_append;
-- Tests for ADD CONSTRAINT is not only subcommand
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
-- Should error out since add constraint is not the single subcommand
ALTER TABLE products ADD CONSTRAINT unn_1 UNIQUE(product_no, price), ADD CONSTRAINT unn_2 UNIQUE(product_no, name);
-- Tests for constraints without name
-- Commands below should error out since constraints do not have the name
ALTER TABLE products ADD UNIQUE(product_no);
ALTER TABLE products ADD PRIMARY KEY(product_no);
ALTER TABLE products ADD CHECK(product_no <> 0);
ALTER TABLE products ADD EXCLUDE USING btree (product_no with =);
DROP TABLE products;
-- Tests with transactions
CREATE TABLE products (
product_no integer,
name text,
price numeric,
discounted_price numeric
);
SELECT create_distributed_table('products', 'product_no');
BEGIN;
INSERT INTO products VALUES(1,'product_1', 5);
-- Should error out since conflicts with the above single-shard data modification command.
ALTER TABLE products ADD CONSTRAINT unn_pno UNIQUE(product_no);
ROLLBACK;
BEGIN;
-- Add constraints
ALTER TABLE products ADD CONSTRAINT unn_pno UNIQUE(product_no);
ALTER TABLE products ADD CONSTRAINT check_price CHECK(price > discounted_price);
ALTER TABLE products ALTER COLUMN product_no SET NOT NULL;
ALTER TABLE products ADD CONSTRAINT p_key_product PRIMARY KEY(product_no);
-- Single shard DML command can't be located in the same transaction with above commands.
INSERT INTO products VALUES(1,'product_1', 10, 8);
ROLLBACK;
-- There should be no constraint on master and worker(s)
\d products
\c - - - :worker_1_port
\d products_1450199
\c - - - :master_port
-- Tests to check the effect of rollback
BEGIN;
-- Add constraints (which will be rollbacked)
ALTER TABLE products ADD CONSTRAINT unn_pno UNIQUE(product_no);
ALTER TABLE products ADD CONSTRAINT check_price CHECK(price > discounted_price);
ALTER TABLE products ADD CONSTRAINT p_key_product PRIMARY KEY(product_no);
ROLLBACK;
-- There should be no constraint on master and worker(s)
\d products
\c - - - :worker_1_port
\d products_1450199
\c - - - :master_port
DROP TABLE products;

View File

@ -220,7 +220,7 @@ FROM
WHERE
li.l_quantity > 25
ORDER BY
li.l_quantity
li.l_quantity, li.l_partkey, o.o_custkey
LIMIT 10 OFFSET 20;
RESET client_min_messages;

View File

@ -535,7 +535,9 @@ FROM
NATURAL JOIN pg_dist_shard
NATURAL JOIN pg_dist_shard_placement
WHERE
logicalrelid = 'mx_ref'::regclass;
logicalrelid = 'mx_ref'::regclass
ORDER BY
nodeport;
SELECT shardid AS ref_table_shardid FROM pg_dist_shard WHERE logicalrelid='mx_ref'::regclass \gset