diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index f9ebf8af1..3247d3c98 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -485,34 +485,48 @@ ErrorIfNotSupportedForeignConstraint(Relation relation, char distributionMethod, " supported in ON UPDATE operation."))); } - /* to enforce foreign constraints, tables must be co-located */ referencedTableId = constraintForm->confrelid; - /* check that the relation is not already distributed */ - if (!IsDistributedTable(referencedTableId)) - { - ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION), - errmsg("cannot create foreign key constraint"), - errdetail("Referenced table must be a distributed table."))); - } - - - 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. + * Some checks are not meaningful if foreign key references the table itself. + * Therefore we will skip those checks. */ - referencedTablePartitionColumn = PartitionKey(referencedTableId); + if (referencedTableId != relation->rd_id) + { + 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 @@ -559,7 +573,8 @@ ErrorIfNotSupportedForeignConstraint(Relation relation, char distributionMethod, * greater than 1. Because in our current design, multiple replicas may cause * locking problems and inconsistent shard contents. */ - if (ShardReplicationFactor > 1 || !SingleReplicatedTable(referencedTableId)) + if (ShardReplicationFactor > 1 || (referencedTableId != relation->rd_id && + !SingleReplicatedTable(referencedTableId))) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("cannot create foreign key constraint"), diff --git a/src/backend/distributed/executor/multi_utility.c b/src/backend/distributed/executor/multi_utility.c index c06088924..4b3183542 100644 --- a/src/backend/distributed/executor/multi_utility.c +++ b/src/backend/distributed/executor/multi_utility.c @@ -31,6 +31,7 @@ #include "commands/tablecmds.h" #include "commands/prepare.h" #include "distributed/citus_ruleutils.h" +#include "distributed/colocation_utils.h" #include "distributed/commit_protocol.h" #include "distributed/connection_cache.h" #include "distributed/master_metadata_utility.h" @@ -124,8 +125,13 @@ static void CreateLocalTable(RangeVar *relation, char *nodeName, int32 nodePort) static bool IsAlterTableRenameStmt(RenameStmt *renameStatement); static void ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString, bool isTopLevel); +static void ExecuteDistributedForeignKeyCommand(Oid leftRelationId, Oid rightRelationId, + const char *ddlCommandString, + bool isTopLevel); static void ShowNoticeIfNotUsing2PC(void); static List * DDLTaskList(Oid relationId, const char *commandString); +static List * ForeignKeyTaskList(Oid leftRelationId, Oid rightRelationId, + const char *commandString); static void RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid, void *arg); static void CheckCopyPermissions(CopyStmt *copyStatement); @@ -755,24 +761,84 @@ static Node * ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCommand, bool isTopLevel) { - /* first check whether a distributed relation is affected */ - if (alterTableStatement->relation != NULL) - { - LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); - Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode); - if (OidIsValid(relationId)) - { - bool isDistributedRelation = IsDistributedTable(relationId); - if (isDistributedRelation) - { - ErrorIfUnsupportedAlterTableStmt(alterTableStatement); + LOCKMODE lockmode = 0; + Oid leftRelationId = InvalidOid; + Oid rightRelationId = InvalidOid; + bool isDistributedRelation = false; + List *commandList = NIL; + ListCell *commandCell = NULL; - /* if it is supported, go ahead and execute the command */ - ExecuteDistributedDDLCommand(relationId, alterTableCommand, isTopLevel); + /* first check whether a distributed relation is affected */ + if (alterTableStatement->relation == NULL) + { + return (Node *) alterTableStatement; + } + + lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); + leftRelationId = AlterTableLookupRelation(alterTableStatement, lockmode); + if (!OidIsValid(leftRelationId)) + { + return (Node *) alterTableStatement; + } + + isDistributedRelation = IsDistributedTable(leftRelationId); + if (!isDistributedRelation) + { + return (Node *) alterTableStatement; + } + + ErrorIfUnsupportedAlterTableStmt(alterTableStatement); + + /* + * We check if there is a ADD FOREIGN CONSTRAINT command in sub commands list. + * If there is we assign referenced releation id to rightRelationId and we also + * set skip_validation to true to prevent PostgreSQL to verify validity of the + * foreign constraint in master. Validity will be checked in workers anyway. + */ + commandList = alterTableStatement->cmds; + + foreach(commandCell, commandList) + { + AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell); + AlterTableType alterTableType = command->subtype; + + if (alterTableType == AT_AddConstraint) + { + Constraint *constraint = (Constraint *) command->def; + if (constraint->contype == CONSTR_FOREIGN) + { + /* + * We only support ALTER TABLE ADD CONSTRAINT ... FOREIGN KEY, if it is + * only subcommand of ALTER TABLE. It was already checked in + * ErrorIfUnsupportedAlterTableStmt. + */ + Assert(commandList->length <= 1); + + rightRelationId = RangeVarGetRelid(constraint->pktable, lockmode, + alterTableStatement->missing_ok); + + /* + * Foreign constraint validations will be done in workers. If we do not + * set this flag, PostgreSQL tries to do additional checking when we drop + * to standard_ProcessUtility. standard_ProcessUtility tries to open new + * connections to workers to verify foreign constraints while original + * transaction is in process, which causes deadlock. + */ + constraint->skip_validation = true; } } } + if (rightRelationId) + { + ExecuteDistributedForeignKeyCommand(leftRelationId, rightRelationId, + alterTableCommand, isTopLevel); + } + else + { + ExecuteDistributedDDLCommand(leftRelationId, alterTableCommand, isTopLevel); + } + return (Node *) alterTableStatement; } @@ -932,6 +998,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 */ static void ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement) @@ -1016,13 +1083,185 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement) break; } + 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 */ + 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."), + errhint("You can issue each subcommand separately"))); + } + + /* + * 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. + */ + 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 */ + referencingTableId = RangeVarGetRelid(alterTableStatement->relation, + lockmode, + alterTableStatement->missing_ok); + referencedTableId = RangeVarGetRelid(constraint->pktable, lockmode, + alterTableStatement->missing_ok); + + 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."))); + } + + /* + * 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."))); + } + + break; + } + + case AT_DropConstraint: + { + /* we will no perform any special check for ALTER TABLE DROP CONSTRAINT */ + break; + } + default: { 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 and TYPE subcommands are" - " supported."))); + " SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN" + " KEY and TYPE subcommands are supported."))); } } } @@ -1356,6 +1595,40 @@ ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString, } +/* + * ExecuteDistributedForeignKeyCommand applies a given foreign key command to the given + * distributed table in a distributed transaction. If the multi shard commit protocol is + * in its default value of '1pc', then a notice message indicating that '2pc' might be + * used for extra safety. In the commit protocol, a BEGIN is sent after connection to + * each shard placement and COMMIT/ROLLBACK is handled by + * CompleteShardPlacementTransactions function. + * + * leftRelationId is the relation id of actual distributed table which given foreign key + * command is applied. rightRelationId is the relation id of distributed table which + * foreign key refers to. + */ +static void +ExecuteDistributedForeignKeyCommand(Oid leftRelationId, Oid rightRelationId, + const char *ddlCommandString, bool isTopLevel) +{ + List *taskList = NIL; + + if (XactModificationLevel == XACT_MODIFICATION_DATA) + { + ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION), + errmsg("distributed DDL commands must not appear within " + "transaction blocks containing single-shard data " + "modifications"))); + } + + ShowNoticeIfNotUsing2PC(); + + taskList = ForeignKeyTaskList(leftRelationId, rightRelationId, ddlCommandString); + + ExecuteModifyTasksWithoutResults(taskList); +} + + /* * ShowNoticeIfNotUsing2PC shows a notice message about using 2PC by setting * citus.multi_shard_commit_protocol to 2PC. The notice message is shown only once in a @@ -1376,7 +1649,7 @@ ShowNoticeIfNotUsing2PC(void) /* - * DDLCommandList builds a list of tasks to execute a DDL command on a + * DDLTaskList builds a list of tasks to execute a DDL command on a * given list of shards. */ static List * @@ -1402,6 +1675,10 @@ DDLTaskList(Oid relationId, const char *commandString) StringInfo applyCommand = makeStringInfo(); Task *task = NULL; + /* + * If rightRelationId is not InvalidOid, instead of worker_apply_shard_ddl_command + * we use worker_apply_inter_shard_ddl_command. + */ appendStringInfo(applyCommand, WORKER_APPLY_SHARD_DDL_COMMAND, shardId, escapedSchemaName, escapedCommandString); @@ -1421,6 +1698,69 @@ DDLTaskList(Oid relationId, const char *commandString) } +/* + * ForeignKeyTaskList builds a list of tasks to execute a foreign key command on a + * shards of given list of distributed table. + * + * leftRelationId is the relation id of actual distributed table which given foreign key + * command is applied. rightRelationId is the relation id of distributed table which + * foreign key refers to. + */ +static List * +ForeignKeyTaskList(Oid leftRelationId, Oid rightRelationId, + const char *commandString) +{ + List *taskList = NIL; + + List *leftShardList = LoadShardIntervalList(leftRelationId); + ListCell *leftShardCell = NULL; + Oid leftSchemaId = get_rel_namespace(leftRelationId); + char *leftSchemaName = get_namespace_name(leftSchemaId); + char *escapedLeftSchemaName = quote_literal_cstr(leftSchemaName); + + List *rightShardList = LoadShardIntervalList(rightRelationId); + ListCell *rightShardCell = NULL; + Oid rightSchemaId = get_rel_namespace(rightRelationId); + char *rightSchemaName = get_namespace_name(rightSchemaId); + char *escapedRightSchemaName = quote_literal_cstr(rightSchemaName); + + char *escapedCommandString = quote_literal_cstr(commandString); + uint64 jobId = INVALID_JOB_ID; + int taskId = 1; + + /* lock metadata before getting placement lists */ + LockShardListMetadata(leftShardList, ShareLock); + + forboth(leftShardCell, leftShardList, rightShardCell, rightShardList) + { + ShardInterval *leftShardInterval = (ShardInterval *) lfirst(leftShardCell); + uint64 leftShardId = leftShardInterval->shardId; + StringInfo applyCommand = makeStringInfo(); + Task *task = NULL; + + ShardInterval *rightShardInterval = (ShardInterval *) lfirst(rightShardCell); + uint64 rightShardId = rightShardInterval->shardId; + + appendStringInfo(applyCommand, WORKER_APPLY_INTER_SHARD_DDL_COMMAND, + leftShardId, escapedLeftSchemaName, rightShardId, + escapedRightSchemaName, escapedCommandString); + + task = CitusMakeNode(Task); + task->jobId = jobId; + task->taskId = taskId++; + task->taskType = SQL_TASK; + task->queryString = applyCommand->data; + task->dependedTaskList = NULL; + task->anchorShardId = leftShardId; + task->taskPlacementList = FinalizedShardPlacementList(leftShardId); + + taskList = lappend(taskList, task); + } + + return taskList; +} + + /* * Before acquiring a table lock, check whether we have sufficient rights. * In the case of DROP INDEX, also try to lock the table before the index. diff --git a/src/backend/distributed/master/master_stage_protocol.c b/src/backend/distributed/master/master_stage_protocol.c index 97b65b485..1c821da2b 100644 --- a/src/backend/distributed/master/master_stage_protocol.c +++ b/src/backend/distributed/master/master_stage_protocol.c @@ -391,18 +391,6 @@ CreateShardPlacements(Oid relationId, int64 shardId, List *ddlEventList, int shardIndex = -1; /* not used in this code path */ bool created = false; - /* - * In this code path, we create not co-located tables. Therefore we should error - * out if there is a foreign key constraint specified. - */ - if (foreignConstraintCommandList != NIL) - { - ereport(ERROR, (errmsg("could only create distributed table"), - errdetail("Table contains foreign key constraints. Foreign " - "key constraints only supported with co-located " - "tables"))); - } - created = WorkerCreateShard(relationId, nodeName, nodePort, shardIndex, shardId, newPlacementOwner, ddlEventList, foreignConstraintCommandList); @@ -507,7 +495,22 @@ WorkerCreateShard(Oid relationId, char *nodeName, uint32 nodePort, referencedSchemaId = get_rel_namespace(referencedRelationId); referencedSchemaName = get_namespace_name(referencedSchemaId); escapedReferencedSchemaName = quote_literal_cstr(referencedSchemaName); - referencedShardId = ColocatedShardIdInRelation(referencedRelationId, shardIndex); + + /* + * In case of self referencing shards, relation itself might not be distributed + * already. Therefore we cannot use ColocatedShardIdInRelation which assumes + * given relation is distributed. Besides, since we know foreign key references + * itself, referencedShardId is actual shardId anyway. + */ + if (relationId == referencedRelationId) + { + referencedShardId = shardId; + } + else + { + referencedShardId = ColocatedShardIdInRelation(referencedRelationId, + shardIndex); + } appendStringInfo(applyForeignConstraintCommand, WORKER_APPLY_INTER_SHARD_DDL_COMMAND, shardId, escapedSchemaName, diff --git a/src/test/regress/expected/multi_foreign_key.out b/src/test/regress/expected/multi_foreign_key.out index a78758ba3..970d47c37 100644 --- a/src/test/regress/expected/multi_foreign_key.out +++ b/src/test/regress/expected/multi_foreign_key.out @@ -379,3 +379,397 @@ SELECT * FROM referencing_table; DROP TABLE referencing_table; DROP TABLE referenced_table; +-- Similar tests, but this time we push foreign key constraints created by ALTER TABLE queries +-- create tables +CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column)); +SELECT master_create_distributed_table('referenced_table', 'id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('referenced_table', 4, 1); + master_create_worker_shards +----------------------------- + +(1 row) + +CREATE TABLE referencing_table(id int, ref_id int); +SELECT master_create_distributed_table('referencing_table', 'ref_id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('referencing_table', 4, 1); + master_create_worker_shards +----------------------------- + +(1 row) + +-- test foreign constraint creation +-- test foreign constraint creation with not supported parameters +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET NULL; +ERROR: cannot create foreign key constraint +DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation. +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET DEFAULT; +ERROR: cannot create foreign key constraint +DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation. +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET NULL; +ERROR: cannot create foreign key constraint +DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation. +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET DEFAULT; +ERROR: cannot create foreign key constraint +DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation. +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE CASCADE; +ERROR: cannot create foreign key constraint +DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation. +-- 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. +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. +-- 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 +DETAIL: Foreign key constraint can only be created on co-located tables. +-- create co-located tables +DROP TABLE referencing_table; +DROP TABLE referenced_table; +CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column)); +CREATE TABLE referencing_table(id int, ref_id int); +SELECT create_distributed_table('referenced_table', 'id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('referencing_table', 'ref_id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +-- 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 +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 in incorrect order +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(id, ref_id) REFERENCES referenced_table(id, test_column); +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. +-- 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); +ERROR: insert or update on table "referencing_table_1350080" violates foreign key constraint "test_constraint_1350080" +DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350076". +CONTEXT: while executing command on localhost:57637 +-- test foreign constraint with correct conditions +DELETE FROM referencing_table WHERE ref_id = 1; +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id); +-- test inserts +-- test insert to referencing table while there is NO corresponding value in referenced table +INSERT INTO referencing_table VALUES(1, 1); +ERROR: insert or update on table "referencing_table_1350080" violates foreign key constraint "test_constraint_1350080" +DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350076". +CONTEXT: while executing command on localhost:57637 +-- test insert to referencing while there is corresponding value in referenced table +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); +-- test deletes +-- test delete from referenced table while there is corresponding value in referencing table +DELETE FROM referenced_table WHERE id = 1; +ERROR: update or delete on table "referenced_table_1350076" violates foreign key constraint "test_constraint_1350080" on table "referencing_table_1350080" +DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350080". +CONTEXT: while executing command on localhost:57637 +-- test delete from referenced table while there is NO corresponding value in referencing table +DELETE FROM referencing_table WHERE ref_id = 1; +DELETE FROM referenced_table WHERE id = 1; +-- test DROP CONSTRAINT +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- test foreign constraint options +-- test ON DELETE CASCADE +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE CASCADE; +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); +DELETE FROM referenced_table WHERE id = 1; +SELECT * FROM referencing_table; + id | ref_id +----+-------- +(0 rows) + +SELECT * FROM referenced_table; + id | test_column +----+------------- +(0 rows) + +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- test ON DELETE NO ACTION + DEFERABLE + INITIALLY DEFERRED +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE NO ACTION DEFERRABLE INITIALLY DEFERRED; +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); +DELETE FROM referenced_table WHERE id = 1; +ERROR: update or delete on table "referenced_table_1350076" violates foreign key constraint "test_constraint_1350080" on table "referencing_table_1350080" +DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350080". +CONTEXT: while executing command on localhost:57637 +BEGIN; +DELETE FROM referenced_table WHERE id = 1; +DELETE FROM referencing_table WHERE ref_id = 1; +COMMIT; +SELECT * FROM referencing_table; + id | ref_id +----+-------- +(0 rows) + +SELECT * FROM referenced_table; + id | test_column +----+------------- +(0 rows) + +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- test ON DELETE RESTRICT +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE RESTRICT; +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); +BEGIN; +DELETE FROM referenced_table WHERE id = 1; +ERROR: update or delete on table "referenced_table_1350076" violates foreign key constraint "test_constraint_1350080" on table "referencing_table_1350080" +DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350080". +CONTEXT: while executing command on localhost:57637 +DELETE FROM referencing_table WHERE ref_id = 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +SELECT * FROM referencing_table; + id | ref_id +----+-------- + 1 | 1 +(1 row) + +SELECT * FROM referenced_table; + id | test_column +----+------------- + 1 | 1 +(1 row) + +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- test ON UPDATE NO ACTION + DEFERABLE + INITIALLY DEFERRED +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) ON UPDATE NO ACTION DEFERRABLE INITIALLY DEFERRED; +UPDATE referenced_table SET test_column = 10 WHERE id = 1; +ERROR: update or delete on table "referenced_table_1350076" violates foreign key constraint "test_constraint_1350080" on table "referencing_table_1350080" +DETAIL: Key (id, test_column)=(1, 1) is still referenced from table "referencing_table_1350080". +CONTEXT: while executing command on localhost:57637 +BEGIN; +UPDATE referenced_table SET test_column = 10 WHERE id = 1; +UPDATE referencing_table SET id = 10 WHERE ref_id = 1; +COMMIT; +SELECT * FROM referencing_table; + id | ref_id +----+-------- + 10 | 1 +(1 row) + +SELECT * FROM referenced_table; + id | test_column +----+------------- + 1 | 10 +(1 row) + +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- test ON UPDATE RESTRICT +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) ON UPDATE RESTRICT; +BEGIN; +UPDATE referenced_table SET test_column = 20 WHERE id = 1; +ERROR: update or delete on table "referenced_table_1350076" violates foreign key constraint "test_constraint_1350080" on table "referencing_table_1350080" +DETAIL: Key (id, test_column)=(1, 10) is still referenced from table "referencing_table_1350080". +CONTEXT: while executing command on localhost:57637 +UPDATE referencing_table SET id = 20 WHERE ref_id = 1; +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +SELECT * FROM referencing_table; + id | ref_id +----+-------- + 10 | 1 +(1 row) + +SELECT * FROM referenced_table; + id | test_column +----+------------- + 1 | 10 +(1 row) + +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- test MATCH SIMPLE +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) MATCH SIMPLE; +INSERT INTO referencing_table VALUES(null, 2); +SELECT * FROM referencing_table; + id | ref_id +----+-------- + 10 | 1 + | 2 +(2 rows) + +DELETE FROM referencing_table WHERE ref_id = 2; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- test MATCH FULL +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) MATCH FULL; +INSERT INTO referencing_table VALUES(null, 2); +ERROR: insert or update on table "referencing_table_1350083" violates foreign key constraint "test_constraint_1350083" +DETAIL: MATCH FULL does not allow mixing of null and nonnull key values. +CONTEXT: while executing command on localhost:57638 +SELECT * FROM referencing_table; + id | ref_id +----+-------- + 10 | 1 +(1 row) + +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; +-- we no longer need those tables +DROP TABLE referencing_table; +DROP TABLE referenced_table; +-- test cyclical foreign keys +CREATE TABLE cyclic_reference_table1(id int, table2_id int, PRIMARY KEY(id, table2_id)); +CREATE TABLE cyclic_reference_table2(id int, table1_id int, PRIMARY KEY(id, table1_id)); +SELECT create_distributed_table('cyclic_reference_table1', 'id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('cyclic_reference_table2', 'table1_id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +ALTER TABLE cyclic_reference_table1 ADD CONSTRAINT cyclic_constraint1 FOREIGN KEY(id, table2_id) REFERENCES cyclic_reference_table2(table1_id, id) DEFERRABLE INITIALLY DEFERRED; +ALTER TABLE cyclic_reference_table2 ADD CONSTRAINT cyclic_constraint2 FOREIGN KEY(id, table1_id) REFERENCES cyclic_reference_table1(table2_id, id) DEFERRABLE INITIALLY DEFERRED; +-- test insertion to a table which has cyclic foreign constraints, we expect that to fail +INSERT INTO cyclic_reference_table1 VALUES(1, 1); +ERROR: insert or update on table "cyclic_reference_table1_1350084" violates foreign key constraint "cyclic_constraint1_1350084" +DETAIL: Key (id, table2_id)=(1, 1) is not present in table "cyclic_reference_table2_1350088". +CONTEXT: while executing command on localhost:57637 +-- proper insertion to table with cyclic dependency +BEGIN; +INSERT INTO cyclic_reference_table1 VALUES(1, 1); +INSERT INTO cyclic_reference_table2 VALUES(1, 1); +COMMIT; +-- verify that rows are actually inserted +SELECT * FROM cyclic_reference_table1; + id | table2_id +----+----------- + 1 | 1 +(1 row) + +SELECT * FROM cyclic_reference_table2; + id | table1_id +----+----------- + 1 | 1 +(1 row) + +-- test dropping cyclic referenced tables +-- we expect those two queries to fail +DROP TABLE cyclic_reference_table1; +ERROR: cannot drop table cyclic_reference_table1 because other objects depend on it +DETAIL: constraint cyclic_constraint2 on table cyclic_reference_table2 depends on table cyclic_reference_table1 +HINT: Use DROP ... CASCADE to drop the dependent objects too. +DROP TABLE cyclic_reference_table2; +ERROR: cannot drop table cyclic_reference_table2 because other objects depend on it +DETAIL: constraint cyclic_constraint1 on table cyclic_reference_table1 depends on table cyclic_reference_table2 +HINT: Use DROP ... CASCADE to drop the dependent objects too. +-- proper way of DROP with CASCADE option +DROP TABLE cyclic_reference_table1 CASCADE; +NOTICE: drop cascades to constraint cyclic_constraint2 on table cyclic_reference_table2 +DROP TABLE cyclic_reference_table2 CASCADE; +-- test creation of foreign keys in a transaction +CREATE TABLE transaction_referenced_table(id int PRIMARY KEY); +CREATE TABLE transaction_referencing_table(id int, ref_id int); +BEGIN; +ALTER TABLE transaction_referencing_table ADD CONSTRAINT transaction_fk_constraint FOREIGN KEY(ref_id) REFERENCES transaction_referenced_table(id); +COMMIT; +-- test insertion to referencing table, we expect that to fail +INSERT INTO transaction_referencing_table VALUES(1, 1); +ERROR: insert or update on table "transaction_referencing_table" violates foreign key constraint "transaction_fk_constraint" +DETAIL: Key (ref_id)=(1) is not present in table "transaction_referenced_table". +-- proper insertion to both referenced and referencing tables +INSERT INTO transaction_referenced_table VALUES(1); +INSERT INTO transaction_referencing_table VALUES(1, 1); +-- verify that rows are actually inserted +SELECT * FROM transaction_referenced_table; + id +---- + 1 +(1 row) + +SELECT * FROM transaction_referencing_table; + id | ref_id +----+-------- + 1 | 1 +(1 row) + +-- we no longer need those tables +DROP TABLE transaction_referencing_table; +DROP TABLE transaction_referenced_table; +-- test self referencing foreign key +CREATE TABLE self_referencing_table1( + id int, + other_column int, + other_column_ref int, + PRIMARY KEY(id, other_column), + FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_table1(id, other_column) +); +SELECT create_distributed_table('self_referencing_table1', 'id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +-- test insertion to self referencing table +INSERT INTO self_referencing_table1 VALUES(1, 1, 1); +-- we expect this query to fail +INSERT INTO self_referencing_table1 VALUES(1, 2, 3); +ERROR: insert or update on table "self_referencing_table1_1350092" violates foreign key constraint "self_referencing_table1_id_fkey_1350092" +DETAIL: Key (id, other_column_ref)=(1, 3) is not present in table "self_referencing_table1_1350092". +CONTEXT: while executing command on localhost:57637 +-- verify that rows are actually inserted +SELECT * FROM self_referencing_table1; + id | other_column | other_column_ref +----+--------------+------------------ + 1 | 1 | 1 +(1 row) + +-- we no longer need those tables +DROP TABLE self_referencing_table1; +-- test self referencing foreign key with ALTER TABLE +CREATE TABLE self_referencing_table2(id int, other_column int, other_column_ref int, PRIMARY KEY(id, other_column)); +SELECT create_distributed_table('self_referencing_table2', 'id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +ALTER TABLE self_referencing_table2 ADD CONSTRAINT self_referencing_fk_constraint FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_table2(id, other_column); +-- test insertion to self referencing table +INSERT INTO self_referencing_table2 VALUES(1, 1, 1); +-- we expect this query to fail +INSERT INTO self_referencing_table2 VALUES(1, 2, 3); +ERROR: insert or update on table "self_referencing_table2_1350096" violates foreign key constraint "self_referencing_fk_constraint_1350096" +DETAIL: Key (id, other_column_ref)=(1, 3) is not present in table "self_referencing_table2_1350096". +CONTEXT: while executing command on localhost:57637 +-- verify that rows are actually inserted +SELECT * FROM self_referencing_table2; + id | other_column | other_column_ref +----+--------------+------------------ + 1 | 1 | 1 +(1 row) + +-- we no longer need those tables +DROP TABLE self_referencing_table2; diff --git a/src/test/regress/expected/multi_name_lengths.out b/src/test/regress/expected/multi_name_lengths.out index 3990408fa..b4b9fec38 100644 --- a/src/test/regress/expected/multi_name_lengths.out +++ b/src/test/regress/expected/multi_name_lengths.out @@ -78,14 +78,14 @@ 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: alter table command is currently unsupported -DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. +ERROR: cannot create constraint +DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY. ALTER TABLE name_lengths ADD EXCLUDE (int_col_12345678901234567890123456789012345678901234567890 WITH =); -ERROR: alter table command is currently unsupported -DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. +ERROR: cannot create constraint +DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY. ALTER TABLE name_lengths ADD CHECK (date_col_12345678901234567890123456789012345678901234567890 > '2014-01-01'::date); -ERROR: alter table command is currently unsupported -DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. +ERROR: cannot create constraint +DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY. \c - - - :worker_1_port \d name_lengths_* Table "public.name_lengths_225002" @@ -113,14 +113,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: alter table command is currently unsupported -DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. +ERROR: cannot create constraint +DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY. ALTER TABLE name_lengths ADD CONSTRAINT nl_exclude_12345678901234567890123456789012345678901234567890 EXCLUDE (int_col_12345678901234567890123456789012345678901234567890 WITH =); -ERROR: alter table command is currently unsupported -DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. +ERROR: cannot create constraint +DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY. ALTER TABLE name_lengths ADD CONSTRAINT nl_checky_12345678901234567890123456789012345678901234567890 CHECK (date_col_12345678901234567890123456789012345678901234567890 >= '2014-01-01'::date); -ERROR: alter table command is currently unsupported -DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. +ERROR: cannot create constraint +DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY. \c - - - :worker_1_port \d nl_* \c - - - :master_port diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index 3db9dbc77..93a826c4a 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -318,7 +318,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 and TYPE subcommands are supported. +DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN KEY and TYPE subcommands are supported. ALTER TABLE lineitem_alter DROP COLUMN int_column1, DROP COLUMN int_column2; \d lineitem_alter Table "public.lineitem_alter" @@ -350,13 +350,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 and TYPE subcommands are supported. +DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN KEY and TYPE subcommands are supported. ALTER TABLE lineitem_alter DROP CONSTRAINT IF EXISTS non_existent_contraint; -ERROR: alter table command is currently unsupported -DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. +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 and TYPE subcommands are supported. +DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN KEY 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_foreign_key.sql b/src/test/regress/sql/multi_foreign_key.sql index 1627157e3..9690b0e26 100644 --- a/src/test/regress/sql/multi_foreign_key.sql +++ b/src/test/regress/sql/multi_foreign_key.sql @@ -208,3 +208,244 @@ INSERT INTO referencing_table VALUES(null, 2); SELECT * FROM referencing_table; DROP TABLE referencing_table; DROP TABLE referenced_table; + + +-- Similar tests, but this time we push foreign key constraints created by ALTER TABLE queries +-- create tables +CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column)); +SELECT master_create_distributed_table('referenced_table', 'id', 'hash'); +SELECT master_create_worker_shards('referenced_table', 4, 1); + +CREATE TABLE referencing_table(id int, ref_id int); +SELECT master_create_distributed_table('referencing_table', 'ref_id', 'hash'); +SELECT master_create_worker_shards('referencing_table', 4, 1); + + +-- test foreign constraint creation +-- test foreign constraint creation with not supported parameters +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET NULL; +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET DEFAULT; +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET NULL; +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET DEFAULT; +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE CASCADE; + +-- 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); + +-- test foreign constraint creation without giving explicit name +ALTER TABLE referencing_table ADD FOREIGN KEY(ref_id) REFERENCES referenced_table(id); + +-- 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); + +-- create co-located tables +DROP TABLE referencing_table; +DROP TABLE referenced_table; +CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column)); +CREATE TABLE referencing_table(id int, ref_id int); +SELECT create_distributed_table('referenced_table', 'id', 'hash'); +SELECT create_distributed_table('referencing_table', 'ref_id', 'hash'); + +-- test foreign constraint creation on non-partition columns +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(id) REFERENCES referenced_table(id); + +-- test foreign constraint creation while column list are in incorrect order +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(id, ref_id) REFERENCES referenced_table(id, test_column); + +-- 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); + +-- 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); + +-- test foreign constraint with correct conditions +DELETE FROM referencing_table WHERE ref_id = 1; +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id); + + +-- test inserts +-- test insert to referencing table while there is NO corresponding value in referenced table +INSERT INTO referencing_table VALUES(1, 1); + +-- test insert to referencing while there is corresponding value in referenced table +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); + + +-- test deletes +-- test delete from referenced table while there is corresponding value in referencing table +DELETE FROM referenced_table WHERE id = 1; + +-- test delete from referenced table while there is NO corresponding value in referencing table +DELETE FROM referencing_table WHERE ref_id = 1; +DELETE FROM referenced_table WHERE id = 1; + + +-- test DROP CONSTRAINT +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + + +-- test foreign constraint options +-- test ON DELETE CASCADE +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE CASCADE; +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); +DELETE FROM referenced_table WHERE id = 1; +SELECT * FROM referencing_table; +SELECT * FROM referenced_table; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + +-- test ON DELETE NO ACTION + DEFERABLE + INITIALLY DEFERRED +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE NO ACTION DEFERRABLE INITIALLY DEFERRED; +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); +DELETE FROM referenced_table WHERE id = 1; +BEGIN; +DELETE FROM referenced_table WHERE id = 1; +DELETE FROM referencing_table WHERE ref_id = 1; +COMMIT; +SELECT * FROM referencing_table; +SELECT * FROM referenced_table; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + +-- test ON DELETE RESTRICT +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE RESTRICT; +INSERT INTO referenced_table VALUES(1, 1); +INSERT INTO referencing_table VALUES(1, 1); +BEGIN; +DELETE FROM referenced_table WHERE id = 1; +DELETE FROM referencing_table WHERE ref_id = 1; +COMMIT; +SELECT * FROM referencing_table; +SELECT * FROM referenced_table; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + +-- test ON UPDATE NO ACTION + DEFERABLE + INITIALLY DEFERRED +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) ON UPDATE NO ACTION DEFERRABLE INITIALLY DEFERRED; +UPDATE referenced_table SET test_column = 10 WHERE id = 1; +BEGIN; +UPDATE referenced_table SET test_column = 10 WHERE id = 1; +UPDATE referencing_table SET id = 10 WHERE ref_id = 1; +COMMIT; +SELECT * FROM referencing_table; +SELECT * FROM referenced_table; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + +-- test ON UPDATE RESTRICT +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) ON UPDATE RESTRICT; +BEGIN; +UPDATE referenced_table SET test_column = 20 WHERE id = 1; +UPDATE referencing_table SET id = 20 WHERE ref_id = 1; +COMMIT; +SELECT * FROM referencing_table; +SELECT * FROM referenced_table; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + +-- test MATCH SIMPLE +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) MATCH SIMPLE; +INSERT INTO referencing_table VALUES(null, 2); +SELECT * FROM referencing_table; +DELETE FROM referencing_table WHERE ref_id = 2; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + +-- test MATCH FULL +ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id, id) REFERENCES referenced_table(id, test_column) MATCH FULL; +INSERT INTO referencing_table VALUES(null, 2); +SELECT * FROM referencing_table; +ALTER TABLE referencing_table DROP CONSTRAINT test_constraint; + +-- we no longer need those tables +DROP TABLE referencing_table; +DROP TABLE referenced_table; + +-- test cyclical foreign keys +CREATE TABLE cyclic_reference_table1(id int, table2_id int, PRIMARY KEY(id, table2_id)); +CREATE TABLE cyclic_reference_table2(id int, table1_id int, PRIMARY KEY(id, table1_id)); +SELECT create_distributed_table('cyclic_reference_table1', 'id', 'hash'); +SELECT create_distributed_table('cyclic_reference_table2', 'table1_id', 'hash'); +ALTER TABLE cyclic_reference_table1 ADD CONSTRAINT cyclic_constraint1 FOREIGN KEY(id, table2_id) REFERENCES cyclic_reference_table2(table1_id, id) DEFERRABLE INITIALLY DEFERRED; +ALTER TABLE cyclic_reference_table2 ADD CONSTRAINT cyclic_constraint2 FOREIGN KEY(id, table1_id) REFERENCES cyclic_reference_table1(table2_id, id) DEFERRABLE INITIALLY DEFERRED; + +-- test insertion to a table which has cyclic foreign constraints, we expect that to fail +INSERT INTO cyclic_reference_table1 VALUES(1, 1); + +-- proper insertion to table with cyclic dependency +BEGIN; +INSERT INTO cyclic_reference_table1 VALUES(1, 1); +INSERT INTO cyclic_reference_table2 VALUES(1, 1); +COMMIT; + +-- verify that rows are actually inserted +SELECT * FROM cyclic_reference_table1; +SELECT * FROM cyclic_reference_table2; + +-- test dropping cyclic referenced tables +-- we expect those two queries to fail +DROP TABLE cyclic_reference_table1; +DROP TABLE cyclic_reference_table2; + +-- proper way of DROP with CASCADE option +DROP TABLE cyclic_reference_table1 CASCADE; +DROP TABLE cyclic_reference_table2 CASCADE; + +-- test creation of foreign keys in a transaction +CREATE TABLE transaction_referenced_table(id int PRIMARY KEY); +CREATE TABLE transaction_referencing_table(id int, ref_id int); + +BEGIN; +ALTER TABLE transaction_referencing_table ADD CONSTRAINT transaction_fk_constraint FOREIGN KEY(ref_id) REFERENCES transaction_referenced_table(id); +COMMIT; + +-- test insertion to referencing table, we expect that to fail +INSERT INTO transaction_referencing_table VALUES(1, 1); + +-- proper insertion to both referenced and referencing tables +INSERT INTO transaction_referenced_table VALUES(1); +INSERT INTO transaction_referencing_table VALUES(1, 1); + +-- verify that rows are actually inserted +SELECT * FROM transaction_referenced_table; +SELECT * FROM transaction_referencing_table; + +-- we no longer need those tables +DROP TABLE transaction_referencing_table; +DROP TABLE transaction_referenced_table; + +-- test self referencing foreign key +CREATE TABLE self_referencing_table1( + id int, + other_column int, + other_column_ref int, + PRIMARY KEY(id, other_column), + FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_table1(id, other_column) +); +SELECT create_distributed_table('self_referencing_table1', 'id', 'hash'); + +-- test insertion to self referencing table +INSERT INTO self_referencing_table1 VALUES(1, 1, 1); +-- we expect this query to fail +INSERT INTO self_referencing_table1 VALUES(1, 2, 3); + +-- verify that rows are actually inserted +SELECT * FROM self_referencing_table1; + +-- we no longer need those tables +DROP TABLE self_referencing_table1; + +-- test self referencing foreign key with ALTER TABLE +CREATE TABLE self_referencing_table2(id int, other_column int, other_column_ref int, PRIMARY KEY(id, other_column)); +SELECT create_distributed_table('self_referencing_table2', 'id', 'hash'); +ALTER TABLE self_referencing_table2 ADD CONSTRAINT self_referencing_fk_constraint FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_table2(id, other_column); + +-- test insertion to self referencing table +INSERT INTO self_referencing_table2 VALUES(1, 1, 1); +-- we expect this query to fail +INSERT INTO self_referencing_table2 VALUES(1, 2, 3); + +-- verify that rows are actually inserted +SELECT * FROM self_referencing_table2; + +-- we no longer need those tables +DROP TABLE self_referencing_table2;