mirror of https://github.com/citusdata/citus.git
Merge pull request #876 from citusdata/foreign_key_push_down_for_alter_table
Add Foreign Key Support to ALTER TABLE commandspull/1022/head
commit
62b12f413c
|
@ -485,18 +485,23 @@ 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 */
|
||||
/*
|
||||
* 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))
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
|
||||
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);
|
||||
if (relation->rd_id != referencedTableId &&
|
||||
(colocationId == INVALID_COLOCATION_ID ||
|
||||
|
@ -513,6 +518,15 @@ ErrorIfNotSupportedForeignConstraint(Relation relation, char distributionMethod,
|
|||
* 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"),
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
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 */
|
||||
if (alterTableStatement->relation != NULL)
|
||||
if (alterTableStatement->relation == NULL)
|
||||
{
|
||||
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
|
||||
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
|
||||
if (OidIsValid(relationId))
|
||||
return (Node *) alterTableStatement;
|
||||
}
|
||||
|
||||
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
|
||||
leftRelationId = AlterTableLookupRelation(alterTableStatement, lockmode);
|
||||
if (!OidIsValid(leftRelationId))
|
||||
{
|
||||
bool isDistributedRelation = IsDistributedTable(relationId);
|
||||
if (isDistributedRelation)
|
||||
return (Node *) alterTableStatement;
|
||||
}
|
||||
|
||||
isDistributedRelation = IsDistributedTable(leftRelationId);
|
||||
if (!isDistributedRelation)
|
||||
{
|
||||
return (Node *) 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;
|
||||
}
|
||||
|
||||
|
@ -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.
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
Loading…
Reference in New Issue