Merge pull request #2240 from citusdata/foreign_master

Foreign Key support from distributed to reference tables
pull/2251/head
Mehmet Furkan ŞAHİN 2018-07-03 17:55:59 +03:00 committed by GitHub
commit b480bf8d0e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 9860 additions and 251 deletions

View File

@ -64,6 +64,7 @@ OBJS = src/backend/distributed/shared_library_init.o \
src/backend/distributed/test/distributed_deadlock_detection.o \
src/backend/distributed/test/distribution_metadata.o \
src/backend/distributed/test/fake_fdw.o \
src/backend/distributed/test/foreign_key_relationship_query.o \
src/backend/distributed/test/generate_ddl_commands.o \
src/backend/distributed/test/metadata_sync.o \
src/backend/distributed/test/partitioning_utils.o \
@ -90,6 +91,7 @@ OBJS = src/backend/distributed/shared_library_init.o \
src/backend/distributed/utils/colocation_utils.o \
src/backend/distributed/utils/distribution_column.o \
src/backend/distributed/utils/errormessage.o \
src/backend/distributed/utils/foreign_key_relationship.o \
src/backend/distributed/utils/hash_helpers.o \
src/backend/distributed/utils/listutils.o \
src/backend/distributed/utils/maintenanced.o \

View File

@ -47,6 +47,7 @@
#include "distributed/pg_dist_colocation.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/reference_table_utils.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h"
#include "distributed/worker_protocol.h"
#include "distributed/worker_transaction.h"
@ -100,6 +101,7 @@ static bool LocalTableEmpty(Oid tableId);
static void CopyLocalDataIntoShards(Oid relationId);
static List * TupleDescColumnNameList(TupleDesc tupleDescriptor);
static bool RelationUsesIdentityColumns(TupleDesc relationDesc);
static bool CanUseExclusiveConnections(Oid relationId, bool localTableEmpty);
/* exports for SQL callable functions */
PG_FUNCTION_INFO_V1(master_create_distributed_table);
@ -484,26 +486,8 @@ CreateHashDistributedTableShards(Oid relationId, Oid colocatedTableId,
*/
if (RegularTable(relationId))
{
if (!localTableEmpty && MultiShardConnectionType == SEQUENTIAL_CONNECTION)
{
char *relationName = get_rel_name(relationId);
ereport(ERROR, (errmsg("cannot distribute \"%s\" in sequential mode "
"because it is not empty", relationName),
errhint("If you have manually set "
"citus.multi_shard_modify_mode to 'sequential', "
"try with 'parallel' option. If that is not the "
"case, try distributing local tables when they "
"are empty.")));
}
else if (MultiShardConnectionType == SEQUENTIAL_CONNECTION)
{
useExclusiveConnection = false;
}
else if (!localTableEmpty || IsTransactionBlock())
{
useExclusiveConnection = true;
}
useExclusiveConnection = CanUseExclusiveConnections(relationId,
localTableEmpty);
}
if (colocatedTableId != InvalidOid)
@ -776,6 +760,11 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn,
colocationId);
if (TableReferenced(relationId) || TableReferencing(relationId))
{
InvalidateForeignKeyGraph();
}
relation_close(relation, NoLock);
}
@ -1107,6 +1096,64 @@ LocalTableEmpty(Oid tableId)
}
/*
* CanUseExclusiveConnections checks if we can open parallel connections
* while creating shards. We simply error out if we need to execute
* sequentially but there is data in the table, since we cannot copy the
* data to shards sequentially.
*/
static bool
CanUseExclusiveConnections(Oid relationId, bool localTableEmpty)
{
bool hasForeignKeyToReferenceTable = HasForeignKeyToReferenceTable(relationId);
bool shouldRunSequential = MultiShardConnectionType == SEQUENTIAL_CONNECTION ||
hasForeignKeyToReferenceTable;
if (!localTableEmpty && shouldRunSequential)
{
char *relationName = get_rel_name(relationId);
ereport(ERROR, (errmsg("cannot distribute \"%s\" in sequential mode "
"because it is not empty", relationName),
errhint("If you have manually set "
"citus.multi_shard_modify_mode to 'sequential', "
"try with 'parallel' option. If that is not the "
"case, try distributing local tables when they "
"are empty.")));
}
else if (shouldRunSequential && ParallelQueryExecutedInTransaction())
{
char *relationName = get_rel_name(relationId);
/*
* If there has already been a parallel query executed, the sequential mode
* would still use the already opened parallel connections to the workers,
* thus contradicting our purpose of using sequential mode.
*/
ereport(ERROR, (errmsg("cannot distribute relation \"%s\" in this "
"transaction because it has a foreign key to "
"a reference table", relationName),
errdetail("If a hash distributed table has a foreign key "
"to a reference table, it has to be created "
"in sequential mode before any parallel commands "
"have been executed in the same transaction"),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
else if (shouldRunSequential)
{
return false;
}
else if (!localTableEmpty || IsTransactionBlock())
{
return true;
}
return false;
}
/*
* CreateTruncateTrigger creates a truncate trigger on table identified by relationId
* and assigns citus_truncate_trigger() as handler.

View File

@ -2324,6 +2324,9 @@ CitusCopyDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest)
/* mark as multi shard to skip doing the same thing over and over */
copyDest->multiShardCopy = true;
/* error out of conflicting COPY */
CheckConflictingParallelCopyAccesses(relationId);
/* when we see multiple shard connections, we mark COPY as parallel modify */
RecordParallelModifyAccess(relationId);
}

View File

@ -22,24 +22,91 @@
#include "distributed/master_protocol.h"
#include "distributed/multi_join_order.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/rel.h"
#include "utils/relcache.h"
#include "utils/ruleutils.h"
#include "utils/syscache.h"
static bool HeapTupleOfForeignConstraintIncludesColumn(HeapTuple heapTuple, Oid
relationId, int pgConstraintKey,
char *columnName);
/*
* ConstraintIsAForeignKeyToReferenceTable function scans the pgConstraint to
* fetch all of the constraints on the given relationId and see if at least one
* of them is a foreign key referencing to a reference table.
*/
bool
ConstraintIsAForeignKeyToReferenceTable(char *constraintName, Oid relationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
HeapTuple heapTuple = NULL;
bool foreignKeyToReferenceTable = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_contype, BTEqualStrategyNumber, F_CHAREQ,
CharGetDatum(CONSTRAINT_FOREIGN));
scanDescriptor = systable_beginscan(pgConstraint, InvalidOid, false,
NULL, scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Oid referencedTableId = InvalidOid;
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
char *constraintName = (constraintForm->conname).data;
if (strncmp(constraintName, constraintName, NAMEDATALEN) != 0 ||
constraintForm->conrelid != relationId)
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
referencedTableId = constraintForm->confrelid;
Assert(IsDistributedTable(referencedTableId));
if (PartitionMethod(referencedTableId) == DISTRIBUTE_BY_NONE)
{
foreignKeyToReferenceTable = true;
break;
}
heapTuple = systable_getnext(scanDescriptor);
}
/* clean up scan and close system catalog */
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
return foreignKeyToReferenceTable;
}
/*
* ErrorIfUnsupportedForeignConstraint runs checks related to foreign constraints and
* errors out if it is not possible to create one of the foreign constraint in distributed
* environment.
*
* To support foreign constraints, we require that;
* - Referencing and referenced tables are hash distributed.
* - If referencing and referenced tables are hash-distributed
* - Referencing and referenced tables are co-located.
* - Foreign constraint is defined over distribution column.
* - ON DELETE/UPDATE SET NULL, ON DELETE/UPDATE SET DEFAULT and ON UPDATE CASCADE options
* are not used.
* - Replication factors of referencing and referenced table are 1.
* - If referenced table is a reference table
* - ON DELETE/UPDATE SET NULL, ON DELETE/UPDATE SET DEFAULT and ON UPDATE CASCADE options
* are not used on the distribution key of the referencing column.
* - If referencing table is a reference table, error out
*/
void
ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
@ -56,16 +123,18 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
uint32 referencedTableColocationId = INVALID_COLOCATION_ID;
Var *referencedTablePartitionColumn = NULL;
Datum referencingColumnsDatum;
Datum *referencingColumnArray;
Datum referencingColumnsDatum = 0;
Datum *referencingColumnArray = NULL;
int referencingColumnCount = 0;
Datum referencedColumnsDatum;
Datum *referencedColumnArray;
Datum referencedColumnsDatum = 0;
Datum *referencedColumnArray = NULL;
int referencedColumnCount = 0;
bool isNull = false;
int attrIdx = 0;
bool foreignConstraintOnPartitionColumn = false;
bool selfReferencingTable = false;
bool referencedTableIsAReferenceTable = false;
bool referencingColumnsIncludeDistKey = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
@ -85,55 +154,26 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
continue;
}
/*
* We should make this check in this loop because the error message will only
* be given if the table has a foreign constraint and the table is a reference
* table.
*/
if (distributionMethod == DISTRIBUTE_BY_NONE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint because "
"reference tables are not supported as the "
"referencing table of a foreign constraint"),
errdetail("Reference tables are only supported as the "
"referenced table of a foreign key when the "
"referencing table is a hash distributed "
"table")));
}
referencedTableId = constraintForm->confrelid;
selfReferencingTable = referencingTableId == referencedTableId;
/*
* We do not support foreign keys for reference tables. Here we skip the second
* part of check if the table is a self referencing table because;
* - PartitionMethod only works for distributed tables and this table may not be
* distributed yet.
* - Since referencing and referenced tables are same, it is OK to not checking
* distribution method twice.
*/
if (distributionMethod == DISTRIBUTE_BY_NONE ||
(!selfReferencingTable &&
PartitionMethod(referencedTableId) == DISTRIBUTE_BY_NONE))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint from or to "
"reference tables")));
}
/*
* ON DELETE SET NULL and ON DELETE SET DEFAULT is not supported. Because we do
* not want to set partition column to NULL or default value.
*/
if (constraintForm->confdeltype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confdeltype == FKCONSTR_ACTION_SETDEFAULT)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL or SET DEFAULT is not supported"
" in ON DELETE operation.")));
}
/*
* ON UPDATE SET NULL, ON UPDATE SET DEFAULT and UPDATE CASCADE is not supported.
* Because we do not want to set partition column to NULL or default value. Also
* cascading update operation would require re-partitioning. Updating partition
* column value is not allowed anyway even outside of foreign key concept.
*/
if (constraintForm->confupdtype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confupdtype == FKCONSTR_ACTION_SETDEFAULT ||
constraintForm->confupdtype == FKCONSTR_ACTION_CASCADE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL, SET DEFAULT or CASCADE is not"
" supported in ON UPDATE operation.")));
}
/*
* Some checks are not meaningful if foreign key references the table itself.
* Therefore we will skip those checks.
@ -148,28 +188,43 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
"table.")));
}
/* to enforce foreign constraints, tables must be co-located */
referencedTableColocationId = TableColocationId(referencedTableId);
if (colocationId == INVALID_COLOCATION_ID ||
colocationId != referencedTableColocationId)
/*
* PartitionMethod errors out when it is called for non-distributed
* tables. This is why we make this check under !selfReferencingTable
* and after !IsDistributedTable(referencedTableId).
*/
if (PartitionMethod(referencedTableId) == DISTRIBUTE_BY_NONE)
{
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.")));
referencedTableIsAReferenceTable = true;
}
/*
* Partition column must exist in both referencing and referenced side of the
* foreign key constraint. They also must be in same ordinal.
* To enforce foreign constraints, tables must be co-located unless a
* reference table is referenced.
*/
referencedTableColocationId = TableColocationId(referencedTableId);
if (colocationId == INVALID_COLOCATION_ID ||
(colocationId != referencedTableColocationId &&
!referencedTableIsAReferenceTable))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint since "
"relations are not colocated or not referencing "
"a reference table"),
errdetail(
"A distributed table can only have foreign keys "
"if it is referencing another colocated hash "
"distributed table or a reference table")));
}
referencedTablePartitionColumn = DistPartitionKey(referencedTableId);
}
else
{
/*
* Partition column must exist in both referencing and referenced side of the
* foreign key constraint. They also must be in same ordinal.
* If the referenced table is not a reference table, the distribution
* column in referencing table should be the distribution column in
* referenced table as well.
*/
referencedTablePartitionColumn = distributionColumn;
}
@ -198,28 +253,83 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
AttrNumber referencedAttrNo = DatumGetInt16(referencedColumnArray[attrIdx]);
if (distributionColumn->varattno == referencingAttrNo &&
referencedTablePartitionColumn->varattno == referencedAttrNo)
(!referencedTableIsAReferenceTable &&
referencedTablePartitionColumn->varattno == referencedAttrNo))
{
foreignConstraintOnPartitionColumn = true;
}
if (distributionColumn->varattno == referencingAttrNo)
{
referencingColumnsIncludeDistKey = true;
}
}
if (!foreignConstraintOnPartitionColumn)
/*
* If columns in the foreign key includes the distribution key from the
* referencing side, we do not allow update/delete operations through
* foreign key constraints (e.g. ... ON UPDATE SET NULL)
*/
if (referencingColumnsIncludeDistKey)
{
/*
* ON DELETE SET NULL and ON DELETE SET DEFAULT is not supported. Because we do
* not want to set partition column to NULL or default value.
*/
if (constraintForm->confdeltype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confdeltype == FKCONSTR_ACTION_SETDEFAULT)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("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.")));
errdetail("SET NULL or SET DEFAULT is not supported"
" in ON DELETE operation when distribution "
"key is included in the foreign key constraint")));
}
/*
* ON UPDATE SET NULL, ON UPDATE SET DEFAULT and UPDATE CASCADE is not supported.
* Because we do not want to set partition column to NULL or default value. Also
* cascading update operation would require re-partitioning. Updating partition
* column value is not allowed anyway even outside of foreign key concept.
*/
if (constraintForm->confupdtype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confupdtype == FKCONSTR_ACTION_SETDEFAULT ||
constraintForm->confupdtype == FKCONSTR_ACTION_CASCADE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL, SET DEFAULT or CASCADE is not "
"supported in ON UPDATE operation when "
"distribution key included in the foreign "
"constraint.")));
}
}
/*
* if tables are hash-distributed and colocated, we need to make sure that
* the distribution key is included in foreign constraint.
*/
if (!referencedTableIsAReferenceTable && !foreignConstraintOnPartitionColumn)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Foreign keys are supported in two cases, "
"either in between two colocated tables including "
"partition column in the same ordinal in the both "
"tables or from distributed to reference tables")));
}
/*
* We do not allow to create foreign constraints if shard replication factor is
* greater than 1. Because in our current design, multiple replicas may cause
* locking problems and inconsistent shard contents. We don't check the referenced
* table, since referenced and referencing tables should be co-located and
* colocation check has been done above.
* locking problems and inconsistent shard contents.
*
* Note that we allow referenced table to be a reference table (e.g., not a
* single replicated table). This is allowed since (a) we are sure that
* placements always be in the same state (b) executors are aware of reference
* tables and handle concurrency related issues accordingly.
*/
if (IsDistributedTable(referencingTableId))
{
@ -231,6 +341,8 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
}
else
{
Assert(distributionMethod == DISTRIBUTE_BY_HASH);
/* check whether creating single replicated table with foreign constraint */
if (ShardReplicationFactor > 1)
{
@ -260,6 +372,87 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
}
/*
* ColumnAppearsInForeignKeyToReferenceTable checks if there is foreign constraint
* from/to a reference table on the given column. We iterate pgConstraint to fetch
* the constraint on the given relationId and find if any of the constraints
* includes the given column.
*/
bool
ColumnAppearsInForeignKeyToReferenceTable(char *columnName, Oid relationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
HeapTuple heapTuple = NULL;
bool foreignKeyToReferenceTableIncludesGivenColumn = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_contype, BTEqualStrategyNumber, F_CHAREQ,
CharGetDatum(CONSTRAINT_FOREIGN));
scanDescriptor = systable_beginscan(pgConstraint, InvalidOid, false,
NULL, scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Oid referencedTableId = InvalidOid;
Oid referencingTableId = InvalidOid;
int pgConstraintKey = 0;
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
referencedTableId = constraintForm->confrelid;
referencingTableId = constraintForm->conrelid;
if (referencedTableId == relationId)
{
pgConstraintKey = Anum_pg_constraint_confkey;
}
else if (referencingTableId == relationId)
{
pgConstraintKey = Anum_pg_constraint_conkey;
}
else
{
/*
* If the constraint is not from/to the given relation, we should simply
* skip.
*/
heapTuple = systable_getnext(scanDescriptor);
continue;
}
/*
* We check if the referenced table is a reference table. There cannot be
* any foreign constraint from a distributed table to a local table.
*/
Assert(IsDistributedTable(referencedTableId));
if (PartitionMethod(referencedTableId) != DISTRIBUTE_BY_NONE)
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
if (HeapTupleOfForeignConstraintIncludesColumn(heapTuple, relationId,
pgConstraintKey, columnName))
{
foreignKeyToReferenceTableIncludesGivenColumn = true;
break;
}
heapTuple = systable_getnext(scanDescriptor);
}
/* clean up scan and close system catalog */
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
return foreignKeyToReferenceTableIncludesGivenColumn;
}
/*
* GetTableForeignConstraints takes in a relationId, and returns the list of foreign
* constraint commands needed to reconstruct foreign constraints of that table.
@ -321,6 +514,62 @@ GetTableForeignConstraintCommands(Oid relationId)
}
/*
* HasForeignKeyToReferenceTable function scans the pgConstraint table to
* fetch all of the constraints on the given relationId and see if at least one
* of them is a foreign key referencing to a reference table.
*/
bool
HasForeignKeyToReferenceTable(Oid relationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
HeapTuple heapTuple = NULL;
bool hasForeignKeyToReferenceTable = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
relationId);
scanDescriptor = systable_beginscan(pgConstraint, ConstraintRelidIndexId, true, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Oid referencedTableId = InvalidOid;
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
if (constraintForm->contype != CONSTRAINT_FOREIGN)
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
referencedTableId = constraintForm->confrelid;
if (!IsDistributedTable(referencedTableId))
{
continue;
}
if (PartitionMethod(referencedTableId) == DISTRIBUTE_BY_NONE)
{
hasForeignKeyToReferenceTable = true;
break;
}
heapTuple = systable_getnext(scanDescriptor);
}
/* clean up scan and close system catalog */
systable_endscan(scanDescriptor);
heap_close(pgConstraint, NoLock);
return hasForeignKeyToReferenceTable;
}
/*
* TableReferenced function checks whether given table is referenced by another table
* via foreign constraints. If it is referenced, this function returns true. To check
@ -367,3 +616,130 @@ TableReferenced(Oid relationId)
return false;
}
/*
* HeapTupleOfForeignConstraintIncludesColumn fetches the columns from the foreign
* constraint and checks if the given column name matches one of them.
*/
static bool
HeapTupleOfForeignConstraintIncludesColumn(HeapTuple heapTuple, Oid relationId,
int pgConstraintKey, char *columnName)
{
Datum columnsDatum = 0;
Datum *columnArray = NULL;
int columnCount = 0;
int attrIdx = 0;
bool isNull = false;
columnsDatum = SysCacheGetAttr(CONSTROID, heapTuple, pgConstraintKey, &isNull);
deconstruct_array(DatumGetArrayTypeP(columnsDatum), INT2OID, 2, true,
's', &columnArray, NULL, &columnCount);
for (attrIdx = 0; attrIdx < columnCount; ++attrIdx)
{
AttrNumber attrNo = DatumGetInt16(columnArray[attrIdx]);
char *colName = get_relid_attribute_name(relationId, attrNo);
if (strncmp(colName, columnName, NAMEDATALEN) == 0)
{
return true;
}
}
return false;
}
/*
* TableReferencing function checks whether given table is referencing by another table
* via foreign constraints. If it is referencing, this function returns true. To check
* that, this function searches given relation at pg_constraints system catalog. However
* since there is no index for the column we searched, this function performs sequential
* search, therefore call this function with caution.
*/
bool
TableReferencing(Oid relationId)
{
Relation pgConstraint = NULL;
HeapTuple heapTuple = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Oid scanIndexId = InvalidOid;
bool useIndex = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
relationId);
scanDescriptor = systable_beginscan(pgConstraint, scanIndexId, useIndex, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
if (constraintForm->contype == CONSTRAINT_FOREIGN)
{
systable_endscan(scanDescriptor);
heap_close(pgConstraint, NoLock);
return true;
}
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
heap_close(pgConstraint, NoLock);
return false;
}
/*
* ConstraintIsAForeignKey returns true if the given constraint name
* is a foreign key to defined on the relation.
*/
bool
ConstraintIsAForeignKey(char *constraintNameInput, Oid relationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
HeapTuple heapTuple = NULL;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_contype, BTEqualStrategyNumber, F_CHAREQ,
CharGetDatum(CONSTRAINT_FOREIGN));
scanDescriptor = systable_beginscan(pgConstraint, InvalidOid, false,
NULL, scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
char *constraintName = (constraintForm->conname).data;
if (strncmp(constraintName, constraintNameInput, NAMEDATALEN) == 0 &&
constraintForm->conrelid == relationId)
{
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
return true;
}
heapTuple = systable_getnext(scanDescriptor);
}
/* clean up scan and close system catalog */
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
return false;
}

View File

@ -310,8 +310,6 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
void
SetLocalMultiShardModifyModeToSequential()
{
WarnNoTransactionChain(true, "SET LOCAL");
set_config_option("citus.multi_shard_modify_mode", "sequential",
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
GUC_ACTION_LOCAL, true, 0, false);

View File

@ -49,6 +49,7 @@
#include "distributed/multi_shard_transaction.h"
#include "distributed/multi_utility.h" /* IWYU pragma: keep */
#include "distributed/pg_dist_partition.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/resource_lock.h"
#include "distributed/transaction_management.h"
#include "distributed/transmit.h"
@ -75,6 +76,7 @@
#include "utils/builtins.h"
#include "utils/elog.h"
#include "utils/errcodes.h"
#include "utils/fmgroids.h"
#include "utils/guc.h"
#include "utils/hsearch.h"
#include "utils/inval.h"
@ -86,6 +88,10 @@
bool EnableDDLPropagation = true; /* ddl propagation is enabled */
static bool shouldInvalidateForeignKeyGraph = false;
/*
* This struct defines the state for the callback for drop statements.
* It is copied as it is from commands/tablecmds.c in Postgres source.
@ -162,7 +168,17 @@ static void CheckCopyPermissions(CopyStmt *copyStatement);
static List * CopyGetAttnums(TupleDesc tupDesc, Relation rel, List *attnamelist);
static void PostProcessUtility(Node *parsetree);
static List * CollectGrantTableIdList(GrantStmt *grantStmt);
static char * GetSchemaNameFromDropObject(ListCell *dropSchemaCell);
static void ProcessDropTableStmt(DropStmt *dropTableStatement);
static void ProcessDropSchemaStmt(DropStmt *dropSchemaStatement);
static void InvalidateForeignKeyGraphForDDL(void);
/*
* We need to run some of the commands sequentially if there is a foreign constraint
* from/to reference table.
*/
static bool ShouldExecuteAlterTableSequentially(Oid relationId, AlterTableCmd *command);
/*
@ -365,6 +381,11 @@ multi_ProcessUtility(PlannedStmt *pstmt,
{
ProcessDropTableStmt(dropStatement);
}
if (dropStatement->removeType == OBJECT_SCHEMA)
{
ProcessDropSchemaStmt(dropStatement);
}
}
if (IsA(parsetree, AlterTableStmt))
@ -524,6 +545,16 @@ multi_ProcessUtility(PlannedStmt *pstmt,
SetUserIdAndSecContext(savedUserId, savedSecurityContext);
}
/*
* Re-forming the foreign key graph relies on the command being executed
* on the local table first. However, in order to decide whether the
* command leads to an invalidation, we need to check before the command
* is being executed since we read pg_constraint table. Thus, we maintain a
* local flag and do the invalidation after multi_ProcessUtility,
* before ExecuteDistributedDDLJob().
*/
InvalidateForeignKeyGraphForDDL();
/* after local command has completed, finish by executing worker DDLJobs, if any */
if (ddlJobs != NIL)
{
@ -550,8 +581,27 @@ multi_ProcessUtility(PlannedStmt *pstmt,
if (alterTableType == AT_AddConstraint)
{
LOCKMODE lockmode = NoLock;
Oid relationId = InvalidOid;
Constraint *constraint = NULL;
Assert(list_length(commandList) == 1);
ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement);
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId))
{
continue;
}
constraint = (Constraint *) command->def;
if (ConstraintIsAForeignKey(constraint->conname, relationId))
{
InvalidateForeignKeyGraph();
}
}
}
}
@ -611,6 +661,22 @@ multi_ProcessUtility(PlannedStmt *pstmt,
}
/*
* InvalidateForeignKeyGraphForDDL simply keeps track of whether
* the foreign key graph should be invalidated due to a DDL.
*/
static void
InvalidateForeignKeyGraphForDDL(void)
{
if (shouldInvalidateForeignKeyGraph)
{
InvalidateForeignKeyGraph();
shouldInvalidateForeignKeyGraph = false;
}
}
/*
* IsCitusExtensionStmt returns whether a given utility is a CREATE or ALTER
* EXTENSION statement which references the citus extension. This function
@ -1247,6 +1313,7 @@ PlanAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCo
bool isDistributedRelation = false;
List *commandList = NIL;
ListCell *commandCell = NULL;
bool executeSequentially = false;
/* first check whether a distributed relation is affected */
if (alterTableStatement->relation == NULL)
@ -1295,8 +1362,8 @@ PlanAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCo
}
/*
* 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
* We check if there is a ADD/DROP FOREIGN CONSTRAINT command in sub commands list.
* If there is we assign referenced relation 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.
*/
@ -1368,19 +1435,29 @@ PlanAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCo
rightRelationId = RangeVarGetRelid(partitionCommand->name, NoLock, false);
}
#endif
executeSequentially |= ShouldExecuteAlterTableSequentially(leftRelationId,
command);
}
ddlJob = palloc0(sizeof(DDLJob));
ddlJob->targetRelationId = leftRelationId;
ddlJob->concurrentIndexCmd = false;
ddlJob->commandString = alterTableCommand;
ddlJob->executeSequentially = executeSequentially;
if (rightRelationId)
{
if (!IsDistributedTable(rightRelationId))
{
ddlJob->taskList = NIL;
}
else
{
/* if foreign key related, use specialized task list function ... */
ddlJob->taskList = InterShardDDLTaskList(leftRelationId, rightRelationId,
alterTableCommand);
}
}
else
{
/* ... otherwise use standard DDL task list function */
@ -2227,8 +2304,25 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
}
#endif
case AT_SetNotNull:
case AT_DropConstraint:
{
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId))
{
return;
}
if (ConstraintIsAForeignKey(command->name, relationId))
{
shouldInvalidateForeignKeyGraph = true;
}
break;
}
case AT_SetNotNull:
case AT_EnableTrigAll:
case AT_DisableTrigAll:
case AT_ReplicaIdentity:
@ -2885,13 +2979,14 @@ ExecuteDistributedDDLJob(DDLJob *ddlJob)
SendCommandToWorkers(WORKERS_WITH_METADATA, (char *) ddlJob->commandString);
}
if (MultiShardConnectionType == PARALLEL_CONNECTION)
if (MultiShardConnectionType == SEQUENTIAL_CONNECTION ||
ddlJob->executeSequentially)
{
ExecuteModifyTasksWithoutResults(ddlJob->taskList);
ExecuteModifyTasksSequentiallyWithoutResults(ddlJob->taskList, CMD_UTILITY);
}
else
{
ExecuteModifyTasksSequentiallyWithoutResults(ddlJob->taskList, CMD_UTILITY);
ExecuteModifyTasksWithoutResults(ddlJob->taskList);
}
}
else
@ -3102,6 +3197,7 @@ InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
char *leftSchemaName = get_namespace_name(leftSchemaId);
char *escapedLeftSchemaName = quote_literal_cstr(leftSchemaName);
char rightPartitionMethod = PartitionMethod(rightRelationId);
List *rightShardList = LoadShardIntervalList(rightRelationId);
ListCell *rightShardCell = NULL;
Oid rightSchemaId = get_rel_namespace(rightRelationId);
@ -3112,6 +3208,29 @@ InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
uint64 jobId = INVALID_JOB_ID;
int taskId = 1;
/*
* If the rightPartitionMethod is a reference table, we need to make sure
* that the tasks are created in a way that the right shard stays the same
* since we only have one placement per worker. This hack is first implemented
* for foreign constraint support from distributed tables to reference tables.
*/
if (rightPartitionMethod == DISTRIBUTE_BY_NONE)
{
ShardInterval *rightShardInterval = NULL;
int rightShardCount = list_length(rightShardList);
int leftShardCount = list_length(leftShardList);
int shardCounter = 0;
Assert(rightShardCount == 1);
rightShardInterval = (ShardInterval *) linitial(rightShardList);
for (shardCounter = rightShardCount; shardCounter < leftShardCount;
shardCounter++)
{
rightShardList = lappend(rightShardList, rightShardInterval);
}
}
/* lock metadata before getting placement lists */
LockShardListMetadata(leftShardList, ShareLock);
@ -3705,6 +3824,99 @@ RoleSpecString(RoleSpec *spec)
}
/*
* ProcessDropSchemaStmt invalidates the foreign key cache if any table created
* under dropped schema involved in any foreign key relationship.
*/
static void
ProcessDropSchemaStmt(DropStmt *dropStatement)
{
Relation pgClass = NULL;
HeapTuple heapTuple = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Oid scanIndexId = InvalidOid;
bool useIndex = false;
ListCell *dropSchemaCell;
if (dropStatement->behavior != DROP_CASCADE)
{
return;
}
foreach(dropSchemaCell, dropStatement->objects)
{
char *schemaString = GetSchemaNameFromDropObject(dropSchemaCell);
Oid namespaceOid = get_namespace_oid(schemaString, true);
if (namespaceOid == InvalidOid)
{
continue;
}
pgClass = heap_open(RelationRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_class_relnamespace, BTEqualStrategyNumber,
F_OIDEQ, namespaceOid);
scanDescriptor = systable_beginscan(pgClass, scanIndexId, useIndex, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_class relationForm = (Form_pg_class) GETSTRUCT(heapTuple);
char *relationName = NameStr(relationForm->relname);
Oid relationId = get_relname_relid(relationName, namespaceOid);
/* we're not interested in non-valid, non-distributed relations */
if (relationId == InvalidOid || !IsDistributedTable(relationId))
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
/* invalidate foreign key cache if the table involved in any foreign key */
if (TableReferenced(relationId) || TableReferencing(relationId))
{
shouldInvalidateForeignKeyGraph = true;
systable_endscan(scanDescriptor);
heap_close(pgClass, NoLock);
return;
}
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
heap_close(pgClass, NoLock);
}
}
/*
* GetSchemaNameFromDropObject gets the name of the drop schema from given
* list cell. This function is defined due to API change between PG 9.6 and
* PG 10.
*/
static char *
GetSchemaNameFromDropObject(ListCell *dropSchemaCell)
{
char *schemaString = NULL;
#if (PG_VERSION_NUM >= 100000)
Value *schemaValue = (Value *) lfirst(dropSchemaCell);
schemaString = strVal(schemaValue);
#else
List *schemaNameList = (List *) lfirst(dropSchemaCell);
schemaString = NameListToString(schemaNameList);
#endif
return schemaString;
}
/*
* ProcessDropTableStmt processes DROP TABLE commands for partitioned tables.
* If we are trying to DROP partitioned tables, we first need to go to MX nodes
@ -3732,10 +3944,20 @@ ProcessDropTableStmt(DropStmt *dropTableStatement)
Oid relationId = RangeVarGetRelid(tableRangeVar, AccessShareLock, missingOK);
if (relationId == InvalidOid ||
!IsDistributedTable(relationId) ||
!ShouldSyncTableMetadata(relationId) ||
!PartitionedTable(relationId))
/* we're not interested in non-valid, non-distributed relations */
if (relationId == InvalidOid || !IsDistributedTable(relationId))
{
continue;
}
/* invalidate foreign key cache if the table involved in any foreign key */
if ((TableReferenced(relationId) || TableReferencing(relationId)))
{
shouldInvalidateForeignKeyGraph = true;
}
/* we're only interested in partitioned and mx tables */
if (!ShouldSyncTableMetadata(relationId) || !PartitionedTable(relationId))
{
continue;
}
@ -3760,3 +3982,90 @@ ProcessDropTableStmt(DropStmt *dropTableStatement)
}
}
}
/*
* ShouldExecuteAlterTableSequentially checks if the given ALTER TABLE
* statements should be executed sequentially when there is a foreign
* constraint from a distributed table to a reference table.
* In case of a column related ALTER TABLE operation, we check explicitly
* if there is a foreign constraint on this column from/to a reference table.
* Additionally, if the command is run inside a transaction block, we call
* SetLocalMultiShardModifyModeToSequential so that the further commands
* in the same transaction uses the same connections and does not error out.
*/
static bool
ShouldExecuteAlterTableSequentially(Oid relationId, AlterTableCmd *command)
{
bool executeSequentially = false;
AlterTableType alterTableType = command->subtype;
if (alterTableType == AT_DropConstraint)
{
char *constraintName = command->name;
if (ConstraintIsAForeignKeyToReferenceTable(constraintName, relationId))
{
executeSequentially = true;
}
}
else if (alterTableType == AT_DropColumn || alterTableType == AT_AlterColumnType)
{
char *affectedColumnName = command->name;
if (ColumnAppearsInForeignKeyToReferenceTable(affectedColumnName,
relationId))
{
if (IsTransactionBlock() && alterTableType == AT_AlterColumnType)
{
SetLocalMultiShardModifyModeToSequential();
}
executeSequentially = true;
}
}
else if (alterTableType == AT_AddConstraint)
{
/*
* We need to execute the ddls working with reference tables on the
* right side sequentially, because parallel ddl operations
* relating to one and only shard of a reference table on a worker
* may cause self-deadlocks.
*/
Constraint *constraint = (Constraint *) command->def;
if (constraint->contype == CONSTR_FOREIGN)
{
Oid rightRelationId = RangeVarGetRelid(constraint->pktable, NoLock,
false);
if (IsDistributedTable(rightRelationId) &&
PartitionMethod(rightRelationId) == DISTRIBUTE_BY_NONE)
{
executeSequentially = true;
}
}
}
/*
* If there has already been a parallel query executed, the sequential mode
* would still use the already opened parallel connections to the workers for
* the distributed tables, thus contradicting our purpose of using
* sequential mode.
*/
if (executeSequentially && IsDistributedTable(relationId) &&
PartitionMethod(relationId) != DISTRIBUTE_BY_NONE &&
ParallelQueryExecutedInTransaction())
{
char *relationName = get_rel_name(relationId);
ereport(ERROR, (errmsg("cannot modify table \"%s\" because there "
"was a parallel operation on a distributed table"
"in the transaction", relationName),
errdetail("When there is a foreign key to a reference "
"table, Citus needs to perform all operations "
"over a single connection per node to ensure "
"consistency."),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
return executeSequentially;
}

View File

@ -26,6 +26,7 @@
#include "commands/event_trigger.h"
#include "distributed/citus_clauses.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/foreign_constraint.h"
#include "distributed/listutils.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/master_protocol.h"
@ -57,6 +58,7 @@
static List * ModifyMultipleShardsTaskList(Query *query, List *shardIntervalList, TaskType
taskType);
static bool ShouldExecuteTruncateStmtSequential(TruncateStmt *command);
PG_FUNCTION_INFO_V1(master_modify_multiple_shards);
@ -134,6 +136,11 @@ master_modify_multiple_shards(PG_FUNCTION_ARGS)
}
EnsureTablePermissions(relationId, ACL_TRUNCATE);
if (ShouldExecuteTruncateStmtSequential(truncateStatement))
{
SetLocalMultiShardModifyModeToSequential();
}
}
else
{
@ -243,3 +250,34 @@ ModifyMultipleShardsTaskList(Query *query, List *shardIntervalList, TaskType tas
return taskList;
}
/*
* ShouldExecuteTruncateStmtSequential decides if the TRUNCATE stmt needs
* to run sequential. If so, it calls SetLocalMultiShardModifyModeToSequential().
*
* If a reference table which has a foreign key from a distributed table is truncated
* we need to execute the command sequentially to avoid self-deadlock.
*/
static bool
ShouldExecuteTruncateStmtSequential(TruncateStmt *command)
{
List *relationList = command->relations;
ListCell *relationCell = NULL;
bool failOK = false;
foreach(relationCell, relationList)
{
RangeVar *rangeVar = (RangeVar *) lfirst(relationCell);
Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK);
if (IsDistributedTable(relationId) &&
PartitionMethod(relationId) == DISTRIBUTE_BY_NONE &&
TableReferenced(relationId))
{
return true;
}
}
return false;
}

View File

@ -188,6 +188,16 @@ RepairShardPlacement(int64 shardId, char *sourceNodeName, int32 sourceNodePort,
"not supported.", relationName)));
}
/*
* We take a lock on the referenced table if there is a foreign constraint
* during the copy procedure. If we do not block DMLs on the referenced
* table, we cannot avoid the inconsistency between the two copies of the
* data. Currently, we do not support replication factor > 1 on the tables
* with foreign constraints, so this command will fail for this case anyway.
* However, it is taken as a precaution in case we support it one day.
*/
LockReferencedReferenceShardDistributionMetadata(shardId, ExclusiveLock);
/*
* We plan to move the placement to the healthy state, so we need to grab a shard
* metadata lock (in exclusive mode).
@ -338,8 +348,31 @@ CopyShardCommandList(ShardInterval *shardInterval,
List *
CopyShardForeignConstraintCommandList(ShardInterval *shardInterval)
{
List *copyShardForeignConstraintCommandList = NIL;
List *colocatedShardForeignConstraintCommandList = NIL;
List *referenceTableForeignConstraintList = NIL;
CopyShardForeignConstraintCommandListGrouped(shardInterval,
&
colocatedShardForeignConstraintCommandList,
&referenceTableForeignConstraintList);
return list_concat(colocatedShardForeignConstraintCommandList,
referenceTableForeignConstraintList);
}
/*
* CopyShardForeignConstraintCommandListGrouped generates command lists
* to create foreign constraints existing in source shard after copying it to other
* node in separate groups for foreign constraints in between hash distributed tables
* and from a hash distributed to reference tables.
*/
void
CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
List **
colocatedShardForeignConstraintCommandList,
List **referenceTableForeignConstraintList)
{
Oid schemaId = get_rel_namespace(shardInterval->relationId);
char *schemaName = get_namespace_name(schemaId);
char *escapedSchemaName = quote_literal_cstr(schemaName);
@ -354,6 +387,9 @@ CopyShardForeignConstraintCommandList(ShardInterval *shardInterval)
shardIndex = ShardIndex(shardInterval);
}
*colocatedShardForeignConstraintCommandList = NIL;
*referenceTableForeignConstraintList = NIL;
foreach(commandCell, commandList)
{
char *command = (char *) lfirst(commandCell);
@ -364,6 +400,7 @@ CopyShardForeignConstraintCommandList(ShardInterval *shardInterval)
char *referencedSchemaName = NULL;
char *escapedReferencedSchemaName = NULL;
uint64 referencedShardId = INVALID_SHARD_ID;
bool colocatedForeignKey = false;
StringInfo applyForeignConstraintCommand = makeStringInfo();
@ -379,19 +416,40 @@ CopyShardForeignConstraintCommandList(ShardInterval *shardInterval)
referencedSchemaId = get_rel_namespace(referencedRelationId);
referencedSchemaName = get_namespace_name(referencedSchemaId);
escapedReferencedSchemaName = quote_literal_cstr(referencedSchemaName);
referencedShardId = ColocatedShardIdInRelation(referencedRelationId, shardIndex);
if (PartitionMethod(referencedRelationId) == DISTRIBUTE_BY_NONE)
{
List *shardList = LoadShardList(referencedRelationId);
uint64 *shardIdPointer = (uint64 *) linitial(shardList);
referencedShardId = (*shardIdPointer);
}
else
{
referencedShardId = ColocatedShardIdInRelation(referencedRelationId,
shardIndex);
colocatedForeignKey = true;
}
appendStringInfo(applyForeignConstraintCommand,
WORKER_APPLY_INTER_SHARD_DDL_COMMAND, shardInterval->shardId,
escapedSchemaName, referencedShardId,
escapedReferencedSchemaName, escapedCommand);
copyShardForeignConstraintCommandList = lappend(
copyShardForeignConstraintCommandList,
if (colocatedForeignKey)
{
*colocatedShardForeignConstraintCommandList = lappend(
*colocatedShardForeignConstraintCommandList,
applyForeignConstraintCommand->data);
}
return copyShardForeignConstraintCommandList;
else
{
*referenceTableForeignConstraintList = lappend(
*referenceTableForeignConstraintList,
applyForeignConstraintCommand->data);
}
}
}

View File

@ -662,12 +662,22 @@ WorkerCreateShard(Oid relationId, int shardIndex, uint64 shardId, List *ddlComma
* 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.
* itself, referencedShardId is actual shardId anyway. Also, if the referenced
* relation is a reference table, we cannot use ColocatedShardIdInRelation since
* reference tables only have one shard. Instead, we fetch the one and only shard
* from shardlist and use it.
*/
if (relationId == referencedRelationId)
{
referencedShardId = shardId;
}
else if (PartitionMethod(referencedRelationId) == DISTRIBUTE_BY_NONE)
{
List *shardList = LoadShardList(referencedRelationId);
uint64 *shardIdPointer = (uint64 *) linitial(shardList);
referencedShardId = (*shardIdPointer);
}
else
{
referencedShardId = ColocatedShardIdInRelation(referencedRelationId,

View File

@ -38,6 +38,7 @@
#include "distributed/multi_utility.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/placement_connection.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/query_pushdown_planning.h"
#include "distributed/query_stats.h"
#include "distributed/remote_commands.h"
@ -401,6 +402,20 @@ RegisterCitusConfigVariables(void)
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enforce_foreign_key_restrictions",
gettext_noop("Enforce restrictions while querying distributed/reference "
"tables with foreign keys"),
gettext_noop("When enabled, cascading modifications from reference tables "
"to distributed tables are traced and acted accordingly "
"to avoid creating distributed deadlocks and ensure correctness."),
&EnforceForeignKeyRestrictions,
true,
PGC_USERSET,
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.subquery_pushdown",
gettext_noop("Enables supported subquery pushdown to workers."),

View File

@ -0,0 +1,123 @@
/*-------------------------------------------------------------------------
*
* foreign_key_relationship_query.c
*
* This file contains UDFs for getting foreign constraint relationship between
* distributed tables.
*
* Copyright (c) 2018, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "fmgr.h"
#include "funcapi.h"
#include "distributed/metadata_cache.h"
/* these functions are only exported in the regression tests */
PG_FUNCTION_INFO_V1(get_referencing_relation_id_list);
PG_FUNCTION_INFO_V1(get_referenced_relation_id_list);
/*
* get_referencing_relation_id_list returns the list of table oids that is referencing
* by given oid recursively. It uses the list cached in the distributed table cache
* entry.
*/
Datum
get_referencing_relation_id_list(PG_FUNCTION_ARGS)
{
FuncCallContext *functionContext = NULL;
ListCell *foreignRelationCell = NULL;
CheckCitusVersion(ERROR);
/* for the first we call this UDF, we need to populate the result to return set */
if (SRF_IS_FIRSTCALL())
{
Oid relationId = PG_GETARG_OID(0);
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
List *refList = cacheEntry->referencingRelationsViaForeignKey;
/* create a function context for cross-call persistence */
functionContext = SRF_FIRSTCALL_INIT();
foreignRelationCell = list_head(refList);
functionContext->user_fctx = foreignRelationCell;
}
/*
* On every call to this function, we get the current position in the
* statement list. We then iterate to the next position in the list and
* return the current statement, if we have not yet reached the end of
* list.
*/
functionContext = SRF_PERCALL_SETUP();
foreignRelationCell = (ListCell *) functionContext->user_fctx;
if (foreignRelationCell != NULL)
{
Oid refId = lfirst_oid(foreignRelationCell);
functionContext->user_fctx = lnext(foreignRelationCell);
SRF_RETURN_NEXT(functionContext, PointerGetDatum(refId));
}
else
{
SRF_RETURN_DONE(functionContext);
}
}
/*
* get_referenced_relation_id_list returns the list of table oids that is referenced
* by given oid recursively. It uses the list cached in the distributed table cache
* entry.
*/
Datum
get_referenced_relation_id_list(PG_FUNCTION_ARGS)
{
FuncCallContext *functionContext = NULL;
ListCell *foreignRelationCell = NULL;
CheckCitusVersion(ERROR);
/* for the first we call this UDF, we need to populate the result to return set */
if (SRF_IS_FIRSTCALL())
{
Oid relationId = PG_GETARG_OID(0);
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
List *refList = cacheEntry->referencedRelationsViaForeignKey;
/* create a function context for cross-call persistence */
functionContext = SRF_FIRSTCALL_INIT();
foreignRelationCell = list_head(refList);
functionContext->user_fctx = foreignRelationCell;
}
/*
* On every call to this function, we get the current position in the
* statement list. We then iterate to the next position in the list and
* return the current statement, if we have not yet reached the end of
* list.
*/
functionContext = SRF_PERCALL_SETUP();
foreignRelationCell = (ListCell *) functionContext->user_fctx;
if (foreignRelationCell != NULL)
{
Oid refId = lfirst_oid(foreignRelationCell);
functionContext->user_fctx = lnext(foreignRelationCell);
SRF_RETURN_NEXT(functionContext, PointerGetDatum(refId));
}
else
{
SRF_RETURN_DONE(functionContext);
}
}

View File

@ -20,14 +20,27 @@
#include "access/xact.h"
#include "distributed/colocation_utils.h"
#include "distributed/hash_helpers.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_join_order.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/metadata_cache.h"
#include "distributed/relation_access_tracking.h"
#include "utils/hsearch.h"
#include "utils/lsyscache.h"
/* Config variables managed via guc.c */
bool EnforceForeignKeyRestrictions = true;
#define PARALLEL_MODE_FLAG_OFFSET 3
#define PARALLEL_ACCESS_MASK (int) (0 | \
(1 << (PLACEMENT_ACCESS_SELECT + \
PARALLEL_MODE_FLAG_OFFSET)) | \
(1 << (PLACEMENT_ACCESS_DML + \
PARALLEL_MODE_FLAG_OFFSET)) | \
(1 << (PLACEMENT_ACCESS_DDL + \
PARALLEL_MODE_FLAG_OFFSET)))
/*
* Hash table mapping relations to the
@ -66,6 +79,7 @@ typedef struct RelationAccessHashEntry
static HTAB *RelationAccessHash;
/* functions related to access recording */
static void RecordRelationAccess(Oid relationId, ShardPlacementAccessType accessType);
static void RecordPlacementAccessToCache(Oid relationId,
ShardPlacementAccessType accessType);
@ -76,6 +90,26 @@ static void RecordParallelRelationAccess(Oid relationId, ShardPlacementAccessTyp
static void RecordParallelRelationAccessToCache(Oid relationId,
ShardPlacementAccessType placementAccess);
/* functions related to access conflict checks */
static char * PlacementAccessTypeToText(ShardPlacementAccessType accessType);
static void CheckConflictingRelationAccesses(Oid relationId,
ShardPlacementAccessType accessType);
static bool HoldsConflictingLockWithReferencingRelations(Oid relationId,
ShardPlacementAccessType
placementAccess,
Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode);
static void CheckConflictingParallelRelationAccesses(Oid relationId,
ShardPlacementAccessType
accessType);
static bool HoldsConflictingLockWithReferencedRelations(Oid relationId,
ShardPlacementAccessType
placementAccess,
Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode);
/*
* Empty RelationAccessHash, without destroying the hash table itself.
@ -131,6 +165,40 @@ AssociatePlacementAccessWithRelation(ShardPlacement *placement,
}
/*
* PlacementAccessTypeToText converts ShardPlacementAccessType to
* text representation.
*/
static char *
PlacementAccessTypeToText(ShardPlacementAccessType accessType)
{
switch (accessType)
{
case PLACEMENT_ACCESS_SELECT:
{
return "SELECT";
break;
}
case PLACEMENT_ACCESS_DML:
{
return "DML";
}
case PLACEMENT_ACCESS_DDL:
{
return "DDL";
}
default:
{
return "None";
break;
}
}
}
/*
* RecordRelationAccess associates the access to the distributed relation. The
* function takes partitioned relations into account as well.
@ -142,6 +210,9 @@ AssociatePlacementAccessWithRelation(ShardPlacement *placement,
static void
RecordRelationAccess(Oid relationId, ShardPlacementAccessType accessType)
{
/* make sure that this is not a conflicting access */
CheckConflictingRelationAccesses(relationId, accessType);
/*
* If a relation is partitioned, record accesses to all of its partitions as well.
* We prefer to use PartitionedTableNoLock() because at this point the necessary
@ -374,6 +445,9 @@ RecordParallelRelationAccess(Oid relationId, ShardPlacementAccessType placementA
return;
}
/* act accordingly if it's a conflicting access */
CheckConflictingParallelRelationAccesses(relationId, placementAccess);
/*
* If a relation is partitioned, record accesses to all of its partitions as well.
* We prefer to use PartitionedTableNoLock() because at this point the necessary
@ -434,6 +508,40 @@ RecordParallelRelationAccessToCache(Oid relationId,
}
/*
* ParallelQueryExecutedInTransaction returns true if any parallel query
* is executed in the current transaction.
*/
bool
ParallelQueryExecutedInTransaction(void)
{
HASH_SEQ_STATUS status;
RelationAccessHashEntry *hashEntry;
if (!ShouldRecordRelationAccess() || RelationAccessHash == NULL)
{
return false;
}
hash_seq_init(&status, RelationAccessHash);
hashEntry = (RelationAccessHashEntry *) hash_seq_search(&status);
while (hashEntry != NULL)
{
int relationAccessMode = hashEntry->relationAccessMode;
if ((relationAccessMode & PARALLEL_ACCESS_MASK))
{
hash_seq_term(&status);
return true;
}
hashEntry = (RelationAccessHashEntry *) hash_seq_search(&status);
}
return false;
}
/*
* GetRelationSelectAccessMode is a wrapper around GetRelationAccessMode.
*/
@ -524,10 +632,414 @@ GetRelationAccessMode(Oid relationId, ShardPlacementAccessType accessType)
bool
ShouldRecordRelationAccess()
{
if (IsTransactionBlock() || InCoordinatedTransaction())
if (EnforceForeignKeyRestrictions &&
(IsTransactionBlock() || InCoordinatedTransaction()))
{
return true;
}
return false;
}
/*
* CheckConflictingRelationAccesses is mostly a wrapper around
* HoldsConflictingLockWithReferencingRelations(). We're only interested in accesses
* to reference tables that are referenced via a foreign constraint by a
* hash distributed tables.
*/
static void
CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType accessType)
{
DistTableCacheEntry *cacheEntry = NULL;
Oid conflictingReferencingRelationId = InvalidOid;
ShardPlacementAccessType conflictingAccessType = PLACEMENT_ACCESS_SELECT;
if (!EnforceForeignKeyRestrictions || !IsDistributedTable(relationId))
{
return;
}
cacheEntry = DistributedTableCacheEntry(relationId);
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE &&
cacheEntry->referencingRelationsViaForeignKey != NIL))
{
return;
}
if (HoldsConflictingLockWithReferencingRelations(relationId, accessType,
&conflictingReferencingRelationId,
&conflictingAccessType))
{
char *relationName = get_rel_name(relationId);
char *conflictingRelationName = get_rel_name(conflictingReferencingRelationId);
char *accessTypeText = PlacementAccessTypeToText(accessType);
char *conflictingAccessTypeText =
PlacementAccessTypeToText(conflictingAccessType);
ereport(ERROR, (errmsg("cannot execute %s on reference relation \"%s\" because "
"there was a parallel %s access to distributed relation "
"\"%s\" in the same transaction",
accessTypeText, relationName, conflictingAccessTypeText,
conflictingRelationName),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
else if (cacheEntry->referencingRelationsViaForeignKey != NIL &&
accessType > PLACEMENT_ACCESS_SELECT)
{
char *relationName = get_rel_name(relationId);
if (ParallelQueryExecutedInTransaction())
{
/*
* If there has already been a parallel query executed, the sequential mode
* would still use the already opened parallel connections to the workers,
* thus contradicting our purpose of using sequential mode.
*/
ereport(ERROR, (errmsg("cannot modify reference table \"%s\" because there "
"was a parallel operation on a distributed table",
relationName),
errdetail("When there is a foreign key to a reference "
"table, Citus needs to perform all operations "
"over a single connection per node to ensure "
"consistency."),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
else if (MultiShardConnectionType == PARALLEL_CONNECTION)
{
/*
* We can still continue with multi-shard queries in sequential mode, so
* set it.
*/
ereport(DEBUG1, (errmsg("switching to sequential query execution mode"),
errdetail(
"Reference relation \"%s\" is modified, which might lead "
"to data inconsistencies or distributed deadlocks via "
"parallel accesses to hash distributed relations due to "
"foreign keys. Any parallel modification to "
"those hash distributed relations in the same "
"transaction can only be executed in sequential query "
"execution mode", relationName)));
/*
* Switching to sequential mode is admittedly confusing and, could be useless
* and less performant in some cases. However, if we do not switch to
* sequential mode at this point, we'd loose the opportunity to do so
* later when a parallel query is executed on the hash distributed relations
* that are referencing this reference table.
*/
SetLocalMultiShardModifyModeToSequential();
}
}
}
/*
* CheckConflictingParallelRelationAccesses is mostly a wrapper around
* HoldsConflictingLockWithReferencedRelations(). We're only interested in parallel
* accesses to distributed tables that refers reference tables via foreign constraint.
*
*/
static void
CheckConflictingParallelRelationAccesses(Oid relationId, ShardPlacementAccessType
accessType)
{
DistTableCacheEntry *cacheEntry = NULL;
Oid conflictingReferencingRelationId = InvalidOid;
ShardPlacementAccessType conflictingAccessType = PLACEMENT_ACCESS_SELECT;
if (!EnforceForeignKeyRestrictions || !IsDistributedTable(relationId))
{
return;
}
cacheEntry = DistributedTableCacheEntry(relationId);
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_HASH &&
cacheEntry->referencedRelationsViaForeignKey != NIL))
{
return;
}
if (MultiShardConnectionType == PARALLEL_CONNECTION &&
HoldsConflictingLockWithReferencedRelations(relationId, accessType,
&conflictingReferencingRelationId,
&conflictingAccessType))
{
char *relationName = get_rel_name(relationId);
char *conflictingRelationName = get_rel_name(conflictingReferencingRelationId);
char *accessTypeText = PlacementAccessTypeToText(accessType);
char *conflictingAccessTypeText =
PlacementAccessTypeToText(conflictingAccessType);
if (ParallelQueryExecutedInTransaction())
{
/*
* If there has already been a parallel query executed, the sequential mode
* would still use the already opened parallel connections to the workers,
* thus contradicting our purpose of using sequential mode.
*/
ereport(ERROR, (errmsg("cannot execute parallel %s on relation \"%s\" "
"after %s command on reference relation "
"\"%s\" because there is a foreign key between "
"them and \"%s\" has been accessed in this transaction",
accessTypeText, relationName,
conflictingAccessTypeText, conflictingRelationName,
conflictingRelationName),
errdetail("When there is a foreign key to a reference "
"table, Citus needs to perform all operations "
"over a single connection per node to ensure "
"consistency."),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
}
else
{
ereport(DEBUG1, (errmsg("switching to sequential query execution mode"),
errdetail("cannot execute parallel %s on relation \"%s\" "
"after %s command on reference relation "
"\"%s\" because there is a foreign key between "
"them and \"%s\" has been accessed in this transaction",
accessTypeText, relationName,
conflictingAccessTypeText, conflictingRelationName,
conflictingRelationName)));
SetLocalMultiShardModifyModeToSequential();
}
}
}
/*
* CheckConflictingParallelCopyAccesses is mostly a wrapper around
* HoldsConflictingLockWithReferencedRelations(). We're only interested in parallel
* accesses to distributed tables that refers reference tables via foreign constraint.
* Since COPY cannot be used in sequential mode, we're erroring out.
*/
void
CheckConflictingParallelCopyAccesses(Oid relationId)
{
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
Oid conflictingReferencingRelationId = InvalidOid;
ShardPlacementAccessType conflictingAccessType = PLACEMENT_ACCESS_SELECT;
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_HASH &&
cacheEntry->referencedRelationsViaForeignKey != NIL))
{
return;
}
if (HoldsConflictingLockWithReferencedRelations(relationId, PLACEMENT_ACCESS_DML,
&conflictingReferencingRelationId,
&conflictingAccessType))
{
char *relationName = get_rel_name(relationId);
char *conflictingRelationName = get_rel_name(conflictingReferencingRelationId);
char *conflictingAccessTypeText =
PlacementAccessTypeToText(conflictingAccessType);
ereport(ERROR, (errmsg("cannot execute parallel COPY on relation \"%s\" "
"after %s command on reference relation "
"\"%s\" because there is a foreign key between "
"them and \"%s\" has been modified in this transaction",
relationName, conflictingAccessTypeText,
conflictingRelationName, conflictingRelationName),
errdetail("COPY to a distributed table uses a separate set of "
"connections which will not be able to see the "
"uncommitted changes to the reference table."),
errhint("Perform the COPY in a separate transaction.")));
}
}
/*
* HoldsConflictingLockWithReferencedRelations returns true if the input relationId is a
* hash distributed table and it holds any conflicting locks with the reference tables that
* the distributed table has a foreign key to the reference table.
*/
static bool
HoldsConflictingLockWithReferencedRelations(Oid relationId, ShardPlacementAccessType
placementAccess,
Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode)
{
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
ListCell *referencedRelationCell = NULL;
foreach(referencedRelationCell, cacheEntry->referencedRelationsViaForeignKey)
{
Oid referencedRelation = lfirst_oid(referencedRelationCell);
RelationAccessMode selectMode = RELATION_NOT_ACCESSED;
RelationAccessMode dmlMode = RELATION_NOT_ACCESSED;
RelationAccessMode ddlMode = RELATION_NOT_ACCESSED;
/* we're only interested in foreign keys to reference tables */
if (PartitionMethod(referencedRelation) != DISTRIBUTE_BY_NONE)
{
continue;
}
/*
* A select on a reference table could conflict with a DDL
* on a distributed table.
*/
selectMode = GetRelationSelectAccessMode(referencedRelation);
if (placementAccess == PLACEMENT_ACCESS_DDL &&
selectMode != RELATION_NOT_ACCESSED)
{
*conflictingRelationId = referencedRelation;
*conflictingAccessMode = PLACEMENT_ACCESS_SELECT;
return true;
}
/*
* Both DML and DDL operations on a reference table conflicts with
* any parallel operation on distributed tables.
*/
dmlMode = GetRelationDMLAccessMode(referencedRelation);
if (dmlMode != RELATION_NOT_ACCESSED)
{
*conflictingRelationId = referencedRelation;
*conflictingAccessMode = PLACEMENT_ACCESS_DML;
return true;
}
ddlMode = GetRelationDDLAccessMode(referencedRelation);
if (ddlMode != RELATION_NOT_ACCESSED)
{
*conflictingRelationId = referencedRelation;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
return true;
}
}
return false;
}
/*
* HoldsConflictingLockWithReferencingRelations returns true when the input relationId is a
* reference table and it holds any conflicting locks with the distributed tables where
* the distributed table has a foreign key to the reference table.
*
* If returns true, the referencing relation and conflictingAccessMode are also set.
*/
static bool
HoldsConflictingLockWithReferencingRelations(Oid relationId, ShardPlacementAccessType
placementAccess, Oid *conflictingRelationId,
ShardPlacementAccessType *
conflictingAccessMode)
{
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
ListCell *referencingRelationCell = NULL;
bool holdsConflictingLocks = false;
Assert(PartitionMethod(relationId) == DISTRIBUTE_BY_NONE);
foreach(referencingRelationCell, cacheEntry->referencingRelationsViaForeignKey)
{
Oid referencingRelation = lfirst_oid(referencingRelationCell);
/*
* We're only interested in foreign keys to reference tables from
* hash distributed tables.
*/
if (!IsDistributedTable(referencingRelation) ||
PartitionMethod(referencingRelation) != DISTRIBUTE_BY_HASH)
{
continue;
}
/*
* Rules that we apply:
* - SELECT on a reference might table conflict with
* a previous parallel DDL on a distributed table
* - DML on a reference table might conflict with
* a previous parallel DML or DDL on a distributed
* table
* - DDL on a reference table might conflict with
* a parellel SELECT, DML or DDL on a distributed
* table
*/
if (placementAccess == PLACEMENT_ACCESS_SELECT)
{
RelationAccessMode ddlMode = GetRelationDDLAccessMode(referencingRelation);
if (ddlMode == RELATION_PARALLEL_ACCESSED)
{
/* SELECT on a distributed table conflicts with DDL / TRUNCATE */
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
}
}
else if (placementAccess == PLACEMENT_ACCESS_DML)
{
RelationAccessMode ddlMode = RELATION_NOT_ACCESSED;
RelationAccessMode dmlMode = GetRelationDMLAccessMode(referencingRelation);
if (dmlMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DML;
}
ddlMode = GetRelationDDLAccessMode(referencingRelation);
if (ddlMode == RELATION_PARALLEL_ACCESSED)
{
/* SELECT on a distributed table conflicts with DDL / TRUNCATE */
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
}
}
else if (placementAccess == PLACEMENT_ACCESS_DDL)
{
RelationAccessMode selectMode = RELATION_NOT_ACCESSED;
RelationAccessMode ddlMode = RELATION_NOT_ACCESSED;
RelationAccessMode dmlMode = RELATION_NOT_ACCESSED;
selectMode = GetRelationSelectAccessMode(referencingRelation);
if (selectMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_SELECT;
}
dmlMode = GetRelationDMLAccessMode(referencingRelation);
if (dmlMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DML;
}
ddlMode = GetRelationDDLAccessMode(referencingRelation);
if (ddlMode == RELATION_PARALLEL_ACCESSED)
{
holdsConflictingLocks = true;
*conflictingAccessMode = PLACEMENT_ACCESS_DDL;
}
}
if (holdsConflictingLocks)
{
*conflictingRelationId = referencingRelation;
return true;
}
}
return false;
}

View File

@ -0,0 +1,445 @@
/*-------------------------------------------------------------------------
*
* foreign_key_relationship.c
* This file contains functions for creating foreign key relationship graph
* between distributed tables. Created relationship graph will be hold by
* a static variable defined in this file until an invalidation comes in.
*
* Copyright (c) 2018, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/htup_details.h"
#include "access/stratnum.h"
#include "catalog/pg_constraint.h"
#include "distributed/foreign_key_relationship.h"
#include "distributed/hash_helpers.h"
#include "distributed/listutils.h"
#include "distributed/version_compat.h"
#include "nodes/pg_list.h"
#include "storage/lockdefs.h"
#include "utils/fmgroids.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
/*
* ForeignConstraintRelationshipGraph holds the graph data structure for foreign constraint relationship
* between relations. We will only have single static instance of that struct and it
* will be invalidated after change on any foreign constraint.
*/
typedef struct ForeignConstraintRelationshipGraph
{
HTAB *nodeMap;
bool isValid;
}ForeignConstraintRelationshipGraph;
/*
* ForeignConstraintRelationshipNode holds the data for each node of the ForeignConstraintRelationshipGraph
* For each node we have relation id, which is the Oid of that relation, visiting
* information for that node in the latest DFS and the list of adjacency nodes.
* Note that we also hold back adjacency nodes for getting referenced node over
* that one.
*/
typedef struct ForeignConstraintRelationshipNode
{
Oid relationId;
bool visited;
List *adjacencyList;
List *backAdjacencyList;
}ForeignConstraintRelationshipNode;
/*
* ForeignConstraintRelationshipEdge will only be used while creating the ForeignConstraintRelationshipGraph.
* It won't show edge information on the graph, yet will be used in the pre-processing
* phase.
*/
typedef struct ForeignConstraintRelationshipEdge
{
Oid referencingRelationOID;
Oid referencedRelationOID;
}ForeignConstraintRelationshipEdge;
static ForeignConstraintRelationshipGraph *fConstraintRelationshipGraph = NULL;
static void CreateForeignConstraintRelationshipGraph(void);
static void PopulateAdjacencyLists(void);
static int CompareForeignConstraintRelationshipEdges(const void *leftElement, const
void *rightElement);
static void AddForeignConstraintRelationshipEdge(Oid referencingOid, Oid referencedOid);
static ForeignConstraintRelationshipNode * CreateOrFindNode(HTAB *adjacencyLists, Oid
relid);
static void GetConnectedListHelper(ForeignConstraintRelationshipNode *node,
List **adjacentNodeList, bool
isReferencing);
static List * GetForeignConstraintRelationshipHelper(Oid relationId, bool isReferencing);
/*
* ReferencedRelationIdList is a wrapper function around GetForeignConstraintRelationshipHelper
* to get list of relation IDs which are referenced by the given relation id.
*
* Note that, if relation A is referenced by relation B and relation B is referenced
* by relation C, then the result list for relation A consists of the relation
* IDs of relation B and relation C.
*/
List *
ReferencedRelationIdList(Oid relationId)
{
return GetForeignConstraintRelationshipHelper(relationId, false);
}
/*
* ReferencingRelationIdList is a wrapper function around GetForeignConstraintRelationshipHelper
* to get list of relation IDs which are referencing by the given relation id.
*
* Note that, if relation A is referenced by relation B and relation B is referenced
* by relation C, then the result list for relation C consists of the relation
* IDs of relation A and relation B.
*/
List *
ReferencingRelationIdList(Oid relationId)
{
return GetForeignConstraintRelationshipHelper(relationId, true);
}
/*
* GetForeignConstraintRelationshipHelper returns the list of oids referenced or
* referencing given relation id. It is a helper function for providing results
* to public functions ReferencedRelationIdList and ReferencingRelationIdList.
*/
static List *
GetForeignConstraintRelationshipHelper(Oid relationId, bool isReferencing)
{
List *foreignConstraintList = NIL;
List *foreignNodeList = NIL;
ListCell *nodeCell = NULL;
bool isFound = false;
ForeignConstraintRelationshipNode *relationNode = NULL;
CreateForeignConstraintRelationshipGraph();
relationNode = (ForeignConstraintRelationshipNode *) hash_search(
fConstraintRelationshipGraph->nodeMap, &relationId,
HASH_FIND, &isFound);
if (!isFound)
{
/*
* If there is no node with the given relation id, that means given table
* is not referencing and is not referenced by any table
*/
return NIL;
}
GetConnectedListHelper(relationNode, &foreignNodeList, isReferencing);
/*
* We need only their OIDs, we get back node list to make their visited
* variable to false for using them iteratively.
*/
foreach(nodeCell, foreignNodeList)
{
ForeignConstraintRelationshipNode *currentNode =
(ForeignConstraintRelationshipNode *) lfirst(nodeCell);
foreignConstraintList = lappend_oid(foreignConstraintList,
currentNode->relationId);
currentNode->visited = false;
}
/* set to false separately, since we don't add itself to foreign node list */
relationNode->visited = false;
return foreignConstraintList;
}
/*
* CreateForeignConstraintRelationshipGraph creates the foreign constraint relation graph using
* foreign constraint provided by pg_constraint metadata table.
*/
static void
CreateForeignConstraintRelationshipGraph()
{
MemoryContext oldContext;
MemoryContext fConstraintRelationshipMemoryContext = NULL;
HASHCTL info;
uint32 hashFlags = 0;
/* if we have already created the graph, use it */
if (IsForeignConstraintRelationshipGraphValid())
{
return;
}
ClearForeignConstraintRelationshipGraphContext();
fConstraintRelationshipMemoryContext = AllocSetContextCreateExtended(
CacheMemoryContext,
"Forign Constraint Relationship Graph Context",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
oldContext = MemoryContextSwitchTo(fConstraintRelationshipMemoryContext);
fConstraintRelationshipGraph = (ForeignConstraintRelationshipGraph *) palloc(
sizeof(ForeignConstraintRelationshipGraph));
fConstraintRelationshipGraph->isValid = false;
/* create (oid) -> [ForeignConstraintRelationshipNode] hash */
memset(&info, 0, sizeof(info));
info.keysize = sizeof(Oid);
info.entrysize = sizeof(ForeignConstraintRelationshipNode);
info.hash = oid_hash;
info.hcxt = CurrentMemoryContext;
hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
fConstraintRelationshipGraph->nodeMap = hash_create(
"foreign key relationship map (oid)",
32, &info, hashFlags);
PopulateAdjacencyLists();
fConstraintRelationshipGraph->isValid = true;
MemoryContextSwitchTo(oldContext);
}
/*
* IsForeignConstraintGraphValid check whether there is a valid graph.
*/
bool
IsForeignConstraintRelationshipGraphValid()
{
if (fConstraintRelationshipGraph != NULL && fConstraintRelationshipGraph->isValid)
{
return true;
}
return false;
}
/*
* SetForeignConstraintGraphInvalid sets the validity of the graph to false.
*/
void
SetForeignConstraintRelationshipGraphInvalid()
{
if (fConstraintRelationshipGraph != NULL)
{
fConstraintRelationshipGraph->isValid = false;
}
}
/*
* GetConnectedListHelper is the function for getting nodes connected (or connecting) to
* the given relation. adjacentNodeList holds the result for recursive calls and
* by changing isReferencing caller function can select connected or connecting
* adjacency list.
*
*/
static void
GetConnectedListHelper(ForeignConstraintRelationshipNode *node, List **adjacentNodeList,
bool isReferencing)
{
ListCell *nodeCell = NULL;
List *neighbourList = NIL;
node->visited = true;
if (isReferencing)
{
neighbourList = node->backAdjacencyList;
}
else
{
neighbourList = node->adjacencyList;
}
foreach(nodeCell, neighbourList)
{
ForeignConstraintRelationshipNode *neighborNode =
(ForeignConstraintRelationshipNode *) lfirst(nodeCell);
if (neighborNode->visited == false)
{
*adjacentNodeList = lappend(*adjacentNodeList, neighborNode);
GetConnectedListHelper(neighborNode, adjacentNodeList, isReferencing);
}
}
}
/*
* PopulateAdjacencyLists gets foreign constraint relationship information from pg_constraint
* metadata table and populates them to the foreign constraint relation graph.
*/
static void
PopulateAdjacencyLists(void)
{
SysScanDesc scanDescriptor;
HeapTuple tuple;
Relation pgConstraint;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Oid prevReferencingOid = InvalidOid;
Oid prevReferencedOid = InvalidOid;
List *frelEdgeList = NIL;
ListCell *frelEdgeCell = NULL;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_contype, BTEqualStrategyNumber, F_CHAREQ,
CharGetDatum(CONSTRAINT_FOREIGN));
scanDescriptor = systable_beginscan(pgConstraint, InvalidOid, false,
NULL, scanKeyCount, scanKey);
while (HeapTupleIsValid(tuple = systable_getnext(scanDescriptor)))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(tuple);
ForeignConstraintRelationshipEdge *currentFConstraintRelationshipEdge = NULL;
currentFConstraintRelationshipEdge = palloc(
sizeof(ForeignConstraintRelationshipEdge));
currentFConstraintRelationshipEdge->referencingRelationOID =
constraintForm->conrelid;
currentFConstraintRelationshipEdge->referencedRelationOID =
constraintForm->confrelid;
frelEdgeList = lappend(frelEdgeList, currentFConstraintRelationshipEdge);
}
/*
* Since there is no index on columns we are planning to sort tuples
* sorting tuples manually instead of using scan keys
*/
frelEdgeList = SortList(frelEdgeList, CompareForeignConstraintRelationshipEdges);
foreach(frelEdgeCell, frelEdgeList)
{
ForeignConstraintRelationshipEdge *currentFConstraintRelationshipEdge =
(ForeignConstraintRelationshipEdge *) lfirst(frelEdgeCell);
/* we just saw this edge, no need to add it twice */
if (currentFConstraintRelationshipEdge->referencingRelationOID ==
prevReferencingOid &&
currentFConstraintRelationshipEdge->referencedRelationOID ==
prevReferencedOid)
{
continue;
}
AddForeignConstraintRelationshipEdge(
currentFConstraintRelationshipEdge->referencingRelationOID,
currentFConstraintRelationshipEdge->
referencedRelationOID);
prevReferencingOid = currentFConstraintRelationshipEdge->referencingRelationOID;
prevReferencedOid = currentFConstraintRelationshipEdge->referencedRelationOID;
}
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
}
/*
* CompareForeignConstraintRelationshipEdges is a helper function to compare two
* ForeignConstraintRelationshipEdge using referencing and referenced ids respectively.
*/
static int
CompareForeignConstraintRelationshipEdges(const void *leftElement, const
void *rightElement)
{
const ForeignConstraintRelationshipEdge *leftEdge = *((const
ForeignConstraintRelationshipEdge
**) leftElement);
const ForeignConstraintRelationshipEdge *rightEdge = *((const
ForeignConstraintRelationshipEdge
**) rightElement);
int referencingDiff = leftEdge->referencingRelationOID -
rightEdge->referencingRelationOID;
int referencedDiff = leftEdge->referencedRelationOID -
rightEdge->referencedRelationOID;
if (referencingDiff != 0)
{
return referencingDiff;
}
return referencedDiff;
}
/*
* AddForeignConstraintRelationshipEdge adds edge between the nodes having given OIDs
* by adding referenced node to the adjacency list referencing node and adding
* referencing node to the back adjacency list of referenced node.
*/
static void
AddForeignConstraintRelationshipEdge(Oid referencingOid, Oid referencedOid)
{
ForeignConstraintRelationshipNode *referencingNode = CreateOrFindNode(
fConstraintRelationshipGraph->nodeMap, referencingOid);
ForeignConstraintRelationshipNode *referencedNode = CreateOrFindNode(
fConstraintRelationshipGraph->nodeMap, referencedOid);
referencingNode->adjacencyList = lappend(referencingNode->adjacencyList,
referencedNode);
referencedNode->backAdjacencyList = lappend(referencedNode->backAdjacencyList,
referencingNode);
}
/*
* CreateOrFindNode either gets or adds new node to the foreign constraint relation graph
*/
static ForeignConstraintRelationshipNode *
CreateOrFindNode(HTAB *adjacencyLists, Oid relid)
{
bool found = false;
ForeignConstraintRelationshipNode *node =
(ForeignConstraintRelationshipNode *) hash_search(adjacencyLists,
&relid, HASH_ENTER,
&found);
if (!found)
{
node->adjacencyList = NIL;
node->backAdjacencyList = NIL;
node->visited = false;
}
return node;
}
/*
* ClearForeignConstraintRelationshipGraphContext clear all the allocated memory obtained
* for foreign constraint relationship graph. Since all the variables of relationship
* graph was obtained within the same context, destroying hash map is enough as
* it deletes the context.
*/
void
ClearForeignConstraintRelationshipGraphContext()
{
if (fConstraintRelationshipGraph == NULL)
{
return;
}
hash_destroy(fConstraintRelationshipGraph->nodeMap);
fConstraintRelationshipGraph = NULL;
}

View File

@ -29,6 +29,7 @@
#include "distributed/colocation_utils.h"
#include "distributed/connection_management.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/foreign_key_relationship.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_logical_optimizer.h"
@ -44,6 +45,8 @@
#include "distributed/worker_protocol.h"
#include "executor/executor.h"
#include "nodes/makefuncs.h"
#include "nodes/memnodes.h"
#include "nodes/pg_list.h"
#include "parser/parse_func.h"
#include "parser/parse_type.h"
#include "storage/lmgr.h"
@ -55,6 +58,7 @@
#include "utils/inval.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/palloc.h"
#include "utils/rel.h"
#include "utils/relfilenodemap.h"
#include "utils/relmapper.h"
@ -182,11 +186,13 @@ static bool HasOverlappingShardInterval(ShardInterval **shardIntervalArray,
static void InitializeCaches(void);
static void InitializeDistTableCache(void);
static void InitializeWorkerNodeCache(void);
static void RegisterForeignKeyGraphCacheCallbacks(void);
static void RegisterWorkerNodeCacheCallbacks(void);
static void RegisterLocalGroupIdCacheCallbacks(void);
static uint32 WorkerNodeHashCode(const void *key, Size keySize);
static void ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry);
static void CreateDistTableCache(void);
static void InvalidateForeignRelationGraphCacheCallback(Datum argument, Oid relationId);
static void InvalidateDistRelationCacheCallback(Datum argument, Oid relationId);
static void InvalidateNodeRelationCacheCallback(Datum argument, Oid relationId);
static void InvalidateLocalGroupIdRelationCacheCallback(Datum argument, Oid relationId);
@ -204,6 +210,7 @@ static ShardPlacement * ResolveGroupShardPlacement(
GroupShardPlacement *groupShardPlacement, ShardCacheEntry *shardEntry);
static WorkerNode * LookupNodeForGroup(uint32 groupid);
static Oid LookupEnumValueId(Oid typeId, char *valueName);
static void InvalidateEntireDistCache(void);
/* exports for SQL callable functions */
@ -981,6 +988,15 @@ BuildDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
cacheEntry->hashFunction = NULL;
}
oldContext = MemoryContextSwitchTo(CacheMemoryContext);
cacheEntry->referencedRelationsViaForeignKey = ReferencedRelationIdList(
cacheEntry->relationId);
cacheEntry->referencingRelationsViaForeignKey = ReferencingRelationIdList(
cacheEntry->relationId);
MemoryContextSwitchTo(oldContext);
heap_close(pgDistPartition, NoLock);
}
@ -2510,6 +2526,7 @@ InitializeCaches(void)
}
InitializeDistTableCache();
RegisterForeignKeyGraphCacheCallbacks();
RegisterWorkerNodeCacheCallbacks();
RegisterLocalGroupIdCacheCallbacks();
}
@ -2701,6 +2718,19 @@ InitializeWorkerNodeCache(void)
}
/*
* RegisterForeignKeyGraphCacheCallbacks registers callbacks required for
* the foreign key graph cache.
*/
static void
RegisterForeignKeyGraphCacheCallbacks(void)
{
/* Watch for invalidation events. */
CacheRegisterRelcacheCallback(InvalidateForeignRelationGraphCacheCallback,
(Datum) 0);
}
/*
* RegisterWorkerNodeCacheCallbacks registers the callbacks required for the
* worker node cache. It's separate from InitializeWorkerNodeCache so the
@ -2905,6 +2935,16 @@ ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
pfree(cacheEntry->arrayOfPlacementArrays);
cacheEntry->arrayOfPlacementArrays = NULL;
}
if (cacheEntry->referencedRelationsViaForeignKey)
{
list_free(cacheEntry->referencedRelationsViaForeignKey);
cacheEntry->referencedRelationsViaForeignKey = NIL;
}
if (cacheEntry->referencingRelationsViaForeignKey)
{
list_free(cacheEntry->referencingRelationsViaForeignKey);
cacheEntry->referencingRelationsViaForeignKey = NIL;
}
cacheEntry->shardIntervalArrayLength = 0;
cacheEntry->hasUninitializedShardInterval = false;
@ -2913,6 +2953,46 @@ ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
}
/*
* InvalidateForeignRelationGraphCacheCallback invalidates the foreign key relation
* graph and entire distributed cache entries.
*/
static void
InvalidateForeignRelationGraphCacheCallback(Datum argument, Oid relationId)
{
/* when invalidation happens simply set the LocalGroupId to the default value */
if (relationId == MetadataCache.distColocationRelationId)
{
SetForeignConstraintRelationshipGraphInvalid();
InvalidateEntireDistCache();
}
}
/*
* InvalidateForeignKeyGraph is used to invalidate the cached foreign key
* graph (see ForeignKeyRelationGraph @ utils/foreign_key_relationship.c).
*
* To invalidate the foreign key graph, we hack around relcache invalidation
* callbacks. Given that there is no metadata table associated with the foreign
* key graph cache, we use pg_dist_colocation, which is never invalidated for
* other purposes.
*
* We acknowledge that it is not a very intiutive way of implementing this cache
* invalidation, but, seems acceptable for now. If this becomes problematic, we
* could try using a magic oid where we're sure that no relation would ever use
* that oid.
*/
void
InvalidateForeignKeyGraph(void)
{
CitusInvalidateRelcacheByRelid(DistColocationRelationId());
/* bump command counter to force invalidation to take effect */
CommandCounterIncrement();
}
/*
* InvalidateDistRelationCacheCallback flushes cache entries when a relation
* is updated (or flushes the entire cache).
@ -2923,21 +3003,14 @@ InvalidateDistRelationCacheCallback(Datum argument, Oid relationId)
/* invalidate either entire cache or a specific entry */
if (relationId == InvalidOid)
{
DistTableCacheEntry *cacheEntry = NULL;
HASH_SEQ_STATUS status;
hash_seq_init(&status, DistTableCacheHash);
while ((cacheEntry = (DistTableCacheEntry *) hash_seq_search(&status)) != NULL)
{
cacheEntry->isValid = false;
}
InvalidateEntireDistCache();
}
else
{
void *hashKey = (void *) &relationId;
bool foundInCache = false;
DistTableCacheEntry *cacheEntry = hash_search(DistTableCacheHash, hashKey,
HASH_FIND, &foundInCache);
if (foundInCache)
@ -2958,6 +3031,24 @@ InvalidateDistRelationCacheCallback(Datum argument, Oid relationId)
}
/*
* InvalidateEntireDistCache makes entire cache entries invalid.
*/
static void
InvalidateEntireDistCache()
{
DistTableCacheEntry *cacheEntry = NULL;
HASH_SEQ_STATUS status;
hash_seq_init(&status, DistTableCacheHash);
while ((cacheEntry = (DistTableCacheEntry *) hash_seq_search(&status)) != NULL)
{
cacheEntry->isValid = false;
}
}
/*
* FlushDistTableCache flushes the entire distributed relation cache, frees
* all entries, and recreates the cache.

View File

@ -25,6 +25,7 @@
#include "distributed/distributed_planner.h"
#include "distributed/multi_router_executor.h"
#include "distributed/relay_utility.h"
#include "distributed/reference_table_utils.h"
#include "distributed/resource_lock.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/worker_protocol.h"
@ -33,6 +34,7 @@
/* local function forward declarations */
static LOCKMODE IntToLockMode(int mode);
static List * GetSortedReferenceShardIntervals(List *relationList);
/* exports for SQL callable functions */
@ -164,6 +166,60 @@ LockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode)
}
/*
* LockReferencedReferenceShardDistributionMetadata acquires the given lock
* on the reference tables which has a foreign key from the given relation.
*/
void
LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lock)
{
ListCell *shardIntervalCell = NULL;
Oid relationId = RelationIdForShard(shardId);
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
List *referencedRelationList = cacheEntry->referencedRelationsViaForeignKey;
List *shardIntervalList = GetSortedReferenceShardIntervals(referencedRelationList);
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
LockShardDistributionMetadata(shardInterval->shardId, lock);
}
}
/*
* GetSortedReferenceShards iterates through the given relation list.
* Lists the shards of reference tables and returns the list after sorting.
*/
static List *
GetSortedReferenceShardIntervals(List *relationList)
{
List *shardIntervalList = NIL;
ListCell *relationCell = NULL;
foreach(relationCell, relationList)
{
Oid relationId = lfirst_oid(relationCell);
List *currentShardIntervalList = NIL;
if (PartitionMethod(relationId) != DISTRIBUTE_BY_NONE)
{
continue;
}
currentShardIntervalList = LoadShardIntervalList(relationId);
shardIntervalList = lappend(shardIntervalList, linitial(
currentShardIntervalList));
}
shardIntervalList = SortList(shardIntervalList, CompareShardIntervalsById);
return shardIntervalList;
}
/*
* TryLockShardDistributionMetadata tries to grab a lock for distribution
* metadata related to the specified shard, returning false if the lock

View File

@ -12,13 +12,21 @@
#include "postgres.h"
#include "postgres_ext.h"
#include "utils/relcache.h"
#include "utils/hsearch.h"
#include "nodes/primnodes.h"
extern bool ConstraintIsAForeignKeyToReferenceTable(char *constraintName,
Oid leftRelationId);
extern void ErrorIfUnsupportedForeignConstraint(Relation relation, char
distributionMethod,
Var *distributionColumn, uint32
colocationId);
extern bool ColumnAppearsInForeignKeyToReferenceTable(char *columnName, Oid
relationId);
extern List * GetTableForeignConstraintCommands(Oid relationId);
extern bool HasForeignKeyToReferenceTable(Oid relationId);
extern bool TableReferenced(Oid relationId);
extern bool TableReferencing(Oid relationId);
extern bool ConstraintIsAForeignKey(char *constraintName, Oid relationId);
#endif

View File

@ -0,0 +1,24 @@
/*-------------------------------------------------------------------------
* foreign_key_relationship.h
*
* Copyright (c) 2018, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef FOREIGN_KEY_RELATIONSHIP_H
#define FOREIGN_KEY_RELATIONSHIP_H
#include "postgres.h"
#include "postgres_ext.h"
#include "utils/relcache.h"
#include "utils/hsearch.h"
#include "nodes/primnodes.h"
extern List * ReferencedRelationIdList(Oid relationId);
extern List * ReferencingRelationIdList(Oid relationId);
extern void SetForeignConstraintRelationshipGraphInvalid(void);
extern bool IsForeignConstraintRelationshipGraphValid(void);
extern void ClearForeignConstraintRelationshipGraphContext(void);
#endif

View File

@ -164,6 +164,11 @@ extern Datum master_copy_shard_placement(PG_FUNCTION_ARGS);
extern List * CopyShardCommandList(ShardInterval *shardInterval, char *sourceNodeName,
int32 sourceNodePort);
extern List * CopyShardForeignConstraintCommandList(ShardInterval *shardInterval);
extern void CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
List **
colocatedShardForeignConstraintCommandList,
List **
referenceTableForeignConstraintList);
extern ShardPlacement * SearchShardPlacementInList(List *shardPlacementList,
char *nodeName, uint32 nodePort,
bool missingOk);

View File

@ -69,6 +69,19 @@ typedef struct
FmgrInfo *shardIntervalCompareFunction;
FmgrInfo *hashFunction; /* NULL if table is not distributed by hash */
/*
* The following two lists consists of relationIds that this distributed
* relation has a foreign key to (e.g., referencedRelationsViaForeignKey) or
* other relations has a foreign key to to this relation (e.g.,
* referencingRelationsViaForeignKey).
*
* Note that we're keeping all transitive foreign key references as well
* such that if relation A refers to B, and B refers to C, we keep A and B
* in C's referencingRelationsViaForeignKey.
*/
List *referencedRelationsViaForeignKey;
List *referencingRelationsViaForeignKey;
/* pg_dist_placement metadata */
GroupShardPlacement **arrayOfPlacementArrays;
int *arrayOfPlacementArrayLengths;
@ -89,6 +102,7 @@ extern List * DistTableOidList(void);
extern List * ShardPlacementList(uint64 shardId);
extern void CitusInvalidateRelcacheByRelid(Oid relationId);
extern void CitusInvalidateRelcacheByShardId(int64 shardId);
extern void InvalidateForeignKeyGraph(void);
extern void FlushDistTableCache(void);
extern void InvalidateMetadataSystemCache(void);
extern Datum DistNodeMetadata(void);

View File

@ -25,6 +25,7 @@ typedef struct DDLJob
{
Oid targetRelationId; /* oid of the target distributed relation */
bool concurrentIndexCmd; /* related to a CONCURRENTLY index command? */
bool executeSequentially;
const char *commandString; /* initial (coordinator) DDL command string */
List *taskList; /* worker DDL tasks to execute */
} DDLJob;

View File

@ -13,6 +13,10 @@
#include "distributed/multi_physical_planner.h" /* access Task struct */
#include "distributed/placement_connection.h"
/* Config variables managed via guc.c */
extern bool EnforceForeignKeyRestrictions;
/* forward declare, to avoid dependency on ShardPlacement definition */
struct ShardPlacement;
@ -37,6 +41,8 @@ extern RelationAccessMode GetRelationDDLAccessMode(Oid relationId);
extern RelationAccessMode GetRelationDMLAccessMode(Oid relationId);
extern RelationAccessMode GetRelationSelectAccessMode(Oid relationId);
extern bool ShouldRecordRelationAccess(void);
extern void CheckConflictingParallelCopyAccesses(Oid relationId);
extern bool ParallelQueryExecutedInTransaction(void);
#endif /* RELATION_ACCESS_TRACKING_H_ */

View File

@ -67,6 +67,10 @@ typedef enum AdvisoryLocktagClass
extern void LockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode);
extern bool TryLockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode);
/* Lock shard/relation metadata of the referenced reference table if exists */
extern void LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE
lock);
/* Lock shard data, for DML commands or remote fetches */
extern void LockShardResource(uint64 shardId, LOCKMODE lockmode);
extern void UnlockShardResource(uint64 shardId, LOCKMODE lockmode);

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -16,47 +16,66 @@ SELECT create_distributed_table('referenced_table', 'id', 'hash');
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET NULL);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
ERROR: cannot create foreign key constraint
DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation.
DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation when distribution key is included in the foreign key constraint
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET DEFAULT);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
ERROR: cannot create foreign key constraint
DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation.
DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation when distribution key is included in the foreign key constraint
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET NULL);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
ERROR: cannot create foreign key constraint
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation.
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation when distribution key included in the foreign constraint.
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET DEFAULT);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
ERROR: cannot create foreign key constraint
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation.
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation when distribution key included in the foreign constraint.
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE CASCADE);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
ERROR: cannot create foreign key constraint
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation.
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation when distribution key included in the foreign constraint.
DROP TABLE referencing_table;
-- self referencing table with replication factor > 1
CREATE TABLE self_referencing_table(id int, ref_id int, PRIMARY KEY (id, ref_id), FOREIGN KEY(id,ref_id) REFERENCES self_referencing_table(id, ref_id));
SELECT create_distributed_table('self_referencing_table', 'id', 'hash');
ERROR: cannot create foreign key constraint
DETAIL: Citus Community Edition currently supports foreign key constraints only for "citus.shard_replication_factor = 1".
HINT: 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.
DROP TABLE self_referencing_table;
CREATE TABLE self_referencing_table(id int, ref_id int, PRIMARY KEY (id, ref_id));
SELECT create_distributed_table('self_referencing_table', 'id', 'hash');
create_distributed_table
--------------------------
(1 row)
ALTER TABLE self_referencing_table ADD CONSTRAINT fkey FOREIGN KEY(id,ref_id) REFERENCES self_referencing_table(id, ref_id);
ERROR: cannot create foreign key constraint
DETAIL: Citus Community Edition currently supports foreign key constraints only for "citus.shard_replication_factor = 1".
HINT: 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.
DROP TABLE self_referencing_table;
-- test foreign constraint creation on NOT co-located tables
SET citus.shard_count TO 8;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
ERROR: cannot create foreign key constraint
DETAIL: Foreign key constraint can only be created on co-located tables.
ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table
DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table
DROP TABLE referencing_table;
SET citus.shard_count TO 32;
-- test foreign constraint creation on non-partition columns
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
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.
DETAIL: Foreign keys are supported in two cases, either in between two colocated tables including partition column in the same ordinal in the both tables or from distributed to reference tables
DROP TABLE referencing_table;
-- test foreign constraint creation while column list are in incorrect order
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(id, ref_id) REFERENCES referenced_table(id, test_column));
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
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.
DETAIL: Foreign keys are supported in two cases, either in between two colocated tables including partition column in the same ordinal in the both tables or from distributed to reference tables
DROP TABLE referencing_table;
-- test foreign constraint with replication factor > 1
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id));
@ -66,9 +85,38 @@ DETAIL: Citus Community Edition currently supports foreign key constraints only
HINT: 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.
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- test foreign constraint with correct conditions
-- test foreign constraint creation on append and range distributed tables
-- foreign keys are supported either in between distributed tables including the
-- distribution column or from distributed tables to reference tables.
SET citus.shard_replication_factor TO 1;
CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column));
SELECT create_distributed_table('referenced_table', 'id', 'hash');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'id', 'append');
ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table
DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table
DROP TABLE referencing_table;
DROP TABLE referenced_table;
CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column));
SELECT create_distributed_table('referenced_table', 'id', 'range');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE referencing_table(id int, ref_id int,FOREIGN KEY (id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'id', 'range');
ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table
DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- test foreign constraint with correct conditions
CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column));
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referenced_table', 'id', 'hash');
create_distributed_table
@ -85,8 +133,8 @@ SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
-- 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_1350065" violates foreign key constraint "referencing_table_ref_id_fkey_1350065"
DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350033".
ERROR: insert or update on table "referencing_table_1350129" violates foreign key constraint "referencing_table_ref_id_fkey_1350129"
DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350097".
CONTEXT: while executing command on localhost:57638
-- test insert to referencing while there is corresponding value in referenced table
INSERT INTO referenced_table VALUES(1, 1);
@ -94,8 +142,8 @@ 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_1350033" violates foreign key constraint "referencing_table_ref_id_fkey_1350065" on table "referencing_table_1350065"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350065".
ERROR: update or delete on table "referenced_table_1350097" violates foreign key constraint "referencing_table_ref_id_fkey_1350129" on table "referencing_table_1350129"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350129".
CONTEXT: while executing command on localhost:57638
-- test delete from referenced table while there is NO corresponding value in referencing table
DELETE FROM referencing_table WHERE ref_id = 1;
@ -189,8 +237,8 @@ SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
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_1350161" violates foreign key constraint "referencing_table_ref_id_fkey_1350193" on table "referencing_table_1350193"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350193".
ERROR: update or delete on table "referenced_table_1350225" violates foreign key constraint "referencing_table_ref_id_fkey_1350257" on table "referencing_table_1350257"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350257".
CONTEXT: while executing command on localhost:57638
BEGIN;
DELETE FROM referenced_table WHERE id = 1;
@ -227,8 +275,8 @@ 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_1350225" violates foreign key constraint "referencing_table_ref_id_fkey_1350257" on table "referencing_table_1350257"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350257".
ERROR: update or delete on table "referenced_table_1350289" violates foreign key constraint "referencing_table_ref_id_fkey_1350321" on table "referencing_table_1350321"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350321".
CONTEXT: while executing command on localhost:57638
DELETE FROM referencing_table WHERE ref_id = 1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
@ -265,8 +313,8 @@ SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
INSERT INTO referenced_table VALUES(1, 1);
INSERT INTO referencing_table VALUES(1, 1);
UPDATE referenced_table SET test_column = 10 WHERE id = 1;
ERROR: update or delete on table "referenced_table_1350289" violates foreign key constraint "referencing_table_ref_id_fkey_1350321" on table "referencing_table_1350321"
DETAIL: Key (id, test_column)=(1, 1) is still referenced from table "referencing_table_1350321".
ERROR: update or delete on table "referenced_table_1350353" violates foreign key constraint "referencing_table_ref_id_fkey_1350385" on table "referencing_table_1350385"
DETAIL: Key (id, test_column)=(1, 1) is still referenced from table "referencing_table_1350385".
CONTEXT: while executing command on localhost:57638
BEGIN;
UPDATE referenced_table SET test_column = 10 WHERE id = 1;
@ -305,8 +353,8 @@ INSERT INTO referenced_table VALUES(1, 1);
INSERT INTO referencing_table VALUES(1, 1);
BEGIN;
UPDATE referenced_table SET test_column = 20 WHERE id = 1;
ERROR: update or delete on table "referenced_table_1350353" violates foreign key constraint "referencing_table_ref_id_fkey_1350385" on table "referencing_table_1350385"
DETAIL: Key (id, test_column)=(1, 1) is still referenced from table "referencing_table_1350385".
ERROR: update or delete on table "referenced_table_1350417" violates foreign key constraint "referencing_table_ref_id_fkey_1350449" on table "referencing_table_1350449"
DETAIL: Key (id, test_column)=(1, 1) is still referenced from table "referencing_table_1350449".
CONTEXT: while executing command on localhost:57638
UPDATE referencing_table SET id = 20 WHERE ref_id = 1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
@ -366,7 +414,7 @@ SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
(1 row)
INSERT INTO referencing_table VALUES(null, 2);
ERROR: insert or update on table "referencing_table_1350536" violates foreign key constraint "referencing_table_ref_id_fkey_1350536"
ERROR: insert or update on table "referencing_table_1350600" violates foreign key constraint "referencing_table_ref_id_fkey_1350600"
DETAIL: MATCH FULL does not allow mixing of null and nonnull key values.
CONTEXT: while executing command on localhost:57637
SELECT * FROM referencing_table;
@ -378,29 +426,18 @@ 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
SET citus.shard_count TO 4;
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
-----------------------------
SELECT create_distributed_table('referenced_table', 'id', 'hash');
create_distributed_table
--------------------------
(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
-----------------------------
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
create_distributed_table
--------------------------
(1 row)
@ -415,19 +452,19 @@ ABORT;
-- 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.
DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation when distribution key is included in the foreign key constraint
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.
DETAIL: SET NULL or SET DEFAULT is not supported in ON DELETE operation when distribution key is included in the foreign key constraint
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.
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation when distribution key included in the foreign constraint.
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.
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation when distribution key included in the foreign constraint.
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.
DETAIL: SET NULL, SET DEFAULT or CASCADE is not supported in ON UPDATE operation when distribution key included in the foreign constraint.
-- 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);
@ -437,9 +474,25 @@ HINT: You can issue each subcommand separately
ALTER TABLE referencing_table ADD FOREIGN KEY(ref_id) REFERENCES referenced_table(id);
ERROR: cannot create constraint without a name on a distributed table
-- test foreign constraint creation on NOT 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));
SELECT create_distributed_table('referenced_table', 'id', 'hash');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash', colocate_with => 'none');
create_distributed_table
--------------------------
(1 row)
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.
ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table
DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table
-- create co-located tables
DROP TABLE referencing_table;
DROP TABLE referenced_table;
@ -463,38 +516,38 @@ ERROR: number of referencing and referenced columns for foreign key disagree
-- test foreign constraint creation on non-partition columns
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(id) REFERENCES referenced_table(id);
ERROR: cannot create foreign key constraint
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.
DETAIL: Foreign keys are supported in two cases, either in between two colocated tables including partition column in the same ordinal in the both tables or from distributed to reference tables
-- 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.
DETAIL: Foreign keys are supported in two cases, either in between two colocated tables including partition column in the same ordinal in the both tables or from distributed to reference tables
-- 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: number of referencing and referenced columns for foreign key disagree
-- test foreign constraint creation while existing tables does not satisfy the constraint
INSERT INTO referencing_table VALUES(1, 1);
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id);
ERROR: insert or update on table "referencing_table_1350585" violates foreign key constraint "test_constraint_1350585"
DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350553".
CONTEXT: while executing command on localhost:57638
ERROR: insert or update on table "referencing_table_1350628" violates foreign key constraint "test_constraint_1350628"
DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350624".
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_1350585" violates foreign key constraint "test_constraint_1350585"
DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350553".
CONTEXT: while executing command on localhost:57638
ERROR: insert or update on table "referencing_table_1350628" violates foreign key constraint "test_constraint_1350628"
DETAIL: Key (ref_id)=(1) is not present in table "referenced_table_1350624".
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_1350553" violates foreign key constraint "test_constraint_1350585" on table "referencing_table_1350585"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350585".
CONTEXT: while executing command on localhost:57638
ERROR: update or delete on table "referenced_table_1350624" violates foreign key constraint "test_constraint_1350628" on table "referencing_table_1350628"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350628".
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;
@ -522,9 +575,9 @@ ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id)
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_1350553" violates foreign key constraint "test_constraint_1350585" on table "referencing_table_1350585"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350585".
CONTEXT: while executing command on localhost:57638
ERROR: update or delete on table "referenced_table_1350624" violates foreign key constraint "test_constraint_1350628" on table "referencing_table_1350628"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350628".
CONTEXT: while executing command on localhost:57637
BEGIN;
DELETE FROM referenced_table WHERE id = 1;
DELETE FROM referencing_table WHERE ref_id = 1;
@ -546,9 +599,9 @@ 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_1350553" violates foreign key constraint "test_constraint_1350585" on table "referencing_table_1350585"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350585".
CONTEXT: while executing command on localhost:57638
ERROR: update or delete on table "referenced_table_1350624" violates foreign key constraint "test_constraint_1350628" on table "referencing_table_1350628"
DETAIL: Key (id)=(1) is still referenced from table "referencing_table_1350628".
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;
@ -568,9 +621,9 @@ 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_1350553" violates foreign key constraint "test_constraint_1350585" on table "referencing_table_1350585"
DETAIL: Key (id, test_column)=(1, 1) is still referenced from table "referencing_table_1350585".
CONTEXT: while executing command on localhost:57638
ERROR: update or delete on table "referenced_table_1350624" violates foreign key constraint "test_constraint_1350628" on table "referencing_table_1350628"
DETAIL: Key (id, test_column)=(1, 1) is still referenced from table "referencing_table_1350628".
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;
@ -592,9 +645,9 @@ ALTER TABLE referencing_table DROP CONSTRAINT test_constraint;
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_1350553" violates foreign key constraint "test_constraint_1350585" on table "referencing_table_1350585"
DETAIL: Key (id, test_column)=(1, 10) is still referenced from table "referencing_table_1350585".
CONTEXT: while executing command on localhost:57638
ERROR: update or delete on table "referenced_table_1350624" violates foreign key constraint "test_constraint_1350628" on table "referencing_table_1350628"
DETAIL: Key (id, test_column)=(1, 10) is still referenced from table "referencing_table_1350628".
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;
@ -626,9 +679,9 @@ 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_1350608" violates foreign key constraint "test_constraint_1350608"
ERROR: insert or update on table "referencing_table_1350631" violates foreign key constraint "test_constraint_1350631"
DETAIL: MATCH FULL does not allow mixing of null and nonnull key values.
CONTEXT: while executing command on localhost:57637
CONTEXT: while executing command on localhost:57638
SELECT * FROM referencing_table;
id | ref_id
----+--------
@ -658,9 +711,9 @@ ALTER TABLE cyclic_reference_table1 ADD CONSTRAINT cyclic_constraint1 FOREIGN KE
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_1350617" violates foreign key constraint "cyclic_constraint1_1350617"
DETAIL: Key (id, table2_id)=(1, 1) is not present in table "cyclic_reference_table2_1350649".
CONTEXT: while executing command on localhost:57638
ERROR: insert or update on table "cyclic_reference_table1_1350632" violates foreign key constraint "cyclic_constraint1_1350632"
DETAIL: Key (id, table2_id)=(1, 1) is not present in table "cyclic_reference_table2_1350636".
CONTEXT: while executing command on localhost:57637
-- proper insertion to table with cyclic dependency
BEGIN;
INSERT INTO cyclic_reference_table1 VALUES(1, 1);
@ -740,9 +793,9 @@ SELECT create_distributed_table('self_referencing_table1', 'id', 'hash');
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_1350681" violates foreign key constraint "self_referencing_table1_id_fkey_1350681"
DETAIL: Key (id, other_column_ref)=(1, 3) is not present in table "self_referencing_table1_1350681".
CONTEXT: while executing command on localhost:57638
ERROR: insert or update on table "self_referencing_table1_1350640" violates foreign key constraint "self_referencing_table1_id_fkey_1350640"
DETAIL: Key (id, other_column_ref)=(1, 3) is not present in table "self_referencing_table1_1350640".
CONTEXT: while executing command on localhost:57637
-- verify that rows are actually inserted
SELECT * FROM self_referencing_table1;
id | other_column | other_column_ref
@ -765,9 +818,9 @@ ALTER TABLE self_referencing_table2 ADD CONSTRAINT self_referencing_fk_constrain
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_1350713" violates foreign key constraint "self_referencing_fk_constraint_1350713"
DETAIL: Key (id, other_column_ref)=(1, 3) is not present in table "self_referencing_table2_1350713".
CONTEXT: while executing command on localhost:57638
ERROR: insert or update on table "self_referencing_table2_1350644" violates foreign key constraint "self_referencing_fk_constraint_1350644"
DETAIL: Key (id, other_column_ref)=(1, 3) is not present in table "self_referencing_table2_1350644".
CONTEXT: while executing command on localhost:57637
-- verify that rows are actually inserted
SELECT * FROM self_referencing_table2;
id | other_column | other_column_ref
@ -788,8 +841,9 @@ SELECT create_distributed_table('referenced_by_reference_table', 'id');
CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_by_reference_table(id));
SELECT create_reference_table('reference_table');
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on CREATE TABLE to reference table
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- test foreign key creation on CREATE TABLE from + to reference table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
SELECT create_reference_table('reference_table');
@ -798,22 +852,18 @@ SELECT create_reference_table('reference_table');
(1 row)
CREATE TABLE references_to_reference_table(id int, referencing_column int REFERENCES reference_table(id));
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on CREATE TABLE from + to reference table
CREATE TABLE reference_table_second(id int, referencing_column int REFERENCES reference_table(id));
SELECT create_reference_table('reference_table_second');
ERROR: cannot create foreign key constraint from or to reference tables
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- test foreign key creation on CREATE TABLE from reference table to local table
CREATE TABLE referenced_local_table(id int PRIMARY KEY, other_column int);
DROP TABLE reference_table CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to constraint references_to_reference_table_referencing_column_fkey on table references_to_reference_table
drop cascades to constraint reference_table_second_referencing_column_fkey on table reference_table_second
NOTICE: drop cascades to constraint reference_table_second_referencing_column_fkey on table reference_table_second
CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_local_table(id));
SELECT create_reference_table('reference_table');
ERROR: cannot create foreign key constraint from or to reference tables
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- test foreign key creation on CREATE TABLE on self referencing reference table
CREATE TABLE self_referencing_reference_table(
id int,
@ -823,7 +873,8 @@ CREATE TABLE self_referencing_reference_table(
FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_reference_table(id, other_column)
);
SELECT create_reference_table('self_referencing_reference_table');
ERROR: cannot create foreign key constraint from or to reference tables
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- test foreign key creation on ALTER TABLE from reference table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
@ -834,9 +885,9 @@ SELECT create_reference_table('reference_table');
(1 row)
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_by_reference_table(id);
ERROR: cannot create foreign key constraint from or to reference tables
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- test foreign key creation on ALTER TABLE to reference table
DROP TABLE references_to_reference_table;
CREATE TABLE references_to_reference_table(id int, referencing_column int);
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');
create_distributed_table
@ -845,7 +896,6 @@ SELECT create_distributed_table('references_to_reference_table', 'referencing_co
(1 row)
ALTER TABLE references_to_reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES reference_table(id);
ERROR: cannot create foreign key constraint from or to reference tables
-- test foreign key creation on ALTER TABLE from + to reference table
DROP TABLE reference_table_second;
CREATE TABLE reference_table_second(id int, referencing_column int);
@ -856,9 +906,11 @@ SELECT create_reference_table('reference_table_second');
(1 row)
ALTER TABLE reference_table_second ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES reference_table(id);
ERROR: cannot create foreign key constraint from or to reference tables
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- test foreign key creation on ALTER TABLE from reference table to local table
DROP TABLE reference_table;
DROP TABLE reference_table CASCADE;
NOTICE: drop cascades to constraint fk on table references_to_reference_table
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
SELECT create_reference_table('reference_table');
create_reference_table
@ -867,7 +919,8 @@ SELECT create_reference_table('reference_table');
(1 row)
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_local_table(id);
ERROR: relation referenced_local_table is not distributed
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- test foreign key creation on ALTER TABLE on self referencing reference table
DROP TABLE self_referencing_reference_table;
CREATE TABLE self_referencing_reference_table(
@ -883,6 +936,7 @@ SELECT create_reference_table('self_referencing_reference_table');
(1 row)
ALTER TABLE self_referencing_reference_table ADD CONSTRAINT fk FOREIGN KEY(id, other_column_ref) REFERENCES self_referencing_reference_table(id, other_column);
ERROR: cannot create foreign key constraint from or to reference tables
ERROR: cannot create foreign key constraint because reference tables are not supported as the referencing table of a foreign constraint
DETAIL: Reference tables are only supported as the referenced table of a foreign key when the referencing table is a hash distributed table
-- we no longer need those tables
DROP TABLE referenced_by_reference_table, references_to_reference_table, reference_table, reference_table_second, referenced_local_table, self_referencing_reference_table;

View File

@ -0,0 +1,620 @@
SET citus.next_shard_id TO 3000000;
SET citus.shard_replication_factor TO 1;
CREATE SCHEMA fkey_graph;
SET search_path TO 'fkey_graph';
CREATE FUNCTION get_referencing_relation_id_list(Oid)
RETURNS SETOF Oid
LANGUAGE C STABLE STRICT
AS 'citus', $$get_referencing_relation_id_list$$;
CREATE FUNCTION get_referenced_relation_id_list(Oid)
RETURNS SETOF Oid
LANGUAGE C STABLE STRICT
AS 'citus', $$get_referenced_relation_id_list$$;
-- Simple case with distributed tables
CREATE TABLE dtt1(id int PRIMARY KEY);
SELECT create_distributed_table('dtt1','id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE dtt2(id int PRIMARY KEY REFERENCES dtt1(id));
SELECT create_distributed_table('dtt2','id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE dtt3(id int PRIMARY KEY REFERENCES dtt2(id));
SELECT create_distributed_table('dtt3','id');
create_distributed_table
--------------------------
(1 row)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt1'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt2'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
dtt1
(1 row)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt3'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
dtt1
dtt2
(2 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt1'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
dtt2
dtt3
(2 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt2'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
dtt3
(1 row)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt3'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
CREATE TABLE dtt4(id int PRIMARY KEY);
SELECT create_distributed_table('dtt4', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt4'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt4'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
ALTER TABLE dtt4 ADD CONSTRAINT dtt4_fkey FOREIGN KEY (id) REFERENCES dtt3(id);
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt4'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
dtt1
dtt2
dtt3
(3 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt4'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt1'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt2'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
dtt1
(1 row)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt3'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
dtt1
dtt2
(2 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt1'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
dtt2
dtt3
dtt4
(3 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt2'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
dtt3
dtt4
(2 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt3'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
dtt4
(1 row)
ALTER TABLE dtt4 DROP CONSTRAINT dtt4_fkey;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt3'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
dtt1
dtt2
(2 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt3'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt4'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt4'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
-- some tests within transction blocks to make sure that
-- cache invalidation works fine
CREATE TABLE test_1 (id int UNIQUE);
CREATE TABLE test_2 (id int UNIQUE);
CREATE TABLE test_3 (id int UNIQUE);
CREATE TABLE test_4 (id int UNIQUE);
CREATE TABLE test_5 (id int UNIQUE);
SELECT create_distributed_Table('test_1', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_Table('test_2', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_Table('test_3', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_Table('test_4', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_Table('test_5', 'id');
create_distributed_table
--------------------------
(1 row)
CREATE VIEW referential_integrity_summary AS
WITH RECURSIVE referential_integrity_summary(n, table_name, referencing_relations, referenced_relations) AS
(
SELECT 0,'0','{}'::regclass[],'{}'::regclass[]
UNION ALL
SELECT
n + 1,
'test_' || n + 1|| '' as table_name,
(SELECT array_agg(get_referencing_relation_id_list::regclass ORDER BY 1) FROM get_referencing_relation_id_list(('test_' || (n +1) ) ::regclass)) as referencing_relations,
(SELECT array_agg(get_referenced_relation_id_list::regclass ORDER BY 1) FROM get_referenced_relation_id_list(('test_' || (n +1) ) ::regclass)) as referenced_by_relations
FROM referential_integrity_summary, pg_class
WHERE
pg_class.relname = ('test_' || (n +1))
AND n < 5
)
SELECT * FROM referential_integrity_summary WHERE n != 0 ORDER BY 1;
-- make sure that invalidation through ALTER TABLE works fine
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2} |
2 | test_2 | | {test_1}
3 | test_3 | |
4 | test_4 | |
5 | test_5 | |
(5 rows)
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2,test_3} |
2 | test_2 | {test_3} | {test_1}
3 | test_3 | | {test_2,test_1}
4 | test_4 | |
5 | test_5 | |
(5 rows)
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+------------------------+------------------------
1 | test_1 | {test_2,test_3,test_4} |
2 | test_2 | {test_3,test_4} | {test_1}
3 | test_3 | {test_4} | {test_2,test_1}
4 | test_4 | | {test_3,test_2,test_1}
5 | test_5 | |
(5 rows)
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-------------------------------+-------------------------------
1 | test_1 | {test_2,test_3,test_4,test_5} |
2 | test_2 | {test_3,test_4,test_5} | {test_1}
3 | test_3 | {test_4,test_5} | {test_2,test_1}
4 | test_4 | {test_5} | {test_3,test_2,test_1}
5 | test_5 | | {test_4,test_3,test_2,test_1}
(5 rows)
ROLLBACK;
-- similar test, but slightly different order of creating foreign keys
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2} |
2 | test_2 | | {test_1}
3 | test_3 | |
4 | test_4 | |
5 | test_5 | |
(5 rows)
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2} |
2 | test_2 | | {test_1}
3 | test_3 | {test_4} |
4 | test_4 | | {test_3}
5 | test_5 | |
(5 rows)
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2} |
2 | test_2 | | {test_1}
3 | test_3 | {test_4,test_5} |
4 | test_4 | {test_5} | {test_3}
5 | test_5 | | {test_4,test_3}
(5 rows)
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-------------------------------+-------------------------------
1 | test_1 | {test_2,test_3,test_4,test_5} |
2 | test_2 | {test_3,test_4,test_5} | {test_1}
3 | test_3 | {test_4,test_5} | {test_2,test_1}
4 | test_4 | {test_5} | {test_3,test_2,test_1}
5 | test_5 | | {test_4,test_3,test_2,test_1}
(5 rows)
ROLLBACK;
-- make sure that DROP CONSTRAINT works invalidates the cache correctly
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-------------------------------+-------------------------------
1 | test_1 | {test_2,test_3,test_4,test_5} |
2 | test_2 | {test_3,test_4,test_5} | {test_1}
3 | test_3 | {test_4,test_5} | {test_2,test_1}
4 | test_4 | {test_5} | {test_3,test_2,test_1}
5 | test_5 | | {test_4,test_3,test_2,test_1}
(5 rows)
ALTER TABLE test_3 DROP CONSTRAINT fkey_1;
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2} |
2 | test_2 | | {test_1}
3 | test_3 | {test_4,test_5} |
4 | test_4 | {test_5} | {test_3}
5 | test_5 | | {test_4,test_3}
(5 rows)
ROLLBACK;
-- make sure that CREATE TABLE invalidates the cache correctly
DROP TABLE test_1, test_2, test_3, test_4, test_5 CASCADE;
BEGIN;
CREATE TABLE test_1 (id int UNIQUE);
SELECT create_distributed_Table('test_1', 'id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE test_2 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_1(id));
SELECT create_distributed_Table('test_2', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2} |
2 | test_2 | | {test_1}
(2 rows)
CREATE TABLE test_3 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_2(id));
SELECT create_distributed_Table('test_3', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2,test_3} |
2 | test_2 | {test_3} | {test_1}
3 | test_3 | | {test_2,test_1}
(3 rows)
CREATE TABLE test_4 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_3(id));
SELECT create_distributed_Table('test_4', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+------------------------+------------------------
1 | test_1 | {test_2,test_3,test_4} |
2 | test_2 | {test_3,test_4} | {test_1}
3 | test_3 | {test_4} | {test_2,test_1}
4 | test_4 | | {test_3,test_2,test_1}
(4 rows)
CREATE TABLE test_5 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_4(id));
SELECT create_distributed_Table('test_5', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-------------------------------+-------------------------------
1 | test_1 | {test_2,test_3,test_4,test_5} |
2 | test_2 | {test_3,test_4,test_5} | {test_1}
3 | test_3 | {test_4,test_5} | {test_2,test_1}
4 | test_4 | {test_5} | {test_3,test_2,test_1}
5 | test_5 | | {test_4,test_3,test_2,test_1}
(5 rows)
COMMIT;
-- DROP TABLE works expected
-- re-create the constraints
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-------------------------------+-------------------------------
1 | test_1 | {test_2,test_3,test_4,test_5} |
2 | test_2 | {test_3,test_4,test_5} | {test_1}
3 | test_3 | {test_4,test_5} | {test_2,test_1}
4 | test_4 | {test_5} | {test_3,test_2,test_1}
5 | test_5 | | {test_4,test_3,test_2,test_1}
(5 rows)
DROP TABLE test_3 CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to constraint test_4_id_fkey on table test_4
drop cascades to constraint fkey_1 on table test_4
SELECT * FROM referential_integrity_summary;
n | table_name | referencing_relations | referenced_relations
---+------------+-----------------------+----------------------
1 | test_1 | {test_2} |
2 | test_2 | | {test_1}
(2 rows)
ROLLBACK;
-- Test schemas
BEGIN;
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;
CREATE TABLE fkey_intermediate_schema_1.test_6(id int PRIMARY KEY);
SELECT create_distributed_table('fkey_intermediate_schema_1.test_6', 'id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE fkey_intermediate_schema_2.test_7(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_1.test_6(id));
SELECT create_distributed_table('fkey_intermediate_schema_2.test_7','id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE fkey_intermediate_schema_1.test_8(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_2.test_7(id));
SELECT create_distributed_table('fkey_intermediate_schema_1.test_8', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_6'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
test_7
test_8
(2 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_7'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
test_8
(1 row)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_8'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_6'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_7'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
test_6
(1 row)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_8'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
test_6
test_7
(2 rows)
DROP SCHEMA fkey_intermediate_schema_2 CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to table test_7
drop cascades to constraint test_8_id_fkey on table test_8
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_6'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_8'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_6'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_8'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
ROLLBACK;
BEGIN;
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;
CREATE TABLE fkey_intermediate_schema_1.test_6(id int PRIMARY KEY);
SELECT create_distributed_table('fkey_intermediate_schema_1.test_6', 'id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE fkey_intermediate_schema_2.test_7(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_1.test_6(id));
SELECT create_distributed_table('fkey_intermediate_schema_2.test_7','id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE fkey_intermediate_schema_1.test_8(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_2.test_7(id));
SELECT create_distributed_table('fkey_intermediate_schema_1.test_8', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_6'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
test_7
test_8
(2 rows)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_7'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
test_8
(1 row)
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_8'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_6'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_7'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
test_6
(1 row)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_8'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
test_6
test_7
(2 rows)
DROP SCHEMA fkey_intermediate_schema_1 CASCADE;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table test_6
drop cascades to constraint test_7_id_fkey on table test_7
drop cascades to table test_8
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_7'::regclass) ORDER BY 1;
get_referencing_relation_id_list
----------------------------------
(0 rows)
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_7'::regclass) ORDER BY 1;
get_referenced_relation_id_list
---------------------------------
(0 rows)
ROLLBACK;
SET search_path TO public;
DROP SCHEMA fkey_graph CASCADE;
NOTICE: drop cascades to 12 other objects
DETAIL: drop cascades to function fkey_graph.get_referencing_relation_id_list(oid)
drop cascades to function fkey_graph.get_referenced_relation_id_list(oid)
drop cascades to table fkey_graph.dtt1
drop cascades to table fkey_graph.dtt2
drop cascades to table fkey_graph.dtt3
drop cascades to table fkey_graph.dtt4
drop cascades to view fkey_graph.referential_integrity_summary
drop cascades to table fkey_graph.test_1
drop cascades to table fkey_graph.test_2
drop cascades to table fkey_graph.test_3
drop cascades to table fkey_graph.test_4
drop cascades to table fkey_graph.test_5

View File

@ -0,0 +1,90 @@
CREATE SCHEMA fkey_reference_table;
SET search_path TO 'fkey_reference_table';
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 8;
SET citus.next_shard_id TO 7000000;
SET citus.next_placement_id TO 7000000;
SET citus.replication_model TO streaming;
-- Setup the view so that we can check if the foreign keys are created properly
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$);
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE TYPE")
(localhost,57638,t,"CREATE TYPE")
(2 rows)
CREATE VIEW table_fkeys_in_workers AS
SELECT
(json_populate_record(NULL::foreign_details,
json_array_elements_text((run_command_on_workers( $$
SELECT
COALESCE(json_agg(row_to_json(d)), '[]'::json)
FROM
(
SELECT
distinct name,
relid::regclass::text,
refd_relid::regclass::text
FROM
table_fkey_cols
)
d $$ )).RESULT::json )::json )).* ;
-- Check if MX can create foreign keys properly on foreign keys from distributed to reference tables
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referenced_table2(test_column int, test_column2 int, PRIMARY KEY(test_column2));
CREATE TABLE referencing_table(id int, ref_id int);
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
ALTER TABLE referencing_table ADD CONSTRAINT foreign_key_2 FOREIGN KEY (id) REFERENCES referenced_table2(test_column2) ON DELETE CASCADE;
SELECT create_reference_table('referenced_table');
create_reference_table
------------------------
(1 row)
SELECT create_reference_table('referenced_table2');
create_reference_table
------------------------
(1 row)
SELECT create_distributed_table('referencing_table', 'id');
create_distributed_table
--------------------------
(1 row)
SET search_path TO 'fkey_reference_table';
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1, 2;
name | relid | refd_relid
-----------------------+------------------------------------------------+------------------------------------------------
fkey_ref | fkey_reference_table.referencing_table | fkey_reference_table.referenced_table
fkey_ref | fkey_reference_table.referencing_table | fkey_reference_table.referenced_table
fkey_ref_7000002 | fkey_reference_table.referencing_table_7000002 | fkey_reference_table.referenced_table_7000000
fkey_ref_7000003 | fkey_reference_table.referencing_table_7000003 | fkey_reference_table.referenced_table_7000000
fkey_ref_7000004 | fkey_reference_table.referencing_table_7000004 | fkey_reference_table.referenced_table_7000000
fkey_ref_7000005 | fkey_reference_table.referencing_table_7000005 | fkey_reference_table.referenced_table_7000000
fkey_ref_7000006 | fkey_reference_table.referencing_table_7000006 | fkey_reference_table.referenced_table_7000000
fkey_ref_7000007 | fkey_reference_table.referencing_table_7000007 | fkey_reference_table.referenced_table_7000000
fkey_ref_7000008 | fkey_reference_table.referencing_table_7000008 | fkey_reference_table.referenced_table_7000000
fkey_ref_7000009 | fkey_reference_table.referencing_table_7000009 | fkey_reference_table.referenced_table_7000000
foreign_key_2 | fkey_reference_table.referencing_table | fkey_reference_table.referenced_table2
foreign_key_2 | fkey_reference_table.referencing_table | fkey_reference_table.referenced_table2
foreign_key_2_7000002 | fkey_reference_table.referencing_table_7000002 | fkey_reference_table.referenced_table2_7000001
foreign_key_2_7000003 | fkey_reference_table.referencing_table_7000003 | fkey_reference_table.referenced_table2_7000001
foreign_key_2_7000004 | fkey_reference_table.referencing_table_7000004 | fkey_reference_table.referenced_table2_7000001
foreign_key_2_7000005 | fkey_reference_table.referencing_table_7000005 | fkey_reference_table.referenced_table2_7000001
foreign_key_2_7000006 | fkey_reference_table.referencing_table_7000006 | fkey_reference_table.referenced_table2_7000001
foreign_key_2_7000007 | fkey_reference_table.referencing_table_7000007 | fkey_reference_table.referenced_table2_7000001
foreign_key_2_7000008 | fkey_reference_table.referencing_table_7000008 | fkey_reference_table.referenced_table2_7000001
foreign_key_2_7000009 | fkey_reference_table.referencing_table_7000009 | fkey_reference_table.referenced_table2_7000001
(20 rows)
DROP SCHEMA fkey_reference_table CASCADE;
NOTICE: drop cascades to 5 other objects
DETAIL: drop cascades to type foreign_details
drop cascades to view table_fkeys_in_workers
drop cascades to table referenced_table
drop cascades to table referenced_table2
drop cascades to table referencing_table
SET search_path TO DEFAULT;

View File

@ -576,7 +576,7 @@ BEGIN;
101
(1 row)
SELECT * FROM relation_acesses WHERE table_name IN ('table_6', 'table_1');
SELECT * FROM relation_acesses WHERE table_name IN ('table_6', 'table_1') ORDER BY 1,2;
table_name | select_access | dml_access | ddl_access
------------+-----------------+--------------+--------------
table_1 | parallel_access | not_accessed | not_accessed

View File

@ -576,7 +576,7 @@ BEGIN;
101
(1 row)
SELECT * FROM relation_acesses WHERE table_name IN ('table_6', 'table_1');
SELECT * FROM relation_acesses WHERE table_name IN ('table_6', 'table_1') ORDER BY 1,2;
table_name | select_access | dml_access | ddl_access
------------+-----------------+--------------+--------------
table_1 | parallel_access | not_accessed | not_accessed

View File

@ -74,7 +74,6 @@ SELECT create_distributed_table('test_table', 'a');
-- not useful if not in transaction
SELECT set_local_multi_shard_modify_mode_to_sequential();
WARNING: SET LOCAL can only be used in transaction blocks
set_local_multi_shard_modify_mode_to_sequential
-------------------------------------------------

View File

@ -25,7 +25,7 @@ test: multi_mx_tpch_query12 multi_mx_tpch_query14 multi_mx_tpch_query19
test: multi_mx_tpch_query3 multi_mx_tpch_query6 multi_mx_tpch_query7
test: multi_mx_tpch_query7_nested multi_mx_ddl
test: recursive_dml_queries_mx
test: multi_mx_repartition_udt_prepare
test: multi_mx_repartition_udt_prepare mx_foreign_key_to_reference_table
test: multi_mx_repartition_join_w1 multi_mx_repartition_join_w2 multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2
test: multi_mx_metadata
test: multi_mx_modifications

View File

@ -69,7 +69,7 @@ test: multi_null_minmax_value_pruning
test: multi_query_directory_cleanup
test: multi_task_assignment_policy multi_cross_shard
test: multi_utility_statements
test: multi_dropped_column_aliases
test: multi_dropped_column_aliases foreign_key_restriction_enforcement
test: multi_binary_master_copy_format
# ----------
@ -164,7 +164,7 @@ test: multi_modifications
test: multi_distribution_metadata
test: multi_generate_ddl_commands multi_create_shards multi_prune_shard_list multi_repair_shards
test: multi_upsert multi_simple_queries multi_create_insert_proxy multi_data_types
test: multi_utilities
test: multi_utilities foreign_key_to_reference_table
test: multi_modifying_xacts
test: multi_repartition_udt multi_repartitioned_subquery_udf multi_subtransactions
test: multi_transaction_recovery
@ -228,7 +228,7 @@ test: multi_citus_tools
# ----------
# multi_foreign_key tests foreign key push down on distributed tables
# ----------
test: multi_foreign_key
test: multi_foreign_key multi_foreign_key_relation_graph
# ----------
# multi_upgrade_reference_table tests for upgrade_reference_table UDF

View File

@ -0,0 +1,577 @@
--
-- Tests multiple commands in transactions where
-- there is foreign key relation between reference
-- tables and distributed tables
--
CREATE SCHEMA test_fkey_to_ref_in_tx;
SET search_path TO 'test_fkey_to_ref_in_tx';
SET citus.next_shard_id TO 2380000;
SET citus.next_placement_id TO 2380000;
SET citus.shard_replication_factor TO 1;
CREATE TABLE referece_table(id int PRIMARY KEY);
SELECT create_reference_table('referece_table');
CREATE TABLE on_update_fkey_table(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('on_update_fkey_table', 'id');
CREATE TABLE unrelated_dist_table(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('unrelated_dist_table', 'id');
ALTER TABLE on_update_fkey_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES referece_table(id) ON UPDATE CASCADE;
INSERT INTO referece_table SELECT i FROM generate_series(0, 100) i;
INSERT INTO on_update_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i;
INSERT INTO unrelated_dist_table SELECT i, i % 100 FROM generate_series(0, 1000) i;
-- in order to see when the mode automatically swithces to sequential execution
SET client_min_messages TO DEBUG1;
-- case 1.1: SELECT to a reference table is followed by a parallel SELECT to a distributed table
BEGIN;
SELECT count(*) FROM referece_table;
SELECT count(*) FROM on_update_fkey_table;
ROLLBACK;
-- case 1.2: SELECT to a reference table is followed by a multiple router SELECTs to a distributed table
BEGIN;
SELECT count(*) FROM referece_table;
SELECT count(*) FROM on_update_fkey_table WHERE id = 15;
SELECT count(*) FROM on_update_fkey_table WHERE id = 16;
SELECT count(*) FROM on_update_fkey_table WHERE id = 17;
SELECT count(*) FROM on_update_fkey_table WHERE id = 18;
ROLLBACK;
-- case 1.3: SELECT to a reference table is followed by a multi-shard UPDATE to a distributed table
BEGIN;
SELECT count(*) FROM referece_table;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ROLLBACK;
-- case 1.4: SELECT to a reference table is followed by a multiple sing-shard UPDATE to a distributed table
BEGIN;
SELECT count(*) FROM referece_table;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE id = 15;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE id = 16;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE id = 17;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE id = 18;
ROLLBACK;
-- case 1.5: SELECT to a reference table is followed by a DDL that touches fkey column
BEGIN;
SELECT count(*) FROM referece_table;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
ROLLBACK;
-- case 1.6: SELECT to a reference table is followed by an unrelated DDL
BEGIN;
SELECT count(*) FROM referece_table;
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
ROLLBACK;
-- case 1.7.1: SELECT to a reference table is followed by a DDL that is on
-- the foreign key column
BEGIN;
SELECT count(*) FROM referece_table;
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ROLLBACK;
-- case 1.7.2: SELECT to a reference table is followed by a DDL that is on
-- the foreign key column after a parallel query has been executed
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
SELECT count(*) FROM referece_table;
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ROLLBACK;
-- case 1.7.3: SELECT to a reference table is followed by a DDL that is not on
-- the foreign key column, and a parallel query has already been executed
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
SELECT count(*) FROM referece_table;
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
ROLLBACK;
-- case 1.8: SELECT to a reference table is followed by a COPY
BEGIN;
SELECT count(*) FROM referece_table;
COPY on_update_fkey_table FROM STDIN WITH CSV;
1001,99
1002,99
1003,99
1004,99
1005,99
\.
ROLLBACK;
-- case 2.1: UPDATE to a reference table is followed by a multi-shard SELECT
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101;
ROLLBACK;
-- case 2.2: UPDATE to a reference table is followed by multiple router SELECT
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 99;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 199;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 299;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 399;
ROLLBACK;
-- case 2.3: UPDATE to a reference table is followed by a multi-shard UPDATE
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
UPDATE on_update_fkey_table SET value_1 = 15;
ROLLBACK;
-- case 2.4: UPDATE to a reference table is followed by multiple router UPDATEs
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 1;
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 2;
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 3;
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 4;
ROLLBACK;
-- case 2.5: UPDATE to a reference table is followed by a DDL that touches fkey column
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
ROLLBACK;
-- case 2.6: UPDATE to a reference table is followed by an unrelated DDL
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT;
ROLLBACK;
-- case 2.7: UPDATE to a reference table is followed by COPY
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
COPY on_update_fkey_table FROM STDIN WITH CSV;
1001,99
1002,99
1003,99
1004,99
1005,99
\.
ROLLBACK;
-- case 2.8: UPDATE to a reference table is followed by TRUNCATE
BEGIN;
UPDATE referece_table SET id = 101 WHERE id = 99;
TRUNCATE on_update_fkey_table;
ROLLBACK;
-- case 3.1: an unrelated DDL to a reference table is followed by a real-time SELECT
BEGIN;
ALTER TABLE referece_table ALTER COLUMN id SET DEFAULT 1001;
SELECT count(*) FROM on_update_fkey_table;
ROLLBACK;
-- case 3.2: DDL that touches fkey column to a reference table is followed by a real-time SELECT
BEGIN;
ALTER TABLE referece_table ALTER COLUMN id SET DATA TYPE int;
SELECT count(*) FROM on_update_fkey_table;
ROLLBACK;
-- case 3.3: DDL to a reference table followed by a multi shard UPDATE
BEGIN;
ALTER TABLE referece_table ALTER COLUMN id SET DEFAULT 1001;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ROLLBACK;
-- case 3.4: DDL to a reference table followed by multiple router UPDATEs
BEGIN;
ALTER TABLE referece_table ALTER COLUMN id SET DEFAULT 1001;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 1;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 2;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 3;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 4;
ROLLBACK;
-- case 3.5: DDL to reference table followed by a DDL to dist table
BEGIN;
ALTER TABLE referece_table ALTER COLUMN id SET DATA TYPE smallint;
CREATE INDEX fkey_test_index_1 ON on_update_fkey_table(value_1);
ROLLBACK;
-- case 4.6: DDL to reference table followed by a DDL to dist table, both touching fkey columns
BEGIN;
ALTER TABLE referece_table ALTER COLUMN id SET DATA TYPE smallint;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
ROLLBACK;
-- case 3.7: DDL to a reference table is followed by COPY
BEGIN;
ALTER TABLE referece_table ADD COLUMN X int;
COPY on_update_fkey_table FROM STDIN WITH CSV;
1001,99
1002,99
1003,99
1004,99
1005,99
\.
ROLLBACK;
-- case 3.8: DDL to a reference table is followed by TRUNCATE
BEGIN;
ALTER TABLE referece_table ADD COLUMN X int;
TRUNCATE on_update_fkey_table;
ROLLBACK;
-- case 3.9: DDL to a reference table is followed by TRUNCATE
BEGIN;
ALTER TABLE referece_table ALTER COLUMN id SET DATA TYPE smallint;
TRUNCATE on_update_fkey_table;
ROLLBACK;
-----
--- Now, start testing the other way araound
-----
-- case 4.1: SELECT to a dist table is follwed by a SELECT to a reference table
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
SELECT count(*) FROM referece_table;
ROLLBACK;
-- case 4.2: SELECT to a dist table is follwed by a DML to a reference table
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
UPDATE referece_table SET id = 101 WHERE id = 99;
ROLLBACK;
-- case 4.3: SELECT to a dist table is follwed by an unrelated DDL to a reference table
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
ALTER TABLE referece_table ADD COLUMN X INT;
ROLLBACK;
-- case 4.4: SELECT to a dist table is follwed by a DDL to a reference table
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
ALTER TABLE referece_table ALTER COLUMN id SET DATA TYPE smallint;
ROLLBACK;
-- case 4.5: SELECT to a dist table is follwed by a TRUNCATE
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
TRUNCATE referece_table CASCADE;
ROLLBACK;
-- case 4.6: Router SELECT to a dist table is followed by a TRUNCATE
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE id = 9;
TRUNCATE referece_table CASCADE;
ROLLBACK;
-- case 5.1: Parallel UPDATE on distributed table follow by a SELECT
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
SELECT count(*) FROM referece_table;
ROLLBACK;
-- case 5.2: Parallel UPDATE on distributed table follow by a UPDATE
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
UPDATE referece_table SET id = 160 WHERE id = 15;
ROLLBACK;
-- case 5.3: Parallel UPDATE on distributed table follow by an unrelated DDL on reference table
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE referece_table ADD COLUMN X INT;
ROLLBACK;
-- case 5.4: Parallel UPDATE on distributed table follow by a related DDL on reference table
-- FIXME: Can we do better?
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE referece_table ALTER COLUMN id SET DATA TYPE smallint;
ROLLBACK;
-- case 6:1: Unrelated parallel DDL on distributed table followed by SELECT on ref. table
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM referece_table;
ROLLBACK;
-- case 6:2: Related parallel DDL on distributed table followed by SELECT on ref. table
BEGIN;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
UPDATE referece_table SET id = 160 WHERE id = 15;
ROLLBACK;
-- case 6:3: Unrelated parallel DDL on distributed table followed by UPDATE on ref. table
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM referece_table;
ROLLBACK;
-- case 6:4: Related parallel DDL on distributed table followed by SELECT on ref. table
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
UPDATE referece_table SET id = 160 WHERE id = 15;
ROLLBACK;
-- case 6:5: Unrelated parallel DDL on distributed table followed by unrelated DDL on ref. table
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
ALTER TABLE referece_table ADD COLUMN X int;
ROLLBACK;
-- case 6:6: Unrelated parallel DDL on distributed table followed by related DDL on ref. table
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
ROLLBACK;
-- some more extensive tests
-- UPDATE on dist table is followed by DELETE to reference table
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
DELETE FROM referece_table WHERE id = 99;
ROLLBACK;
-- an unrelated update followed by update on dist table and update
-- on reference table
BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
UPDATE referece_table SET id = 101 WHERE id = 99;
ROLLBACK;
-- an unrelated update followed by update on the reference table and update
-- on the cascading distributed table
-- note that the UPDATE on the reference table will try to set the execution
-- mode to sequential, which will fail since there is an already opened
-- parallel connections
BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE referece_table SET id = 101 WHERE id = 99;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ROLLBACK;
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_distributed_table('test_table_2', 'id');
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1 CASCADE;
ROLLBACK;
-- the fails since we're trying to switch sequential mode after
-- already executed a parallel query
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
CREATE TABLE tt4(id int PRIMARY KEY, value_1 int, FOREIGN KEY(id) REFERENCES tt4(id));
SELECT create_distributed_table('tt4', 'id');
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id), FOREIGN KEY(id) REFERENCES tt4(id));
SELECT create_distributed_table('test_table_2', 'id');
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1 CASCADE;
ROLLBACK;
-- same test with the above, but this time using
-- sequential mode, succeeds
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
CREATE TABLE tt4(id int PRIMARY KEY, value_1 int, FOREIGN KEY(id) REFERENCES tt4(id));
SELECT create_distributed_table('tt4', 'id');
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id), FOREIGN KEY(id) REFERENCES tt4(id));
SELECT create_distributed_table('test_table_2', 'id');
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1 CASCADE;
ROLLBACK;
-- another test with ALTER TABLE fails since we're already opened
-- parallel connection via create_distributed_table(), later
-- adding foreign key to reference table fails
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_2 ADD CONSTRAINT c_check FOREIGN KEY (value_1) REFERENCES test_table_1(id);
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1, test_table_2;
COMMIT;
-- same test with the above on sequential mode should work fine
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_2 ADD CONSTRAINT c_check FOREIGN KEY (value_1) REFERENCES test_table_1(id);
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1, test_table_2;
COMMIT;
-- similar test with the above, but this time the order of
-- create_distributed_table and create_reference_table is
-- changed
BEGIN;
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('test_table_2', 'id');
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
ALTER TABLE test_table_2 ADD CONSTRAINT c_check FOREIGN KEY (value_1) REFERENCES test_table_1(id);
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1 CASCADE;
ROLLBACK;
-- same test in sequential mode should succeed
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('test_table_2', 'id');
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
ALTER TABLE test_table_2 ADD CONSTRAINT c_check FOREIGN KEY (value_1) REFERENCES test_table_1(id);
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1 CASCADE;
ROLLBACK;
-- again a very similar test, but this time
-- a parallel SELECT is already executed before
-- setting the mode to sequential should fail
BEGIN;
SELECT count(*) FROM on_update_fkey_table;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('test_table_2', 'id');
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
ALTER TABLE test_table_2 ADD CONSTRAINT c_check FOREIGN KEY (value_1) REFERENCES test_table_1(id);
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_1 CASCADE;
ROLLBACK;
-- make sure that we cannot create hash distributed tables with
-- foreign keys to reference tables when they have data in it
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
INSERT INTO test_table_1 SELECT i FROM generate_series(0,100) i;
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
INSERT INTO test_table_2 SELECT i, i FROM generate_series(0,100) i;
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_2, test_table_1;
COMMIT;
-- the same test with above in sequential mode would still not work
-- since COPY cannot be executed in sequential mode
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE TABLE test_table_1(id int PRIMARY KEY);
INSERT INTO test_table_1 SELECT i FROM generate_series(0,100) i;
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
INSERT INTO test_table_2 SELECT i, i FROM generate_series(0,100) i;
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_2, test_table_1;
COMMIT;
-- we should be able to execute and DML/DDL/SELECT after we've
-- switched to sequential via create_distributed_table
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
-- and maybe some other test
CREATE INDEX i1 ON test_table_1(id);
ALTER TABLE test_table_2 ADD CONSTRAINT check_val CHECK (id > 0);
SELECT count(*) FROM test_table_2;
SELECT count(*) FROM test_table_1;
UPDATE test_table_2 SET value_1 = 15;
-- make sure that the output isn't too verbose
SET LOCAL client_min_messages TO ERROR;
DROP TABLE test_table_2, test_table_1;
COMMIT;
RESET client_min_messages;
DROP SCHEMA test_fkey_to_ref_in_tx CASCADE;
SET search_path TO public;

View File

@ -0,0 +1,942 @@
--
-- FOREIGN_KEY_TO_REFERENCE_TABLE
--
CREATE SCHEMA fkey_reference_table;
SET search_path TO 'fkey_reference_table';
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 8;
SET citus.next_shard_id TO 7000000;
SET citus.next_placement_id TO 7000000;
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$);
CREATE VIEW table_fkeys_in_workers AS
SELECT
(json_populate_record(NULL::foreign_details,
json_array_elements_text((run_command_on_workers( $$
SELECT
COALESCE(json_agg(row_to_json(d)), '[]'::json)
FROM
(
SELECT
distinct name,
relid::regclass::text,
refd_relid::regclass::text
FROM
table_fkey_cols
)
d $$ )).RESULT::json )::json )).* ;
CREATE TABLE referenced_table(id int UNIQUE, test_column int);
SELECT create_reference_table('referenced_table');
-- we still do not support update/delete operations through foreign constraints if the foreign key includes the distribution column
-- All should fail
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET NULL;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET NULL);
SELECT create_distributed_table('referencing_table', 'ref_id');
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET DEFAULT;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON DELETE SET DEFAULT);
SELECT create_distributed_table('referencing_table', 'ref_id');
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET NULL;
DROP TABLE referencing_table;
BEGIN;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id) ON UPDATE SET NULL);
SELECT create_distributed_table('referencing_table', 'ref_id');
ROLLBACK;
-- try with multiple columns including the distribution column
DROP TABLE referenced_table;
CREATE TABLE referenced_table(id int, test_column int, PRIMARY KEY(id, test_column));
SELECT create_reference_table('referenced_table');
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id, ref_id) REFERENCES referenced_table(id, test_column) ON UPDATE SET DEFAULT;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(id, ref_id) REFERENCES referenced_table(id, test_column) ON UPDATE SET DEFAULT);
SELECT create_distributed_table('referencing_table', 'ref_id');
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id, ref_id) REFERENCES referenced_table(id, test_column) ON UPDATE CASCADE;
DROP TABLE referencing_table;
BEGIN;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(id, ref_id) REFERENCES referenced_table(id, test_column) ON UPDATE CASCADE);
SELECT create_distributed_table('referencing_table', 'ref_id');
ROLLBACK;
-- all of the above is supported if the foreign key does not include distribution column
DROP TABLE referenced_table;
CREATE TABLE referenced_table(id int, test_column int, PRIMARY KEY(id));
SELECT create_reference_table('referenced_table');
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id) REFERENCES referenced_table(id) ON DELETE SET NULL;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(id) REFERENCES referenced_table(id) ON DELETE SET NULL);
SELECT create_distributed_table('referencing_table', 'ref_id');
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id) REFERENCES referenced_table(id) ON DELETE SET DEFAULT;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
BEGIN;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(id) REFERENCES referenced_table(id) ON DELETE SET DEFAULT);
SELECT create_distributed_table('referencing_table', 'ref_id');
COMMIT;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id) REFERENCES referenced_table(id) ON UPDATE SET NULL;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id) REFERENCES referenced_table(id) ON UPDATE SET DEFAULT;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id) REFERENCES referenced_table(id) ON UPDATE CASCADE;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
-- foreign keys are only supported when the replication factor = 1
SET citus.shard_replication_factor TO 2;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(id);
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
SET citus.shard_replication_factor TO 1;
-- simple create_distributed_table should work in/out transactions on tables with foreign key to reference tables
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'ref_id');
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
DROP TABLE referenced_table;
BEGIN;
CREATE TABLE referenced_table(id int, test_column int, PRIMARY KEY(id));
SELECT create_reference_table('referenced_table');
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'ref_id');
COMMIT;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
-- foreign keys are supported either in between distributed tables including the
-- distribution column or from distributed tables to reference tables.
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id', 'append');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(id);
SELECT * FROM table_fkeys_in_workers WHERE name LIKE 'fkey_ref%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id', 'range');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(id);
SELECT * FROM table_fkeys_in_workers WHERE name LIKE 'fkey_ref%' ORDER BY 1,2,3;
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- test foreign constraint with correct conditions
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_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref 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 SELECT x, x from generate_series(1,1000) as f(x);
INSERT INTO referencing_table SELECT x, x from generate_series(1,500) as f(x);
-- test deletes
-- test delete from referenced table while there is corresponding value in referencing table
DELETE FROM referenced_table WHERE id > 3;
-- test delete from referenced table while there is NO corresponding value in referencing table
DELETE FROM referenced_table WHERE id = 501;
-- test cascading truncate
TRUNCATE referenced_table CASCADE;
SELECT count(*) FROM referencing_table;
-- drop table for next tests
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- self referencing foreign key on reference tables are not allowed
-- TODO try create_reference_table with already created foreign key.
CREATE TABLE referenced_table(id int, test_column int, PRIMARY KEY(id));
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_reference_table('referenced_table');
SELECT create_reference_table('referencing_table');
-- self referencing foreign key
ALTER TABLE referenced_table ADD CONSTRAINT fkey_ref FOREIGN KEY (test_column) REFERENCES referenced_table(id);
-- foreign Keys from reference table to reference table are not allowed
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id) REFERENCES referenced_table(id) ON UPDATE CASCADE;
DROP TABLE referenced_table;
DROP TABLE referencing_table;
-- cascades on delete with different schemas
CREATE SCHEMA referenced_schema;
CREATE SCHEMA referencing_schema;
CREATE TABLE referenced_schema.referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column));
CREATE TABLE referencing_schema.referencing_table(id int, ref_id int);
SELECT create_reference_table('referenced_schema.referenced_table');
SELECT create_distributed_table('referencing_schema.referencing_table', 'id');
ALTER TABLE referencing_schema.referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_schema.referenced_table(id) ON DELETE CASCADE;
INSERT INTO referenced_schema.referenced_table SELECT x, x from generate_series(1,1000) as f(x);
INSERT INTO referencing_schema.referencing_table SELECT x, x from generate_series(1,1000) as f(x);
DELETE FROM referenced_schema.referenced_table WHERE id > 800;
SELECT count(*) FROM referencing_schema.referencing_table;
DROP SCHEMA referenced_schema CASCADE;
DROP SCHEMA referencing_schema CASCADE;
-- on delete set update cascades properly
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, ref_id int DEFAULT 1);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON DELETE SET DEFAULT;
INSERT INTO referenced_table SELECT x, x FROM generate_series(1,1000) AS f(x);
INSERT INTO referencing_table SELECT x, x FROM generate_series(1,1000) AS f(x);
DELETE FROM referenced_table WHERE test_column > 800;
SELECT count(*) FROM referencing_table WHERE ref_id = 1;
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- foreign key as composite key
CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int);
SELECT run_command_on_workers($$CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int)$$) ORDER BY 1;
CREATE TABLE referenced_table(test_column composite, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, referencing_composite composite);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (referencing_composite) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
INSERT INTO referenced_table SELECT (x+1, x+1)::composite FROM generate_series(1,1000) AS f(x);
INSERT INTO referencing_table SELECT x, (x+1, x+1)::composite FROM generate_series(1,1000) AS f(x);
DELETE FROM referenced_table WHERE (test_column).key1 > 900;
SELECT count(*) FROM referencing_table;
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- In the following test, we'll use a SERIAL column as the referenced column
-- in the foreign constraint. We'll first show that and insert on non-serial
-- column successfully inserts into the serial and referenced column.
-- Accordingly, the inserts into the referencing table which references to the
-- serial column will be successful.
CREATE TABLE referenced_table(test_column SERIAL PRIMARY KEY, test_column2 int);
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
INSERT INTO referenced_table(test_column2) SELECT x FROM generate_series(1,1000) AS f(x);
INSERT INTO referencing_table SELECT x, x FROM generate_series(1,1000) AS f(x);
DELETE FROM referenced_table WHERE test_column2 > 10;
SELECT count(*) FROM referencing_table;
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- In the following test, we'll use a SERIAL column as the referencing column
-- in the foreign constraint. We'll first show that the values that exist
-- in the referenced tables are successfully generated by the serial column
-- and inserted to the distributed table. However, if the values that are generated
-- by serial column do not exist on the referenced table, the query fails.
CREATE TABLE referenced_table(test_column int PRIMARY KEY, test_column2 int);
CREATE TABLE referencing_table(id int, ref_id SERIAL);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
INSERT INTO referenced_table SELECT x,x FROM generate_series(1,1000) AS f(x);
-- Success for existing inserts
INSERT INTO referencing_table(id) SELECT x FROM generate_series(1,1000) AS f(x);
-- Fails for non existing value inserts (serial is already incremented)
INSERT INTO referencing_table(id) SELECT x FROM generate_series(1,10) AS f(x);
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- In the following test, we'll use a SERIAL column as the referencing column
-- and referenced columns in a foreign constraint. We'll first show that the
-- the inserts into referenced column will successfully generate and insert
-- data into serial column. Then, we will be successfully insert the same amount
-- of data into referencing table. However, if the values that are generated
-- by serial column do not exist on the referenced table, the query fails.
CREATE TABLE referenced_table(test_column SERIAL PRIMARY KEY, test_column2 int);
CREATE TABLE referencing_table(id int, ref_id SERIAL);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
INSERT INTO referenced_table(test_column2) SELECT x FROM generate_series(1,1000) AS f(x);
-- Success for existing values
INSERT INTO referencing_table(id) SELECT x FROM generate_series(1,1000) AS f(x);
-- Fails for non existing value inserts (serial is already incremented)
INSERT INTO referencing_table(id) SELECT x FROM generate_series(1,10) AS f(x);
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- In the following test, we use a volatile function in the referencing
-- column in a foreign constraint. We show that if the data exists in the
-- referenced table, we can successfully use volatile functions with
-- foreign constraints.
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, ref_id int DEFAULT -1);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON DELETE SET DEFAULT;
INSERT INTO referenced_table SELECT x, x FROM generate_series(0,1000) AS f(x);
INSERT INTO referencing_table SELECT x,(random()*1000)::int FROM generate_series(0,1000) AS f(x);
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- In the following test, we show that Citus currently does not support
-- VALIDATE command.
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, ref_id int DEFAULT -1);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON DELETE SET DEFAULT NOT VALID;
-- Even if the foreign constraint is added with "NOT VALID",
-- we make sure that it is still applied to the upcoming inserts.
INSERT INTO referenced_table SELECT x, x FROM generate_series(0,1000) AS f(x);
INSERT INTO referencing_table SELECT x, x FROM generate_series(0,1000) AS f(x);
-- we expect this to fail because of the foreign constraint.
INSERT INTO referencing_table SELECT x, x FROM generate_series(1000,1001) AS f(x);
-- currently not supported
ALTER TABLE referencing_table VALIDATE CONSTRAINT fkey_ref;
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- In the following tests, we create a foreign constraint with
-- ON UPDATE CASCADE and see if it works properly with cascading upsert
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, ref_id int DEFAULT -1);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON UPDATE CASCADE;
INSERT INTO referenced_table SELECT x, x FROM generate_series(0,1000) AS f(x);
INSERT INTO referencing_table SELECT x, x FROM generate_series(0,1000) AS f(x);
INSERT INTO referenced_table VALUES (1,2), (2,3), (3,4), (4,5)
ON CONFLICT (test_column)
DO UPDATE
SET test_column = -1 * EXCLUDED.test_column;
SELECT * FROM referencing_table WHERE ref_id < 0 ORDER BY 1;
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- create_distributed_table should fail for tables with data if fkey exists to reference table
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, ref_id int DEFAULT -1, FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column) ON UPDATE CASCADE);
INSERT INTO referenced_table VALUES (1,1), (2,2), (3,3);
INSERT INTO referencing_table VALUES (1,1), (2,2), (3,3);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
BEGIN;
SELECT create_distributed_table('referencing_table', 'id');
COMMIT;
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
-- Chained references
-- In the following test, we create foreign keys from one column in a distributed
-- table to two reference tables. We expect to see that even if a data exist in
-- one reference table, it is not going to be inserted in to referencing table
-- because of lack of the key in the other table. Data can only be inserted into
-- referencing table if it exists in both referenced tables.
-- Additionally, delete or update in one referenced table should cascade properly.
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referenced_table2(test_column int, test_column2 int, PRIMARY KEY(test_column2));
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_reference_table('referenced_table');
SELECT create_reference_table('referenced_table2');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
ALTER TABLE referencing_table ADD CONSTRAINT foreign_key_2 FOREIGN KEY (id) REFERENCES referenced_table2(test_column2) ON DELETE CASCADE;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
INSERT INTO referenced_table SELECT x, x+1 FROM generate_series(0,1000) AS f(x);
INSERT INTO referenced_table2 SELECT x, x+1 FROM generate_series(500,1500) AS f(x);
-- should fail
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(0,1500) AS f(x);
-- should fail
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(0,400) AS f(x);
-- should fail
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(1000,1400) AS f(x);
-- should success
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(600,900) AS f(x);
SELECT count(*) FROM referencing_table;
DELETE FROM referenced_table WHERE test_column < 700;
SELECT count(*) FROM referencing_table;
DELETE FROM referenced_table2 WHERE test_column2 > 800;
SELECT count(*) FROM referencing_table;
DROP TABLE referenced_table CASCADE;
DROP TABLE referenced_table2 CASCADE;
DROP TABLE referencing_table CASCADE;
-- check if the above fkeys are created with create_distributed_table
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referenced_table2(test_column int, test_column2 int, PRIMARY KEY(test_column2));
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE, FOREIGN KEY (id) REFERENCES referenced_table2(test_column2) ON DELETE CASCADE);
SELECT create_reference_table('referenced_table');
SELECT create_reference_table('referenced_table2');
SELECT create_distributed_table('referencing_table', 'id');
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE referenced_table CASCADE;
DROP TABLE referenced_table2 CASCADE;
DROP TABLE referencing_table CASCADE;
-- In the following test, we create foreign keys from two columns in a distributed
-- table to two reference tables separately. We expect to see that even if a data
-- exist in one reference table for one column, it is not going to be inserted in
-- to referencing table because the other constraint doesn't hold. Data can only
-- be inserted into referencing table if both columns exist in respective columns
-- in referenced tables.
-- Additionally, delete or update in one referenced table should cascade properly.
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referenced_table2(test_column int, test_column2 int, PRIMARY KEY(test_column2));
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_reference_table('referenced_table');
SELECT create_reference_table('referenced_table2');
SELECT create_distributed_table('referencing_table', 'id');
BEGIN;
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
ALTER TABLE referencing_table ADD CONSTRAINT foreign_key_2 FOREIGN KEY (ref_id) REFERENCES referenced_table2(test_column2) ON DELETE CASCADE;
COMMIT;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
INSERT INTO referenced_table SELECT x, x+1 FROM generate_series(0,1000) AS f(x);
INSERT INTO referenced_table2 SELECT x, x+1 FROM generate_series(500,1500) AS f(x);
-- should fail
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(0,1500) AS f(x);
-- should fail
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(0,400) AS f(x);
-- should fail
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(1000,1400) AS f(x);
-- should success
INSERT INTO referencing_table SELECT x, x+501 FROM generate_series(0,1000) AS f(x);
SELECT count(*) FROM referencing_table;
DELETE FROM referenced_table WHERE test_column < 700;
SELECT count(*) FROM referencing_table;
DELETE FROM referenced_table2 WHERE test_column2 > 800;
SELECT count(*) FROM referencing_table;
DROP TABLE referenced_table CASCADE;
DROP TABLE referenced_table2 CASCADE;
DROP TABLE referencing_table CASCADE;
-- check if the above fkeys are created when create_distributed_table is used for 1 foreign key and alter table for the other
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referenced_table2(test_column int, test_column2 int, PRIMARY KEY(test_column2));
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE);
BEGIN;
SELECT create_reference_table('referenced_table');
SELECT create_reference_table('referenced_table2');
SELECT create_distributed_table('referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT foreign_key_2 FOREIGN KEY (ref_id) REFERENCES referenced_table2(test_column2) ON DELETE CASCADE;
COMMIT;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE referenced_table CASCADE;
DROP TABLE referenced_table2 CASCADE;
DROP TABLE referencing_table CASCADE;
-- two distributed tables are referencing to one reference table and
-- in the same time the distributed table 2 is referencing to
-- distributed table 1. Thus, we have a triangular
-- distributed table 1 has a foreign key from the distribution column to reference table
-- distributed table 2 has a foreign key from a non-distribution column to reference table
-- distributed table 2 has a foreign key to distributed table 1 on the distribution column
-- We show that inserts into distributed table 2 will fail if the data does not exist in distributed table 1
-- Delete from reference table cascades to both of the distributed tables properly
CREATE TABLE referenced_table(test_column int, test_column2 int UNIQUE, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int PRIMARY KEY, ref_id int);
CREATE TABLE referencing_table2(id int, ref_id int);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
SELECT create_distributed_table('referencing_table2', 'id');
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
ALTER TABLE referencing_table2 ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column2) ON DELETE CASCADE;
ALTER TABLE referencing_table2 ADD CONSTRAINT fkey_ref_to_dist FOREIGN KEY (id) REFERENCES referencing_table(id) ON DELETE CASCADE;
COMMIT;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1,2,3;
INSERT INTO referenced_table SELECT x, x+1 FROM generate_series(0,1000) AS f(x);
-- should fail
INSERT INTO referencing_table2 SELECT x, x+1 FROM generate_series(0,100) AS f(x);
-- should success
INSERT INTO referencing_table SELECT x, x+1 FROM generate_series(0,400) AS f(x);
-- should fail
INSERT INTO referencing_table2 SELECT x, x+1 FROM generate_series(200,500) AS f(x);
-- should success
INSERT INTO referencing_table2 SELECT x, x+1 FROM generate_series(0,300) AS f(x);
DELETE FROM referenced_table WHERE test_column < 200;
SELECT count(*) FROM referencing_table;
SELECT count(*) FROM referencing_table2;
DELETE FROM referencing_table WHERE id > 200;
SELECT count(*) FROM referencing_table2;
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
DROP TABLE referencing_table2 CASCADE;
-- Check if the above fkeys are created with create_distributed_table
CREATE TABLE referenced_table(test_column int, test_column2 int UNIQUE, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int PRIMARY KEY, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE);
CREATE TABLE referencing_table2(id int, ref_id int, FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column2) ON DELETE CASCADE, FOREIGN KEY (id) REFERENCES referencing_table(id) ON DELETE CASCADE);
SELECT create_reference_table('referenced_table');
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
SELECT create_distributed_table('referencing_table', 'id');
SELECT create_distributed_table('referencing_table2', 'id');
COMMIT;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
DROP TABLE referencing_table2 CASCADE;
-- In this test we have a chained relationship in form of
-- distributed table (referencing_referencing_table) has a foreign key with two columns
-- to another distributed table (referencing_table)
-- referencing_table has another foreign key with 2 columns to referenced_table.
-- We will show that a cascading delete on referenced_table reaches to referencing_referencing_table.
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column, test_column2));
CREATE TABLE referencing_table(id int, ref_id int, ref_id2 int, PRIMARY KEY(id, ref_id));
CREATE TABLE referencing_referencing_table(id int, ref_id int, FOREIGN KEY (id, ref_id) REFERENCES referencing_table(id, ref_id) ON DELETE CASCADE);
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
SELECT create_distributed_table('referencing_referencing_table', 'id');
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (ref_id, ref_id2) REFERENCES referenced_table(test_column, test_column2) ON DELETE CASCADE;
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.referencing%' ORDER BY 1,2,3;
INSERT INTO referenced_table SELECT x, x+1 FROM generate_series(1,1000) AS f(x);
INSERT INTO referencing_table SELECT x, x+1, x+2 FROM generate_series(1,999) AS f(x);
INSERT INTO referencing_referencing_table SELECT x, x+1 FROM generate_series(1,999) AS f(x);
DELETE FROM referenced_table WHERE test_column > 800;
SELECT max(ref_id) FROM referencing_referencing_table;
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table CASCADE;
DROP TABLE referencing_referencing_table;
-- test if create_distributed_table works in transactions with some edge cases
-- the following checks if create_distributed_table works on foreign keys when
-- one of them is a self-referencing table of multiple distributed tables
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(id) REFERENCES test_table_1(id));
SELECT create_distributed_table('test_table_2', 'id');
CREATE TABLE test_table_3(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id), FOREIGN KEY(id) REFERENCES test_table_2(id));
SELECT create_distributed_table('test_table_3', 'id');
DROP TABLE test_table_1 CASCADE;
ROLLBACK;
-- create_reference_table, create_distributed_table and ALTER TABLE in the same transaction
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
SELECT create_reference_table('test_table_1');
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_2 ADD CONSTRAINT c_check FOREIGN KEY (value_1) REFERENCES test_table_1(id);
DROP TABLE test_table_1, test_table_2;
COMMIT;
-- the order of create_reference_table and create_distributed_table is changed
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY, value_1 int);
SELECT create_distributed_table('test_table_1', 'id');
CREATE TABLE test_table_2(id int PRIMARY KEY);
SELECT create_reference_table('test_table_2');
ALTER TABLE test_table_1 ADD CONSTRAINT c_check FOREIGN KEY (value_1) REFERENCES test_table_2(id);
DROP TABLE test_table_2 CASCADE;
ROLLBACK;
-- make sure that we fail if we need parallel data load
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
INSERT INTO test_table_1 SELECT i FROM generate_series(0,100) i;
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
INSERT INTO test_table_2 SELECT i, i FROM generate_series(0,100) i;
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
DROP TABLE test_table_2, test_table_1;
COMMIT;
-- make sure that other DDLs/DMLs also work fine
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
CREATE INDEX i1 ON test_table_1(id);
ALTER TABLE test_table_2 ADD CONSTRAINT check_val CHECK (id > 0);
DROP TABLE test_table_2, test_table_1;
COMMIT;
-- The following tests check if the DDLs affecting foreign keys work as expected
-- check if we can drop the foreign constraint
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
ALTER TABLE test_table_2 DROP CONSTRAINT test_table_2_value_1_fkey;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can drop the foreign constraint in a transaction right after ADD CONSTRAINT
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_2 ADD CONSTRAINT foreign_key FOREIGN KEY(value_1) REFERENCES test_table_1(id);
ALTER TABLE test_table_2 DROP CONSTRAINT test_table_2_value_1_fkey;
COMMIT;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can drop the primary key which cascades to the foreign key
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_1 DROP CONSTRAINT test_table_1_pkey CASCADE;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can drop the primary key which cascades to the foreign key in a transaction block
BEGIN;
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_1 DROP CONSTRAINT test_table_1_pkey CASCADE;
COMMIT;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can drop the column which foreign key is referencing from
CREATE TABLE test_table_1(id int PRIMARY KEY, id2 int);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_2 DROP COLUMN value_1;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can drop the column which foreign key is referencing from in a transaction block
CREATE TABLE test_table_1(id int PRIMARY KEY, id2 int);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
BEGIN;
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_2 DROP COLUMN value_1;
COMMIT;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can drop the column which foreign key is referencing to
CREATE TABLE test_table_1(id int PRIMARY KEY, id2 int);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_1 DROP COLUMN id CASCADE;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can drop the column which foreign key is referencing from in a transaction block
CREATE TABLE test_table_1(id int PRIMARY KEY, id2 int);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
BEGIN;
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_1 DROP COLUMN id CASCADE;
COMMIT;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can alter the column type which foreign key is referencing to
CREATE TABLE test_table_1(id int PRIMARY KEY, id2 int);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
INSERT INTO test_table_1 VALUES (1,1), (2,2), (3,3);
INSERT INTO test_table_2 VALUES (1,1), (2,2), (3,3);
-- should succeed
ALTER TABLE test_table_2 ALTER COLUMN value_1 SET DATA TYPE bigint;
ALTER TABLE test_table_1 ALTER COLUMN id SET DATA TYPE bigint;
INSERT INTO test_table_1 VALUES (2147483648,4);
INSERT INTO test_table_2 VALUES (4,2147483648);
-- should fail since there is a bigint out of integer range > (2^32 - 1)
ALTER TABLE test_table_2 ALTER COLUMN value_1 SET DATA TYPE int;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1 CASCADE;
DROP TABLE test_table_2;
-- check if we can alter the column type and drop it which foreign key is referencing to in a transaction block
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
BEGIN;
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
ALTER TABLE test_table_2 ALTER COLUMN value_1 SET DATA TYPE bigint;
ALTER TABLE test_table_1 DROP COLUMN id CASCADE;
COMMIT;
SELECT count(*) FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%';
DROP TABLE test_table_1, test_table_2;
-- check if we can TRUNCATE the referenced table
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
INSERT INTO test_table_1 VALUES (1),(2),(3);
INSERT INTO test_table_2 VALUES (1,1),(2,2),(3,3);
TRUNCATE test_table_1 CASCADE;
SELECT * FROM test_table_2;
DROP TABLE test_table_1, test_table_2;
-- check if we can TRUNCATE the referenced table in a transaction
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
INSERT INTO test_table_1 VALUES (1),(2),(3);
INSERT INTO test_table_2 VALUES (1,1),(2,2),(3,3);
BEGIN;
TRUNCATE test_table_1 CASCADE;
COMMIT;
SELECT * FROM test_table_2;
DROP TABLE test_table_1, test_table_2;
-- check if we can TRUNCATE the referenced table in a transaction after inserts
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
BEGIN;
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
INSERT INTO test_table_1 VALUES (1),(2),(3);
INSERT INTO test_table_2 VALUES (1,1),(2,2),(3,3);
TRUNCATE test_table_1 CASCADE;
COMMIT;
SELECT * FROM test_table_2;
DROP TABLE test_table_1, test_table_2;
-- check if we can TRUNCATE the referencing table
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
INSERT INTO test_table_1 VALUES (1),(2),(3);
INSERT INTO test_table_2 VALUES (1,1),(2,2),(3,3);
TRUNCATE test_table_2 CASCADE;
SELECT * FROM test_table_2;
SELECT * FROM test_table_1;
DROP TABLE test_table_1, test_table_2;
-- check if we can TRUNCATE the referencing table in a transaction
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int, FOREIGN KEY(value_1) REFERENCES test_table_1(id));
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
INSERT INTO test_table_1 VALUES (1),(2),(3);
INSERT INTO test_table_2 VALUES (1,1),(2,2),(3,3);
BEGIN;
TRUNCATE test_table_2 CASCADE;
COMMIT;
SELECT * FROM test_table_2;
SELECT * FROM test_table_1;
DROP TABLE test_table_1, test_table_2;
-- check if we successfuly set multi_shard_modify_mode to sequential after sequentially running DDLs
-- in transaction since the upcoming DDLs need to run sequentially.
CREATE TABLE test_table_1(id int PRIMARY KEY);
CREATE TABLE test_table_2(id int PRIMARY KEY, value_1 int);
CREATE TABLE test_table_3(id int PRIMARY KEY, value_1 int);
SELECT create_reference_table('test_table_1');
SELECT create_distributed_table('test_table_2', 'id');
SELECT create_distributed_table('test_table_3', 'id');
BEGIN;
ALTER TABLE test_table_2 ADD CONSTRAINT fkey FOREIGN KEY (value_1) REFERENCES test_table_1(id);
ALTER TABLE test_table_3 ADD COLUMN test_column int;
ALTER TABLE test_table_1 DROP COLUMN id CASCADE;
ALTER TABLE test_table_1 ADD COLUMN id int;
COMMIT;
DROP TABLE test_table_1, test_table_2, test_table_3;
-- NOTE: Postgres does not support foreign keys on partitioned tables currently.
-- However, we can create foreign keys to/from the partitions themselves.
-- The following tests chech if we create the foreign constraints in partitions properly.
CREATE TABLE referenced_table(id int PRIMARY KEY, test_column int);
CREATE TABLE referencing_table(id int, value_1 int) PARTITION BY RANGE (value_1);
CREATE TABLE referencing_table_0 PARTITION OF referencing_table FOR VALUES FROM (0) TO (2);
CREATE TABLE referencing_table_2 PARTITION OF referencing_table FOR VALUES FROM (2) TO (4);
CREATE TABLE referencing_table_4 PARTITION OF referencing_table FOR VALUES FROM (4) TO (6);
-- partitioned tables are not supported as reference tables
select create_reference_table('referencing_table');
-- partitioned tables are supported as hash distributed table
SELECT create_reference_table('referenced_table');
SELECT create_distributed_table('referencing_table', 'id');
-- add foreign constraints in between partitions
ALTER TABLE referencing_table_0 ADD CONSTRAINT pkey PRIMARY KEY (id);
ALTER TABLE referencing_table_4 ADD CONSTRAINT fkey FOREIGN KEY (id) REFERENCES referencing_table_0;
-- add foreign constraint from a partition to reference table
ALTER TABLE referencing_table_4 ADD CONSTRAINT fkey_to_ref FOREIGN KEY (value_1) REFERENCES referenced_table;
-- should fail since the data will flow to partitioning_test_4 and it has a foreign constraint to partitioning_test_0 on id column
INSERT INTO referencing_table VALUES (0, 5);
-- should succeed on partitioning_test_0
INSERT INTO referencing_table VALUES (0, 1);
SELECT * FROM referencing_table;
-- should fail since partitioning_test_4 has foreign constraint to referenced_table on value_1 column
INSERT INTO referencing_table VALUES (0, 5);
INSERT INTO referenced_table VALUES(5,5);
-- should succeed since both of the foreign constraints are positive
INSERT INTO referencing_table VALUES (0, 5);
DROP TABLE referenced_table CASCADE;
DROP TABLE referencing_table;
DROP SCHEMA fkey_reference_table CASCADE;
SET search_path TO DEFAULT;

View File

@ -32,6 +32,16 @@ CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCE
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
DROP TABLE referencing_table;
-- self referencing table with replication factor > 1
CREATE TABLE self_referencing_table(id int, ref_id int, PRIMARY KEY (id, ref_id), FOREIGN KEY(id,ref_id) REFERENCES self_referencing_table(id, ref_id));
SELECT create_distributed_table('self_referencing_table', 'id', 'hash');
DROP TABLE self_referencing_table;
CREATE TABLE self_referencing_table(id int, ref_id int, PRIMARY KEY (id, ref_id));
SELECT create_distributed_table('self_referencing_table', 'id', 'hash');
ALTER TABLE self_referencing_table ADD CONSTRAINT fkey FOREIGN KEY(id,ref_id) REFERENCES self_referencing_table(id, ref_id);
DROP TABLE self_referencing_table;
-- test foreign constraint creation on NOT co-located tables
SET citus.shard_count TO 8;
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id));
@ -55,9 +65,27 @@ SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- test foreign constraint with correct conditions
-- test foreign constraint creation on append and range distributed tables
-- foreign keys are supported either in between distributed tables including the
-- distribution column or from distributed tables to reference tables.
SET citus.shard_replication_factor TO 1;
CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column));
SELECT create_distributed_table('referenced_table', 'id', 'hash');
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'id', 'append');
DROP TABLE referencing_table;
DROP TABLE referenced_table;
CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column));
SELECT create_distributed_table('referenced_table', 'id', 'range');
CREATE TABLE referencing_table(id int, ref_id int,FOREIGN KEY (id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referencing_table', 'id', 'range');
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- test foreign constraint with correct conditions
CREATE TABLE referenced_table(id int UNIQUE, test_column int, PRIMARY KEY(id, test_column));
CREATE TABLE referencing_table(id int, ref_id int, FOREIGN KEY(ref_id) REFERENCES referenced_table(id));
SELECT create_distributed_table('referenced_table', 'id', 'hash');
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
@ -211,13 +239,12 @@ DROP TABLE referenced_table;
-- Similar tests, but this time we push foreign key constraints created by ALTER TABLE queries
-- create tables
SET citus.shard_count TO 4;
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);
SELECT create_distributed_table('referenced_table', 'id', 'hash');
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);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash');
-- verify that we skip foreign key validation when propagation is turned off
-- not skipping validation would result in a distributed query, which emits debug messages
@ -243,6 +270,13 @@ ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id)
ALTER TABLE referencing_table ADD FOREIGN KEY(ref_id) REFERENCES referenced_table(id);
-- test foreign constraint creation on NOT 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));
SELECT create_distributed_table('referenced_table', 'id', 'hash');
CREATE TABLE referencing_table(id int, ref_id int);
SELECT create_distributed_table('referencing_table', 'ref_id', 'hash', colocate_with => 'none');
ALTER TABLE referencing_table ADD CONSTRAINT test_constraint FOREIGN KEY(ref_id) REFERENCES referenced_table(id);
-- create co-located tables
@ -468,15 +502,11 @@ SELECT create_distributed_table('referenced_by_reference_table', 'id');
CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_by_reference_table(id));
SELECT create_reference_table('reference_table');
-- test foreign key creation on CREATE TABLE to reference table
-- test foreign key creation on CREATE TABLE from + to reference table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
SELECT create_reference_table('reference_table');
CREATE TABLE references_to_reference_table(id int, referencing_column int REFERENCES reference_table(id));
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');
-- test foreign key creation on CREATE TABLE from + to reference table
CREATE TABLE reference_table_second(id int, referencing_column int REFERENCES reference_table(id));
SELECT create_reference_table('reference_table_second');
@ -503,7 +533,6 @@ SELECT create_reference_table('reference_table');
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_by_reference_table(id);
-- test foreign key creation on ALTER TABLE to reference table
DROP TABLE references_to_reference_table;
CREATE TABLE references_to_reference_table(id int, referencing_column int);
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');
ALTER TABLE references_to_reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES reference_table(id);
@ -515,7 +544,7 @@ SELECT create_reference_table('reference_table_second');
ALTER TABLE reference_table_second ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES reference_table(id);
-- test foreign key creation on ALTER TABLE from reference table to local table
DROP TABLE reference_table;
DROP TABLE reference_table CASCADE;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
SELECT create_reference_table('reference_table');
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_local_table(id);

View File

@ -0,0 +1,224 @@
SET citus.next_shard_id TO 3000000;
SET citus.shard_replication_factor TO 1;
CREATE SCHEMA fkey_graph;
SET search_path TO 'fkey_graph';
CREATE FUNCTION get_referencing_relation_id_list(Oid)
RETURNS SETOF Oid
LANGUAGE C STABLE STRICT
AS 'citus', $$get_referencing_relation_id_list$$;
CREATE FUNCTION get_referenced_relation_id_list(Oid)
RETURNS SETOF Oid
LANGUAGE C STABLE STRICT
AS 'citus', $$get_referenced_relation_id_list$$;
-- Simple case with distributed tables
CREATE TABLE dtt1(id int PRIMARY KEY);
SELECT create_distributed_table('dtt1','id');
CREATE TABLE dtt2(id int PRIMARY KEY REFERENCES dtt1(id));
SELECT create_distributed_table('dtt2','id');
CREATE TABLE dtt3(id int PRIMARY KEY REFERENCES dtt2(id));
SELECT create_distributed_table('dtt3','id');
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt1'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt2'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt3'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt1'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt2'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt3'::regclass) ORDER BY 1;
CREATE TABLE dtt4(id int PRIMARY KEY);
SELECT create_distributed_table('dtt4', 'id');
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt4'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt4'::regclass) ORDER BY 1;
ALTER TABLE dtt4 ADD CONSTRAINT dtt4_fkey FOREIGN KEY (id) REFERENCES dtt3(id);
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt4'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt4'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt1'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt2'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt3'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt1'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt2'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt3'::regclass) ORDER BY 1;
ALTER TABLE dtt4 DROP CONSTRAINT dtt4_fkey;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt3'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt3'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('dtt4'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('dtt4'::regclass) ORDER BY 1;
-- some tests within transction blocks to make sure that
-- cache invalidation works fine
CREATE TABLE test_1 (id int UNIQUE);
CREATE TABLE test_2 (id int UNIQUE);
CREATE TABLE test_3 (id int UNIQUE);
CREATE TABLE test_4 (id int UNIQUE);
CREATE TABLE test_5 (id int UNIQUE);
SELECT create_distributed_Table('test_1', 'id');
SELECT create_distributed_Table('test_2', 'id');
SELECT create_distributed_Table('test_3', 'id');
SELECT create_distributed_Table('test_4', 'id');
SELECT create_distributed_Table('test_5', 'id');
CREATE VIEW referential_integrity_summary AS
WITH RECURSIVE referential_integrity_summary(n, table_name, referencing_relations, referenced_relations) AS
(
SELECT 0,'0','{}'::regclass[],'{}'::regclass[]
UNION ALL
SELECT
n + 1,
'test_' || n + 1|| '' as table_name,
(SELECT array_agg(get_referencing_relation_id_list::regclass ORDER BY 1) FROM get_referencing_relation_id_list(('test_' || (n +1) ) ::regclass)) as referencing_relations,
(SELECT array_agg(get_referenced_relation_id_list::regclass ORDER BY 1) FROM get_referenced_relation_id_list(('test_' || (n +1) ) ::regclass)) as referenced_by_relations
FROM referential_integrity_summary, pg_class
WHERE
pg_class.relname = ('test_' || (n +1))
AND n < 5
)
SELECT * FROM referential_integrity_summary WHERE n != 0 ORDER BY 1;
-- make sure that invalidation through ALTER TABLE works fine
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
SELECT * FROM referential_integrity_summary;
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
SELECT * FROM referential_integrity_summary;
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
SELECT * FROM referential_integrity_summary;
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
ROLLBACK;
-- similar test, but slightly different order of creating foreign keys
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
SELECT * FROM referential_integrity_summary;
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
SELECT * FROM referential_integrity_summary;
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
SELECT * FROM referential_integrity_summary;
ROLLBACK;
-- make sure that DROP CONSTRAINT works invalidates the cache correctly
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
ALTER TABLE test_3 DROP CONSTRAINT fkey_1;
SELECT * FROM referential_integrity_summary;
ROLLBACK;
-- make sure that CREATE TABLE invalidates the cache correctly
DROP TABLE test_1, test_2, test_3, test_4, test_5 CASCADE;
BEGIN;
CREATE TABLE test_1 (id int UNIQUE);
SELECT create_distributed_Table('test_1', 'id');
CREATE TABLE test_2 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_1(id));
SELECT create_distributed_Table('test_2', 'id');
SELECT * FROM referential_integrity_summary;
CREATE TABLE test_3 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_2(id));
SELECT create_distributed_Table('test_3', 'id');
SELECT * FROM referential_integrity_summary;
CREATE TABLE test_4 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_3(id));
SELECT create_distributed_Table('test_4', 'id');
SELECT * FROM referential_integrity_summary;
CREATE TABLE test_5 (id int UNIQUE, FOREIGN KEY(id) REFERENCES test_4(id));
SELECT create_distributed_Table('test_5', 'id');
SELECT * FROM referential_integrity_summary;
COMMIT;
-- DROP TABLE works expected
-- re-create the constraints
BEGIN;
ALTER TABLE test_2 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_1(id);
ALTER TABLE test_3 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_2(id);
ALTER TABLE test_4 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_3(id);
ALTER TABLE test_5 ADD CONSTRAINT fkey_1 FOREIGN KEY(id) REFERENCES test_4(id);
SELECT * FROM referential_integrity_summary;
DROP TABLE test_3 CASCADE;
SELECT * FROM referential_integrity_summary;
ROLLBACK;
-- Test schemas
BEGIN;
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;
CREATE TABLE fkey_intermediate_schema_1.test_6(id int PRIMARY KEY);
SELECT create_distributed_table('fkey_intermediate_schema_1.test_6', 'id');
CREATE TABLE fkey_intermediate_schema_2.test_7(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_1.test_6(id));
SELECT create_distributed_table('fkey_intermediate_schema_2.test_7','id');
CREATE TABLE fkey_intermediate_schema_1.test_8(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_2.test_7(id));
SELECT create_distributed_table('fkey_intermediate_schema_1.test_8', 'id');
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_6'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_7'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_8'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_6'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_7'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_8'::regclass) ORDER BY 1;
DROP SCHEMA fkey_intermediate_schema_2 CASCADE;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_6'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_8'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_6'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_8'::regclass) ORDER BY 1;
ROLLBACK;
BEGIN;
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;
CREATE TABLE fkey_intermediate_schema_1.test_6(id int PRIMARY KEY);
SELECT create_distributed_table('fkey_intermediate_schema_1.test_6', 'id');
CREATE TABLE fkey_intermediate_schema_2.test_7(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_1.test_6(id));
SELECT create_distributed_table('fkey_intermediate_schema_2.test_7','id');
CREATE TABLE fkey_intermediate_schema_1.test_8(id int PRIMARY KEY REFERENCES fkey_intermediate_schema_2.test_7(id));
SELECT create_distributed_table('fkey_intermediate_schema_1.test_8', 'id');
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_6'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_7'::regclass) ORDER BY 1;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_8'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_6'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_7'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_8'::regclass) ORDER BY 1;
DROP SCHEMA fkey_intermediate_schema_1 CASCADE;
SELECT get_referencing_relation_id_list::regclass FROM get_referencing_relation_id_list('test_7'::regclass) ORDER BY 1;
SELECT get_referenced_relation_id_list::regclass FROM get_referenced_relation_id_list('test_7'::regclass) ORDER BY 1;
ROLLBACK;
SET search_path TO public;
DROP SCHEMA fkey_graph CASCADE;

View File

@ -0,0 +1,45 @@
CREATE SCHEMA fkey_reference_table;
SET search_path TO 'fkey_reference_table';
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 8;
SET citus.next_shard_id TO 7000000;
SET citus.next_placement_id TO 7000000;
SET citus.replication_model TO streaming;
-- Setup the view so that we can check if the foreign keys are created properly
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$);
CREATE VIEW table_fkeys_in_workers AS
SELECT
(json_populate_record(NULL::foreign_details,
json_array_elements_text((run_command_on_workers( $$
SELECT
COALESCE(json_agg(row_to_json(d)), '[]'::json)
FROM
(
SELECT
distinct name,
relid::regclass::text,
refd_relid::regclass::text
FROM
table_fkey_cols
)
d $$ )).RESULT::json )::json )).* ;
-- Check if MX can create foreign keys properly on foreign keys from distributed to reference tables
CREATE TABLE referenced_table(test_column int, test_column2 int, PRIMARY KEY(test_column));
CREATE TABLE referenced_table2(test_column int, test_column2 int, PRIMARY KEY(test_column2));
CREATE TABLE referencing_table(id int, ref_id int);
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE;
ALTER TABLE referencing_table ADD CONSTRAINT foreign_key_2 FOREIGN KEY (id) REFERENCES referenced_table2(test_column2) ON DELETE CASCADE;
SELECT create_reference_table('referenced_table');
SELECT create_reference_table('referenced_table2');
SELECT create_distributed_table('referencing_table', 'id');
SET search_path TO 'fkey_reference_table';
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_reference_table.%' AND refd_relid LIKE 'fkey_reference_table.%' ORDER BY 1, 2;
DROP SCHEMA fkey_reference_table CASCADE;
SET search_path TO DEFAULT;

View File

@ -348,7 +348,7 @@ ROLLBACK;
-- reference table join with a distributed table
BEGIN;
SELECT count(*) FROM table_1 JOIN table_6 USING(key);
SELECT * FROM relation_acesses WHERE table_name IN ('table_6', 'table_1');
SELECT * FROM relation_acesses WHERE table_name IN ('table_6', 'table_1') ORDER BY 1,2;
ROLLBACK;
-- TRUNCATE should be DDL