Merge pull request #876 from citusdata/foreign_key_push_down_for_alter_table

Add Foreign Key Support to ALTER TABLE commands
pull/1022/head
Burak Yücesoy 2016-12-08 14:11:01 +02:00 committed by GitHub
commit 62b12f413c
7 changed files with 1063 additions and 71 deletions

View File

@ -485,18 +485,23 @@ ErrorIfNotSupportedForeignConstraint(Relation relation, char distributionMethod,
" supported in ON UPDATE operation."))); " supported in ON UPDATE operation.")));
} }
/* to enforce foreign constraints, tables must be co-located */
referencedTableId = constraintForm->confrelid; referencedTableId = constraintForm->confrelid;
/* check that the relation is not already distributed */ /*
* Some checks are not meaningful if foreign key references the table itself.
* Therefore we will skip those checks.
*/
if (referencedTableId != relation->rd_id)
{
if (!IsDistributedTable(referencedTableId)) if (!IsDistributedTable(referencedTableId))
{ {
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION), ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("cannot create foreign key constraint"), errmsg("cannot create foreign key constraint"),
errdetail("Referenced table must be a distributed table."))); errdetail("Referenced table must be a distributed "
"table.")));
} }
/* to enforce foreign constraints, tables must be co-located */
referencedTableColocationId = TableColocationId(referencedTableId); referencedTableColocationId = TableColocationId(referencedTableId);
if (relation->rd_id != referencedTableId && if (relation->rd_id != referencedTableId &&
(colocationId == INVALID_COLOCATION_ID || (colocationId == INVALID_COLOCATION_ID ||
@ -513,6 +518,15 @@ ErrorIfNotSupportedForeignConstraint(Relation relation, char distributionMethod,
* foreign key constraint. They also must be in same ordinal. * foreign key constraint. They also must be in same ordinal.
*/ */
referencedTablePartitionColumn = PartitionKey(referencedTableId); 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 * 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 * greater than 1. Because in our current design, multiple replicas may cause
* locking problems and inconsistent shard contents. * 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), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"), errmsg("cannot create foreign key constraint"),

View File

@ -31,6 +31,7 @@
#include "commands/tablecmds.h" #include "commands/tablecmds.h"
#include "commands/prepare.h" #include "commands/prepare.h"
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/colocation_utils.h"
#include "distributed/commit_protocol.h" #include "distributed/commit_protocol.h"
#include "distributed/connection_cache.h" #include "distributed/connection_cache.h"
#include "distributed/master_metadata_utility.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 bool IsAlterTableRenameStmt(RenameStmt *renameStatement);
static void ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString, static void ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString,
bool isTopLevel); bool isTopLevel);
static void ExecuteDistributedForeignKeyCommand(Oid leftRelationId, Oid rightRelationId,
const char *ddlCommandString,
bool isTopLevel);
static void ShowNoticeIfNotUsing2PC(void); static void ShowNoticeIfNotUsing2PC(void);
static List * DDLTaskList(Oid relationId, const char *commandString); 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, static void RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid,
void *arg); void *arg);
static void CheckCopyPermissions(CopyStmt *copyStatement); static void CheckCopyPermissions(CopyStmt *copyStatement);
@ -755,24 +761,84 @@ static Node *
ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCommand, ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCommand,
bool isTopLevel) bool isTopLevel)
{ {
LOCKMODE lockmode = 0;
Oid leftRelationId = InvalidOid;
Oid rightRelationId = InvalidOid;
bool isDistributedRelation = false;
List *commandList = NIL;
ListCell *commandCell = NULL;
/* first check whether a distributed relation is affected */ /* first check whether a distributed relation is affected */
if (alterTableStatement->relation != NULL) if (alterTableStatement->relation == NULL)
{ {
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); return (Node *) alterTableStatement;
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode); }
if (OidIsValid(relationId))
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
leftRelationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(leftRelationId))
{ {
bool isDistributedRelation = IsDistributedTable(relationId); return (Node *) alterTableStatement;
if (isDistributedRelation) }
isDistributedRelation = IsDistributedTable(leftRelationId);
if (!isDistributedRelation)
{ {
return (Node *) alterTableStatement;
}
ErrorIfUnsupportedAlterTableStmt(alterTableStatement); ErrorIfUnsupportedAlterTableStmt(alterTableStatement);
/* if it is supported, go ahead and execute the command */ /*
ExecuteDistributedDDLCommand(relationId, alterTableCommand, isTopLevel); * 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; return (Node *) alterTableStatement;
} }
@ -932,6 +998,7 @@ ErrorIfUnsupportedDropIndexStmt(DropStmt *dropIndexStatement)
* ALTER TABLE ALTER COLUMN SET DATA TYPE * ALTER TABLE ALTER COLUMN SET DATA TYPE
* ALTER TABLE SET|DROP NOT NULL * ALTER TABLE SET|DROP NOT NULL
* ALTER TABLE SET|DROP DEFAULT * ALTER TABLE SET|DROP DEFAULT
* ALTER TABLE ADD|DROP CONSTRAINT FOREIGN
*/ */
static void static void
ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement) ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
@ -1016,13 +1083,185 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
break; 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: default:
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("alter table command is currently unsupported"), errmsg("alter table command is currently unsupported"),
errdetail("Only ADD|DROP COLUMN, SET|DROP NOT NULL," errdetail("Only ADD|DROP COLUMN, SET|DROP NOT NULL,"
" SET|DROP DEFAULT and TYPE subcommands are" " SET|DROP DEFAULT, ADD|DROP CONSTRAINT FOREIGN"
" supported."))); " 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 * 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 * 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. * given list of shards.
*/ */
static List * static List *
@ -1402,6 +1675,10 @@ DDLTaskList(Oid relationId, const char *commandString)
StringInfo applyCommand = makeStringInfo(); StringInfo applyCommand = makeStringInfo();
Task *task = NULL; 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, appendStringInfo(applyCommand, WORKER_APPLY_SHARD_DDL_COMMAND, shardId,
escapedSchemaName, escapedCommandString); 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. * 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. * 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 */ int shardIndex = -1; /* not used in this code path */
bool created = false; 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, created = WorkerCreateShard(relationId, nodeName, nodePort, shardIndex,
shardId, newPlacementOwner, ddlEventList, shardId, newPlacementOwner, ddlEventList,
foreignConstraintCommandList); foreignConstraintCommandList);
@ -507,7 +495,22 @@ WorkerCreateShard(Oid relationId, char *nodeName, uint32 nodePort,
referencedSchemaId = get_rel_namespace(referencedRelationId); referencedSchemaId = get_rel_namespace(referencedRelationId);
referencedSchemaName = get_namespace_name(referencedSchemaId); referencedSchemaName = get_namespace_name(referencedSchemaId);
escapedReferencedSchemaName = quote_literal_cstr(referencedSchemaName); 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, appendStringInfo(applyForeignConstraintCommand,
WORKER_APPLY_INTER_SHARD_DDL_COMMAND, shardId, escapedSchemaName, WORKER_APPLY_INTER_SHARD_DDL_COMMAND, shardId, escapedSchemaName,

View File

@ -379,3 +379,397 @@ SELECT * FROM referencing_table;
DROP TABLE referencing_table; DROP TABLE referencing_table;
DROP TABLE referenced_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; 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 -- 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); ALTER TABLE name_lengths ADD UNIQUE (float_col_12345678901234567890123456789012345678901234567890);
ERROR: alter table command is currently unsupported ERROR: cannot create constraint
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
ALTER TABLE name_lengths ADD EXCLUDE (int_col_12345678901234567890123456789012345678901234567890 WITH =); ALTER TABLE name_lengths ADD EXCLUDE (int_col_12345678901234567890123456789012345678901234567890 WITH =);
ERROR: alter table command is currently unsupported ERROR: cannot create constraint
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. 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); ALTER TABLE name_lengths ADD CHECK (date_col_12345678901234567890123456789012345678901234567890 > '2014-01-01'::date);
ERROR: alter table command is currently unsupported ERROR: cannot create constraint
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
\c - - - :worker_1_port \c - - - :worker_1_port
\d name_lengths_* \d name_lengths_*
Table "public.name_lengths_225002" Table "public.name_lengths_225002"
@ -113,14 +113,14 @@ Indexes:
\c - - - :master_port \c - - - :master_port
-- Placeholders for unsupported add constraints with EXPLICIT names that are too long -- 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); ALTER TABLE name_lengths ADD CONSTRAINT nl_unique_12345678901234567890123456789012345678901234567890 UNIQUE (float_col_12345678901234567890123456789012345678901234567890);
ERROR: alter table command is currently unsupported ERROR: cannot create constraint
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. 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 =); ALTER TABLE name_lengths ADD CONSTRAINT nl_exclude_12345678901234567890123456789012345678901234567890 EXCLUDE (int_col_12345678901234567890123456789012345678901234567890 WITH =);
ERROR: alter table command is currently unsupported ERROR: cannot create constraint
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. 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); ALTER TABLE name_lengths ADD CONSTRAINT nl_checky_12345678901234567890123456789012345678901234567890 CHECK (date_col_12345678901234567890123456789012345678901234567890 >= '2014-01-01'::date);
ERROR: alter table command is currently unsupported ERROR: cannot create constraint
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported. DETAIL: Citus cannot execute ADD CONSTRAINT command other than ADD CONSTRAINT FOREIGN KEY.
\c - - - :worker_1_port \c - - - :worker_1_port
\d nl_* \d nl_*
\c - - - :master_port \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 TABLE lineitem_alter ADD COLUMN int_column3 INTEGER,
ALTER COLUMN int_column1 SET STATISTICS 10; ALTER COLUMN int_column1 SET STATISTICS 10;
ERROR: alter table command is currently unsupported 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; ALTER TABLE lineitem_alter DROP COLUMN int_column1, DROP COLUMN int_column2;
\d lineitem_alter \d lineitem_alter
Table "public.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 -- Verify that we error out on unsupported statement types
ALTER TABLE lineitem_alter ALTER COLUMN l_orderkey SET STATISTICS 100; ALTER TABLE lineitem_alter ALTER COLUMN l_orderkey SET STATISTICS 100;
ERROR: alter table command is currently unsupported 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; ALTER TABLE lineitem_alter DROP CONSTRAINT IF EXISTS non_existent_contraint;
ERROR: alter table command is currently unsupported NOTICE: constraint "non_existent_contraint" of relation "lineitem_alter" does not exist, skipping
DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subcommands are supported.
ALTER TABLE lineitem_alter SET WITHOUT OIDS; ALTER TABLE lineitem_alter SET WITHOUT OIDS;
ERROR: alter table command is currently unsupported 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 -- Verify that we error out in case of postgres errors on supported statement
-- types -- types
ALTER TABLE lineitem_alter ADD COLUMN new_column non_existent_type; 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; SELECT * FROM referencing_table;
DROP TABLE referencing_table; DROP TABLE referencing_table;
DROP TABLE referenced_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;