diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 1cced3549..8b107f177 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -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. diff --git a/src/backend/distributed/executor/multi_utility.c b/src/backend/distributed/executor/multi_utility.c index c45860de1..c81d983f5 100644 --- a/src/backend/distributed/executor/multi_utility.c +++ b/src/backend/distributed/executor/multi_utility.c @@ -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. diff --git a/src/include/distributed/multi_utility.h b/src/include/distributed/multi_utility.h index 84ed37443..7875962e0 100644 --- a/src/include/distributed/multi_utility.h +++ b/src/include/distributed/multi_utility.h @@ -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 */ diff --git a/src/test/regress/expected/multi_alter_table_add_constraints.out b/src/test/regress/expected/multi_alter_table_add_constraints.out new file mode 100644 index 000000000..9b56819cc --- /dev/null +++ b/src/test/regress/expected/multi_alter_table_add_constraints.out @@ -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; diff --git a/src/test/regress/expected/multi_complex_expressions.out b/src/test/regress/expected/multi_complex_expressions.out index 4dfb46a94..3723cff0a 100644 --- a/src/test/regress/expected/multi_complex_expressions.out +++ b/src/test/regress/expected/multi_complex_expressions.out @@ -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; diff --git a/src/test/regress/expected/multi_create_table_constraints.out b/src/test/regress/expected/multi_create_table_constraints.out index b3b5cfbd3..b273e3529 100644 --- a/src/test/regress/expected/multi_create_table_constraints.out +++ b/src/test/regress/expected/multi_create_table_constraints.out @@ -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 ( diff --git a/src/test/regress/expected/multi_foreign_key.out b/src/test/regress/expected/multi_foreign_key.out index c05ce9f3d..a05c9519c 100644 --- a/src/test/regress/expected/multi_foreign_key.out +++ b/src/test/regress/expected/multi_foreign_key.out @@ -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; diff --git a/src/test/regress/expected/multi_metadata_sync.out b/src/test/regress/expected/multi_metadata_sync.out index 72e463ccf..8348344b7 100644 --- a/src/test/regress/expected/multi_metadata_sync.out +++ b/src/test/regress/expected/multi_metadata_sync.out @@ -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) diff --git a/src/test/regress/expected/multi_name_lengths.out b/src/test/regress/expected/multi_name_lengths.out index 0e2745466..4dc78c236 100644 --- a/src/test/regress/expected/multi_name_lengths.out +++ b/src/test/regress/expected/multi_name_lengths.out @@ -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 diff --git a/src/test/regress/expected/multi_reference_table.out b/src/test/regress/expected/multi_reference_table.out index 5d58f1251..59700a0cd 100644 --- a/src/test/regress/expected/multi_reference_table.out +++ b/src/test/regress/expected/multi_reference_table.out @@ -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'); diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index ac9c206d8..d54b4078a 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -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 diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index 1feb2755e..bde6e1170 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -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; diff --git a/src/test/regress/sql/multi_alter_table_add_constraints.sql b/src/test/regress/sql/multi_alter_table_add_constraints.sql new file mode 100644 index 000000000..1daa4e9e8 --- /dev/null +++ b/src/test/regress/sql/multi_alter_table_add_constraints.sql @@ -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; diff --git a/src/test/regress/sql/multi_complex_expressions.sql b/src/test/regress/sql/multi_complex_expressions.sql index 5e60b0edf..236e8f4c6 100644 --- a/src/test/regress/sql/multi_complex_expressions.sql +++ b/src/test/regress/sql/multi_complex_expressions.sql @@ -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; diff --git a/src/test/regress/sql/multi_metadata_sync.sql b/src/test/regress/sql/multi_metadata_sync.sql index e19ce3cc4..e7510d3ff 100644 --- a/src/test/regress/sql/multi_metadata_sync.sql +++ b/src/test/regress/sql/multi_metadata_sync.sql @@ -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