Add Foreign Key Support to ALTER TABLE commands

With this PR, we add foreign key support to ALTER TABLE commands. For now,
we only support foreign constraint creation via ALTER TABLE query, if it
is only subcommand in ALTER TABLE subcommand list.

We also only allow foreign key creation if replication factor is 1.
pull/1938/head
Burak Yucesoy 2016-10-25 16:24:07 +03:00
parent b16540725a
commit cdb725a561
7 changed files with 1063 additions and 71 deletions

View File

@ -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"),

View File

@ -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.

View File

@ -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,

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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;