Support FKs between reference tables

pull/2864/head
Hadi Moshayedi 2019-08-09 18:09:55 -07:00
parent 3de851d3c5
commit a5b087c89b
22 changed files with 3697 additions and 257 deletions

View File

@ -35,14 +35,19 @@
static bool HeapTupleOfForeignConstraintIncludesColumn(HeapTuple heapTuple, Oid
relationId, int pgConstraintKey,
char *columnName);
static void ForeignConstraintFindDistKeys(HeapTuple pgConstraintTuple,
Var *referencingDistColumn,
Var *referencedDistColumn,
int *referencingAttrIndex,
int *referencedAttrIndex);
/*
* 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.
* ConstraintIsAForeignKeyToReferenceTable checks if the given constraint is a
* foreign key constraint from the given relation to a reference table. It does
* that by scanning pg_constraint for foreign key constraints.
*/
bool
ConstraintIsAForeignKeyToReferenceTable(char *constraintNameInput, Oid relationId)
ConstraintIsAForeignKeyToReferenceTable(char *constraintName, Oid relationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
@ -64,9 +69,9 @@ ConstraintIsAForeignKeyToReferenceTable(char *constraintNameInput, Oid relationI
{
Oid referencedTableId = InvalidOid;
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
char *constraintName = (constraintForm->conname).data;
char *tupleConstraintName = (constraintForm->conname).data;
if (strncmp(constraintNameInput, constraintName, NAMEDATALEN) != 0 ||
if (strncmp(constraintName, tupleConstraintName, NAMEDATALEN) != 0 ||
constraintForm->conrelid != relationId)
{
heapTuple = systable_getnext(scanDescriptor);
@ -95,7 +100,7 @@ ConstraintIsAForeignKeyToReferenceTable(char *constraintNameInput, Oid relationI
/*
* ErrorIfUnsupportedForeignConstraint runs checks related to foreign constraints and
* ErrorIfUnsupportedForeignConstraintExists runs checks related to foreign constraints and
* errors out if it is not possible to create one of the foreign constraint in distributed
* environment.
*
@ -109,11 +114,13 @@ ConstraintIsAForeignKeyToReferenceTable(char *constraintNameInput, Oid relationI
* - 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
* - If referencing table is a reference table, error out if the referenced table is not a
* a reference table.
*/
void
ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
Var *distributionColumn, uint32 colocationId)
ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDistMethod,
Var *referencingDistKey,
uint32 referencingColocationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
@ -123,21 +130,20 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
Oid referencingTableId = relation->rd_id;
Oid referencedTableId = InvalidOid;
uint32 referencedTableColocationId = INVALID_COLOCATION_ID;
Var *referencedTablePartitionColumn = NULL;
Datum referencingColumnsDatum = 0;
Datum *referencingColumnArray = NULL;
int referencingColumnCount = 0;
Datum referencedColumnsDatum = 0;
Datum *referencedColumnArray = NULL;
int referencedColumnCount = 0;
bool isNull = false;
int attrIdx = 0;
bool foreignConstraintOnPartitionColumn = false;
uint32 referencedColocationId = INVALID_COLOCATION_ID;
bool selfReferencingTable = false;
bool referencedTableIsAReferenceTable = false;
bool referencingColumnsIncludeDistKey = false;
bool referencingNotReplicated = true;
if (IsDistributedTable(referencingTableId))
{
/* ALTER TABLE command is applied over single replicated table */
referencingNotReplicated = SingleReplicatedTable(referencingTableId);
}
else
{
/* Creating single replicated table with foreign constraint */
referencingNotReplicated = (ShardReplicationFactor == 1);
}
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
@ -149,7 +155,15 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
bool singleReplicatedTable = true;
bool referencedIsDistributed = false;
char referencedDistMethod = 0;
Var *referencedDistKey = NULL;
bool referencingIsReferenceTable = false;
bool referencedIsReferenceTable = false;
int referencingAttrIndex = -1;
int referencedAttrIndex = -1;
bool referencingColumnsIncludeDistKey = false;
bool foreignConstraintOnDistKey = false;
if (constraintForm->contype != CONSTRAINT_FOREIGN)
{
@ -157,124 +171,94 @@ 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)
referencedTableId = constraintForm->confrelid;
selfReferencingTable = (referencingTableId == referencedTableId);
referencedIsDistributed = IsDistributedTable(referencedTableId);
if (!referencedIsDistributed && !selfReferencingTable)
{
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")));
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("cannot create foreign key constraint"),
errdetail("Referenced table must be a distributed table"
" or a reference table.")));
}
referencedTableId = constraintForm->confrelid;
selfReferencingTable = referencingTableId == referencedTableId;
/*
* Some checks are not meaningful if foreign key references the table itself.
* Therefore we will skip those checks.
*/
if (!selfReferencingTable)
{
if (!IsDistributedTable(referencedTableId))
{
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("cannot create foreign key constraint"),
errdetail("Referenced table must be a distributed "
"table.")));
}
/*
* 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)
{
referencedTableIsAReferenceTable = true;
}
/*
* 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);
referencedDistMethod = PartitionMethod(referencedTableId);
referencedDistKey = (referencedDistMethod == DISTRIBUTE_BY_NONE) ?
NULL :
DistPartitionKey(referencedTableId);
referencedColocationId = TableColocationId(referencedTableId);
}
else
{
/*
* 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;
referencedDistMethod = referencingDistMethod;
referencedDistKey = referencingDistKey;
referencedColocationId = referencingColocationId;
}
referencingIsReferenceTable = (referencingDistMethod == DISTRIBUTE_BY_NONE);
referencedIsReferenceTable = (referencedDistMethod == DISTRIBUTE_BY_NONE);
/*
* We support foreign keys between reference tables. No more checks
* are necessary.
*/
if (referencingIsReferenceTable && referencedIsReferenceTable)
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
/*
* Column attributes are not available in Form_pg_constraint, therefore we need
* to find them in the system catalog. After finding them, we iterate over column
* attributes together because partition column must be at the same place in both
* referencing and referenced side of the foreign key constraint
* Foreign keys from reference tables to distributed tables are not
* supported.
*/
referencingColumnsDatum = SysCacheGetAttr(CONSTROID, heapTuple,
Anum_pg_constraint_conkey, &isNull);
referencedColumnsDatum = SysCacheGetAttr(CONSTROID, heapTuple,
Anum_pg_constraint_confkey, &isNull);
deconstruct_array(DatumGetArrayTypeP(referencingColumnsDatum), INT2OID, 2, true,
's', &referencingColumnArray, NULL, &referencingColumnCount);
deconstruct_array(DatumGetArrayTypeP(referencedColumnsDatum), INT2OID, 2, true,
's', &referencedColumnArray, NULL, &referencedColumnCount);
Assert(referencingColumnCount == referencedColumnCount);
for (attrIdx = 0; attrIdx < referencingColumnCount; ++attrIdx)
if (referencingIsReferenceTable && !referencedIsReferenceTable)
{
AttrNumber referencingAttrNo = DatumGetInt16(referencingColumnArray[attrIdx]);
AttrNumber referencedAttrNo = DatumGetInt16(referencedColumnArray[attrIdx]);
if (distributionColumn->varattno == referencingAttrNo &&
(!referencedTableIsAReferenceTable &&
referencedTablePartitionColumn->varattno == referencedAttrNo))
{
foreignConstraintOnPartitionColumn = true;
}
if (distributionColumn->varattno == referencingAttrNo)
{
referencingColumnsIncludeDistKey = true;
}
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint "
"since foreign keys from reference tables "
"to distributed tables are not supported"),
errdetail("A reference table can only have reference "
"keys to other reference tables")));
}
/*
* To enforce foreign constraints, tables must be co-located unless a
* reference table is referenced.
*/
if (referencingColocationId == INVALID_COLOCATION_ID ||
(referencingColocationId != referencedColocationId &&
!referencedIsReferenceTable))
{
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")));
}
ForeignConstraintFindDistKeys(heapTuple,
referencingDistKey,
referencedDistKey,
&referencingAttrIndex,
&referencedAttrIndex);
referencingColumnsIncludeDistKey = (referencingAttrIndex != -1);
foreignConstraintOnDistKey =
(referencingColumnsIncludeDistKey && referencingAttrIndex ==
referencedAttrIndex);
/*
* 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)
{
/*
@ -314,7 +298,7 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
* if tables are hash-distributed and colocated, we need to make sure that
* the distribution key is included in foreign constraint.
*/
if (!referencedTableIsAReferenceTable && !foreignConstraintOnPartitionColumn)
if (!referencedIsReferenceTable && !foreignConstraintOnDistKey)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
@ -334,26 +318,7 @@ ErrorIfUnsupportedForeignConstraint(Relation relation, char distributionMethod,
* placements always be in the same state (b) executors are aware of reference
* tables and handle concurrency related issues accordingly.
*/
if (IsDistributedTable(referencingTableId))
{
/* check whether ALTER TABLE command is applied over single replicated table */
if (!SingleReplicatedTable(referencingTableId))
{
singleReplicatedTable = false;
}
}
else
{
Assert(distributionMethod == DISTRIBUTE_BY_HASH);
/* check whether creating single replicated table with foreign constraint */
if (ShardReplicationFactor > 1)
{
singleReplicatedTable = false;
}
}
if (!singleReplicatedTable)
if (!referencingNotReplicated)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
@ -376,10 +341,72 @@ 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.
* ForeignConstraintFindDistKeys finds the index of the given distribution columns
* in the given foreig key constraint and returns them in referencingAttrIndex
* and referencedAttrIndex. If one of them is not found, it returns -1 instead.
*/
static void
ForeignConstraintFindDistKeys(HeapTuple pgConstraintTuple,
Var *referencingDistColumn,
Var *referencedDistColumn,
int *referencingAttrIndex,
int *referencedAttrIndex)
{
Datum referencingColumnsDatum = 0;
Datum *referencingColumnArray = NULL;
int referencingColumnCount = 0;
Datum referencedColumnsDatum = 0;
Datum *referencedColumnArray = NULL;
int referencedColumnCount = 0;
bool isNull = false;
int attrIdx = 0;
*referencedAttrIndex = -1;
*referencedAttrIndex = -1;
/*
* Column attributes are not available in Form_pg_constraint, therefore we need
* to find them in the system catalog. After finding them, we iterate over column
* attributes together because partition column must be at the same place in both
* referencing and referenced side of the foreign key constraint.
*/
referencingColumnsDatum = SysCacheGetAttr(CONSTROID, pgConstraintTuple,
Anum_pg_constraint_conkey, &isNull);
referencedColumnsDatum = SysCacheGetAttr(CONSTROID, pgConstraintTuple,
Anum_pg_constraint_confkey, &isNull);
deconstruct_array(DatumGetArrayTypeP(referencingColumnsDatum), INT2OID, 2, true,
's', &referencingColumnArray, NULL, &referencingColumnCount);
deconstruct_array(DatumGetArrayTypeP(referencedColumnsDatum), INT2OID, 2, true,
's', &referencedColumnArray, NULL, &referencedColumnCount);
Assert(referencingColumnCount == referencedColumnCount);
for (attrIdx = 0; attrIdx < referencingColumnCount; ++attrIdx)
{
AttrNumber referencingAttrNo = DatumGetInt16(referencingColumnArray[attrIdx]);
AttrNumber referencedAttrNo = DatumGetInt16(referencedColumnArray[attrIdx]);
if (referencedDistColumn != NULL &&
referencedDistColumn->varattno == referencedAttrNo)
{
*referencedAttrIndex = attrIdx;
}
if (referencingDistColumn != NULL &&
referencingDistColumn->varattno == referencingAttrNo)
{
*referencingAttrIndex = attrIdx;
}
}
}
/*
* ColumnAppearsInForeignKeyToReferenceTable checks if there is a foreign key
* constraint from/to a reference table on the given column. We iterate
* pg_constraint 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)
@ -582,9 +609,9 @@ HasForeignKeyToReferenceTable(Oid relationId)
/*
* TableReferenced function checks whether given table is referenced by another table
* via foreign constraints. If it is referenced, 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.
* that, this function searches for the given relation in the pg_constraint system
* catalog table. However since there are no indexes for the column we search for,
* this function performs sequential search. So call this function with caution.
*/
bool
TableReferenced(Oid relationId)

View File

@ -763,8 +763,8 @@ ErrorUnsupportedAlterTableAddColumn(Oid relationId, AlterTableCmd *command,
}
/*ErrorIfUnsupportedConstraint
* run checks related to unique index / exclude
/*
* ErrorIfUnsupportedConstraint runs checks related to unique index / exclude
* constraints.
*
* The function skips the uniqeness checks for reference tables (i.e., distribution
@ -796,8 +796,9 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
* for reference tables, we first check for foreing constraints and if they are OK,
* we do not error out for other types of constraints.
*/
ErrorIfUnsupportedForeignConstraint(relation, distributionMethod, distributionColumn,
colocationId);
ErrorIfUnsupportedForeignConstraintExists(relation, distributionMethod,
distributionColumn,
colocationId);
/*
* Citus supports any kind of uniqueness constraints for reference tables

View File

@ -1456,6 +1456,14 @@ CitusHasBeenLoaded(void)
*/
DistPartitionRelationId();
/*
* This needs to be initialized so we can receive foreign relation graph
* invalidation messages in InvalidateForeignRelationGraphCacheCallback().
* See the comments of InvalidateForeignKeyGraph for more context.
*/
DistColocationRelationId();
/*
* We also reset citusVersionKnownCompatible, so it will be re-read in
* case of extension update.
@ -3089,7 +3097,6 @@ ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
static void
InvalidateForeignRelationGraphCacheCallback(Datum argument, Oid relationId)
{
/* when invalidation happens simply set the LocalGroupId to the default value */
if (relationId == MetadataCache.distColocationRelationId)
{
SetForeignConstraintRelationshipGraphInvalid();
@ -3164,7 +3171,7 @@ InvalidateDistRelationCacheCallback(Datum argument, Oid relationId)
* InvalidateEntireDistCache makes entire cache entries invalid.
*/
static void
InvalidateEntireDistCache()
InvalidateEntireDistCache(void)
{
DistTableCacheEntry *cacheEntry = NULL;
HASH_SEQ_STATUS status;

View File

@ -168,6 +168,18 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
ReplicateShardToNode(shardInterval, nodeName, nodePort);
}
/* create foreign constraints between reference tables */
foreach(referenceShardIntervalCell, referenceShardIntervalList)
{
ShardInterval *shardInterval =
(ShardInterval *) lfirst(referenceShardIntervalCell);
char *tableOwner = TableOwner(shardInterval->relationId);
List *commandList = CopyShardForeignConstraintCommandList(shardInterval);
SendCommandListToWorkerInSingleTransaction(nodeName, nodePort,
tableOwner, commandList);
}
}
/*

View File

@ -69,6 +69,7 @@ static const int lock_mode_to_string_map_count = sizeof(lockmode_to_string_map)
/* local function forward declarations */
static LOCKMODE IntToLockMode(int mode);
static void LockReferencedReferenceShardResources(uint64 shardId, LOCKMODE lockMode);
static void LockShardListResources(List *shardIntervalList, LOCKMODE lockMode);
static void LockShardListResourcesOnFirstWorker(LOCKMODE lockmode,
List *shardIntervalList);
@ -318,14 +319,15 @@ LockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode)
/*
* LockReferencedReferenceShardDistributionMetadata acquires the given lock
* on the reference tables which has a foreign key from the given relation.
* LockReferencedReferenceShardDistributionMetadata acquires shard distribution
* metadata locks with the given lock mode on the reference tables which has a
* foreign key from the given relation.
*
* It also gets metadata locks on worker nodes to prevent concurrent write
* operations on reference tables from metadata nodes.
*/
void
LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lock)
LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lockMode)
{
ListCell *shardIntervalCell = NULL;
Oid relationId = RelationIdForShard(shardId);
@ -336,21 +338,61 @@ LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lock)
if (list_length(shardIntervalList) > 0 && ClusterHasKnownMetadataWorkers())
{
LockShardListMetadataOnWorkers(lock, shardIntervalList);
LockShardListMetadataOnWorkers(lockMode, shardIntervalList);
}
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
LockShardDistributionMetadata(shardInterval->shardId, lock);
LockShardDistributionMetadata(shardInterval->shardId, lockMode);
}
}
/*
* GetSortedReferenceShards iterates through the given relation list.
* Lists the shards of reference tables and returns the list after sorting.
* LockReferencedReferenceShardResources acquires resource locks with the
* given lock mode on the reference tables which has a foreign key from
* the given relation.
*
* It also gets resource locks on worker nodes to prevent concurrent write
* operations on reference tables from metadata nodes.
*/
static void
LockReferencedReferenceShardResources(uint64 shardId, LOCKMODE lockMode)
{
ListCell *shardIntervalCell = NULL;
Oid relationId = RelationIdForShard(shardId);
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
/*
* Note that referencedRelationsViaForeignKey contains transitively referenced
* relations too.
*/
List *referencedRelationList = cacheEntry->referencedRelationsViaForeignKey;
List *referencedShardIntervalList =
GetSortedReferenceShardIntervals(referencedRelationList);
if (list_length(referencedShardIntervalList) > 0 &&
ClusterHasKnownMetadataWorkers() &&
!IsFirstWorkerNode())
{
LockShardListResourcesOnFirstWorker(lockMode, referencedShardIntervalList);
}
foreach(shardIntervalCell, referencedShardIntervalList)
{
ShardInterval *referencedShardInterval = (ShardInterval *) lfirst(
shardIntervalCell);
LockShardResource(referencedShardInterval->shardId, lockMode);
}
}
/*
* GetSortedReferenceShardIntervals iterates through the given relation list,
* lists the shards of reference tables, and returns the list after sorting.
*/
List *
GetSortedReferenceShardIntervals(List *relationList)
@ -534,12 +576,21 @@ SerializeNonCommutativeWrites(List *shardIntervalList, LOCKMODE lockMode)
ShardInterval *firstShardInterval = (ShardInterval *) linitial(shardIntervalList);
int64 firstShardId = firstShardInterval->shardId;
if (ReferenceTableShardId(firstShardId) && ClusterHasKnownMetadataWorkers() &&
!IsFirstWorkerNode())
if (ReferenceTableShardId(firstShardId))
{
LockShardListResourcesOnFirstWorker(lockMode, shardIntervalList);
if (ClusterHasKnownMetadataWorkers() && !IsFirstWorkerNode())
{
LockShardListResourcesOnFirstWorker(lockMode, shardIntervalList);
}
/*
* Referenced tables can cascade their changes to this table, and we
* want to serialize changes to keep different replicas consistent.
*/
LockReferencedReferenceShardResources(firstShardId, lockMode);
}
LockShardListResources(shardIntervalList, lockMode);
}

View File

@ -31,10 +31,10 @@ extern void ErrorIfUnstableCreateOrAlterExtensionStmt(Node *parsetree);
/* foreign_constraint.c - forward declarations */
extern bool ConstraintIsAForeignKeyToReferenceTable(char *constraintName,
Oid leftRelationId);
extern void ErrorIfUnsupportedForeignConstraint(Relation relation, char
distributionMethod,
Var *distributionColumn, uint32
colocationId);
extern void ErrorIfUnsupportedForeignConstraintExists(Relation relation, char
distributionMethod,
Var *distributionColumn, uint32
colocationId);
extern bool ColumnAppearsInForeignKeyToReferenceTable(char *columnName, Oid
relationId);
extern List * GetTableForeignConstraintCommands(Oid relationId);

View File

@ -71,8 +71,8 @@ extern void LockShardListMetadataOnWorkers(LOCKMODE lockmode, List *shardInterva
extern void BlockWritesToShardList(List *shardList);
/* Lock shard/relation metadata of the referenced reference table if exists */
extern void LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE
lock);
extern void LockReferencedReferenceShardDistributionMetadata(uint64 shardId,
LOCKMODE lock);
/* Lock shard data, for DML commands or remote fetches */
extern void LockShardResource(uint64 shardId, LOCKMODE lockmode);

View File

@ -17,6 +17,7 @@ s/assigned task [0-9]+ to node/assigned task to node/
# the generated plan
s/"(foreign_key_2_|fkey_ref_to_dist_|fkey_ref_)[0-9]+"/"\1xxxxxxx"/g
s/"(referenced_table_|referencing_table_|referencing_table2_)[0-9]+"/"\1xxxxxxx"/g
s/"(referencing_table_0_|referenced_table2_)[0-9]+"/"\1xxxxxxx"/g
s/\(id\)=\([0-9]+\)/(id)=(X)/g
s/\(ref_id\)=\([0-9]+\)/(ref_id)=(X)/g
@ -65,3 +66,6 @@ s/ERROR: failed to execute task [0-9]+/ERROR: failed to execute task X/g
# normalize file names for partitioned files
s/(task_[0-9]+\.)[0-9]+/\1xxxx/g
s/(job_[0-9]+\/task_[0-9]+\/p_[0-9]+\.)[0-9]+/\1xxxx/g
# isolation_ref2ref_foreign_keys
s/"(ref_table_[0-9]_|ref_table_[0-9]_value_fkey_)[0-9]+"/"\1xxxxxxx"/g

View File

@ -8,6 +8,7 @@ foreign_key_restriction_enforcement
failure_real_time_select
failure_vacuum
isolation_citus_dist_activity
isolation_ref2ref_foreign_keys
multi_insert_select
multi_insert_select_conflict
multi_multiuser

View File

@ -15,7 +15,14 @@ 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 reference_table(id int PRIMARY KEY);
CREATE TABLE transitive_reference_table(id int PRIMARY KEY);
SELECT create_reference_table('transitive_reference_table');
create_reference_table
------------------------
(1 row)
CREATE TABLE reference_table(id int PRIMARY KEY, value_1 int);
SELECT create_reference_table('reference_table');
create_reference_table
------------------------
@ -37,7 +44,9 @@ SELECT create_distributed_table('unrelated_dist_table', 'id');
(1 row)
ALTER TABLE on_update_fkey_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES reference_table(id) ON UPDATE CASCADE;
INSERT INTO reference_table SELECT i FROM generate_series(0, 100) i;
ALTER TABLE reference_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES transitive_reference_table(id) ON UPDATE CASCADE;
INSERT INTO transitive_reference_table SELECT i FROM generate_series(0, 100) i;
INSERT INTO reference_table SELECT i, 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
@ -56,6 +65,20 @@ BEGIN;
1001
(1 row)
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
SELECT count(*) FROM on_update_fkey_table;
count
-------
1001
(1 row)
ROLLBACK;
-- case 1.2: SELECT to a reference table is followed by a multiple router SELECTs to a distributed table
BEGIN;
@ -89,10 +112,41 @@ BEGIN;
1
(1 row)
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 15;
count
-------
1
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 16;
count
-------
1
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 17;
count
-------
1
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 18;
count
-------
1
(1 row)
ROLLBACK;
-- case 1.3: SELECT to a reference table is followed by a multi-shard UPDATE to a distributed table
BEGIN;
BEGIN;
SELECT count(*) FROM reference_table;
count
-------
@ -101,8 +155,17 @@ BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
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;
BEGIN;
SELECT count(*) FROM reference_table;
count
-------
@ -114,6 +177,18 @@ BEGIN;
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;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
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 reference_table;
@ -127,6 +202,18 @@ DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
-- case 1.6: SELECT to a reference table is followed by an unrelated DDL
BEGIN;
SELECT count(*) FROM reference_table;
@ -139,6 +226,17 @@ BEGIN;
DEBUG: switching to sequential query execution mode
DETAIL: cannot execute parallel DDL on relation "on_update_fkey_table" after SELECT command on reference relation "reference_table" because there is a foreign key between them and "reference_table" has been accessed in this transaction
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
DEBUG: switching to sequential query execution mode
DETAIL: cannot execute parallel DDL on relation "on_update_fkey_table" after SELECT command on reference relation "transitive_reference_table" because there is a foreign key between them and "transitive_reference_table" has been accessed in this transaction
ROLLBACK;
-- case 1.7.1: SELECT to a reference table is followed by a DDL that is on
-- the foreign key column
BEGIN;
@ -152,6 +250,17 @@ BEGIN;
SET LOCAL client_min_messages TO ERROR;
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
-- 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;
@ -172,6 +281,24 @@ ERROR: cannot modify table "on_update_fkey_table" because there was a parallel
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
count
-------
1001
(1 row)
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ERROR: cannot modify table "on_update_fkey_table" because there was a parallel operation on a distributed table in the transaction
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -192,6 +319,24 @@ ERROR: cannot execute parallel DDL on relation "on_update_fkey_table" after SEL
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
count
-------
1001
(1 row)
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
ERROR: cannot execute parallel DDL on relation "on_update_fkey_table" after SELECT command on reference relation "transitive_reference_table" because there is a foreign key between them and "transitive_reference_table" has been accessed in this transaction
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
-- case 1.8: SELECT to a reference table is followed by a COPY
BEGIN;
SELECT count(*) FROM reference_table;
@ -202,6 +347,15 @@ BEGIN;
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
-- case 2.1: UPDATE to a reference table is followed by a multi-shard SELECT
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -219,6 +373,23 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
10
(1 row)
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101;
count
-------
0
(1 row)
ROLLBACK;
-- case 2.2: UPDATE to a reference table is followed by multiple router SELECT
BEGIN;
@ -249,6 +420,35 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
1
(1 row)
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 99;
count
-------
0
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 199;
count
-------
0
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 299;
count
-------
0
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 399;
count
-------
0
(1 row)
ROLLBACK;
-- case 2.3: UPDATE to a reference table is followed by a multi-shard UPDATE
BEGIN;
@ -257,6 +457,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
UPDATE on_update_fkey_table SET value_1 = 15;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table SET id = 101 WHERE id = 99;
@ -267,6 +473,21 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
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;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 1;
ERROR: insert or update on table "on_update_fkey_table_2380002" violates foreign key constraint "fkey_2380002"
DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001".
CONTEXT: while executing command on localhost:57637
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 2;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 4;
ERROR: current transaction is aborted, commands ignored until end of transaction block
ROLLBACK;
-- case 2.5: UPDATE to a reference table is followed by a DDL that touches fkey column
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -277,6 +498,15 @@ DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
-- case 2.6: UPDATE to a reference table is followed by an unrelated DDL
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -284,6 +514,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table SET id = 101 WHERE id = 99;
@ -291,6 +527,14 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ERROR: insert or update on table "on_update_fkey_table_2380005" violates foreign key constraint "fkey_2380005"
DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001".
ROLLBACK;
-- case 2.8: UPDATE to a reference table is followed by TRUNCATE
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -299,6 +543,13 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
-- case 3.1: an unrelated DDL to a reference table is followed by a real-time SELECT
BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DEFAULT 1001;
@ -310,6 +561,17 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
1001
(1 row)
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
SELECT count(*) FROM on_update_fkey_table;
count
-------
1001
(1 row)
ROLLBACK;
-- case 3.2: DDL that touches fkey column to a reference table is followed by a real-time SELECT
BEGIN;
@ -320,6 +582,15 @@ BEGIN;
1001
(1 row)
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE int;
SELECT count(*) FROM on_update_fkey_table;
count
-------
1001
(1 row)
ROLLBACK;
-- case 3.3: DDL to a reference table followed by a multi shard UPDATE
BEGIN;
@ -328,6 +599,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table ALTER COLUMN id SET DEFAULT 1001;
@ -338,6 +615,15 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
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;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table ALTER COLUMN id SET DATA TYPE smallint;
@ -347,6 +633,14 @@ DEBUG: validating foreign key constraint "fkey"
CREATE INDEX fkey_test_index_1 ON on_update_fkey_table(value_1);
DEBUG: building index "fkey_test_index_1" on table "on_update_fkey_table" serially
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table" serially
DEBUG: validating foreign key constraint "fkey"
CREATE INDEX fkey_test_index_1 ON on_update_fkey_table(value_1);
DEBUG: building index "fkey_test_index_1" on table "on_update_fkey_table" serially
ROLLBACK;
-- case 4.6: DDL to reference table followed by a DDL to dist table, both touching fkey columns
BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint;
@ -358,6 +652,16 @@ DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table" serially
DEBUG: validating foreign key constraint "fkey"
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
-- case 3.7: DDL to a reference table is followed by COPY
BEGIN;
ALTER TABLE reference_table ADD COLUMN X int;
@ -365,6 +669,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
-- case 3.8: DDL to a reference table is followed by TRUNCATE
BEGIN;
ALTER TABLE reference_table ADD COLUMN X int;
@ -373,6 +683,13 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
-- case 3.9: DDL to a reference table is followed by TRUNCATE
BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint;
@ -382,6 +699,14 @@ DEBUG: validating foreign key constraint "fkey"
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table" serially
DEBUG: validating foreign key constraint "fkey"
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
-----
--- Now, start testing the other way araound
-----
@ -399,6 +724,20 @@ BEGIN;
101
(1 row)
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ROLLBACK;
-- case 4.2: SELECT to a dist table is follwed by a DML to a reference table
BEGIN;
@ -413,6 +752,18 @@ ERROR: cannot modify reference table "reference_table" because there was a para
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify reference table "transitive_reference_table" because there was a parallel operation on a distributed table
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -425,6 +776,17 @@ BEGIN;
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
ALTER TABLE transitive_reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -440,6 +802,20 @@ DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table" serially
DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -455,6 +831,21 @@ NOTICE: truncate cascades to table "on_update_fkey_table"
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
TRUNCATE transitive_reference_table CASCADE;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
NOTICE: truncate cascades to table "reference_table"
NOTICE: truncate cascades to table "on_update_fkey_table"
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -467,14 +858,50 @@ BEGIN;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
NOTICE: truncate cascades to table "on_update_fkey_table"
DEBUG: truncate cascades to table "on_update_fkey_table_2380003"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380005"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380002"
DETAIL: NOTICE from localhost:57638
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380004"
DETAIL: NOTICE from localhost:57637
DEBUG: building index "reference_table_pkey" on table "reference_table" serially
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE id = 9;
count
-------
1
(1 row)
TRUNCATE transitive_reference_table CASCADE;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
NOTICE: truncate cascades to table "reference_table"
NOTICE: truncate cascades to table "on_update_fkey_table"
DEBUG: truncate cascades to table "reference_table_2380001"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380001"
DEBUG: truncate cascades to table "on_update_fkey_table_2380003"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380005"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "reference_table_2380001"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380002"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380004"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380002"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380003"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380004"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380005"
DETAIL: NOTICE from localhost:57638
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table" serially
DEBUG: building index "reference_table_pkey" on table "reference_table" serially
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
ROLLBACK;
@ -487,6 +914,15 @@ BEGIN;
101
(1 row)
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ROLLBACK;
-- case 5.2: Parallel UPDATE on distributed table follow by a UPDATE
BEGIN;
@ -495,6 +931,12 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
UPDATE transitive_reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -502,6 +944,12 @@ BEGIN;
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
-- case 5.4: Parallel UPDATE on distributed table follow by a related DDL on reference table
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
@ -512,6 +960,15 @@ DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table" serially
DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -519,6 +976,12 @@ BEGIN;
ERROR: cannot execute SELECT on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_table;
ERROR: cannot execute SELECT on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -527,6 +990,13 @@ DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_tabl
DEBUG: validating foreign key constraint "fkey"
UPDATE reference_table SET id = 160 WHERE id = 15;
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table" serially
DEBUG: validating foreign key constraint "fkey"
UPDATE transitive_reference_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;
@ -534,6 +1004,12 @@ BEGIN;
ERROR: cannot execute SELECT on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_table;
ERROR: cannot execute SELECT on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -541,6 +1017,12 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
UPDATE transitive_reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -548,6 +1030,12 @@ BEGIN;
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
ALTER TABLE transitive_reference_table ADD COLUMN X int;
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -564,6 +1052,12 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
DELETE FROM transitive_reference_table WHERE id = 99;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
-- an unrelated update followed by update on dist table and update
-- on reference table
BEGIN;
@ -573,6 +1067,13 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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
@ -1076,8 +1577,8 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator
-- see https://github.com/citusdata/citus_docs/issues/664 for the discussion
WITH t1 AS (DELETE FROM reference_table RETURNING id)
DELETE FROM distributed_table USING t1 WHERE value_1 = t1.id RETURNING *;
DEBUG: generating subplan 92_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 92 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.distributed_table USING (SELECT intermediate_result.id FROM read_intermediate_result('92_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) RETURNING distributed_table.id, distributed_table.value_1, t1.id
DEBUG: generating subplan 170_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 170 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.distributed_table USING (SELECT intermediate_result.id FROM read_intermediate_result('170_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) RETURNING distributed_table.id, distributed_table.value_1, t1.id
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
id | value_1 | id
@ -1097,8 +1598,8 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator
-- see https://github.com/citusdata/citus_docs/issues/664 for the discussion
WITH t1 AS (DELETE FROM reference_table RETURNING id)
SELECT count(*) FROM distributed_table, t1 WHERE value_1 = t1.id;
DEBUG: generating subplan 96_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 96 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('96_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id)
DEBUG: generating subplan 174_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 174 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('174_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id)
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
count
@ -1111,17 +1612,17 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
WITH t1 AS (DELETE FROM distributed_table RETURNING id),
t2 AS (DELETE FROM reference_table RETURNING id)
SELECT count(*) FROM distributed_table, t1, t2 WHERE value_1 = t1.id AND value_1 = t2.id;
DEBUG: generating subplan 98_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 98_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 98 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('98_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('98_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
DEBUG: generating subplan 176_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 176_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 176 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('176_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('176_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "distributed_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
-- similarly this should fail since we first access to a distributed
-- table via t1, and then access to the reference table in the main query
WITH t1 AS (DELETE FROM distributed_table RETURNING id)
DELETE FROM reference_table RETURNING id;
DEBUG: generating subplan 101_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 101 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: generating subplan 179_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 179 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "distributed_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
-- finally, make sure that we can execute the same queries
@ -1132,9 +1633,9 @@ BEGIN;
WITH t1 AS (DELETE FROM distributed_table RETURNING id),
t2 AS (DELETE FROM reference_table RETURNING id)
SELECT count(*) FROM distributed_table, t1, t2 WHERE value_1 = t1.id AND value_1 = t2.id;
DEBUG: generating subplan 103_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 103_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 103 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('103_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('103_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
DEBUG: generating subplan 181_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 181_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 181 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('181_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('181_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
count
-------
0
@ -1146,8 +1647,8 @@ BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
WITH t1 AS (DELETE FROM distributed_table RETURNING id)
DELETE FROM reference_table RETURNING id;
DEBUG: generating subplan 106_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 106 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: generating subplan 184_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 184 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
id
----
(0 rows)
@ -1155,8 +1656,9 @@ DEBUG: Plan 106 query after replacing subqueries and CTEs: DELETE FROM test_fke
ROLLBACK;
RESET client_min_messages;
DROP SCHEMA test_fkey_to_ref_in_tx CASCADE;
NOTICE: drop cascades to 4 other objects
DETAIL: drop cascades to table on_update_fkey_table
NOTICE: drop cascades to 5 other objects
DETAIL: drop cascades to table transitive_reference_table
drop cascades to table on_update_fkey_table
drop cascades to table unrelated_dist_table
drop cascades to table reference_table
drop cascades to table distributed_table

View File

@ -15,7 +15,14 @@ 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 reference_table(id int PRIMARY KEY);
CREATE TABLE transitive_reference_table(id int PRIMARY KEY);
SELECT create_reference_table('transitive_reference_table');
create_reference_table
------------------------
(1 row)
CREATE TABLE reference_table(id int PRIMARY KEY, value_1 int);
SELECT create_reference_table('reference_table');
create_reference_table
------------------------
@ -37,7 +44,9 @@ SELECT create_distributed_table('unrelated_dist_table', 'id');
(1 row)
ALTER TABLE on_update_fkey_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES reference_table(id) ON UPDATE CASCADE;
INSERT INTO reference_table SELECT i FROM generate_series(0, 100) i;
ALTER TABLE reference_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES transitive_reference_table(id) ON UPDATE CASCADE;
INSERT INTO transitive_reference_table SELECT i FROM generate_series(0, 100) i;
INSERT INTO reference_table SELECT i, 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
@ -56,6 +65,20 @@ BEGIN;
1001
(1 row)
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
SELECT count(*) FROM on_update_fkey_table;
count
-------
1001
(1 row)
ROLLBACK;
-- case 1.2: SELECT to a reference table is followed by a multiple router SELECTs to a distributed table
BEGIN;
@ -89,10 +112,41 @@ BEGIN;
1
(1 row)
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 15;
count
-------
1
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 16;
count
-------
1
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 17;
count
-------
1
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE id = 18;
count
-------
1
(1 row)
ROLLBACK;
-- case 1.3: SELECT to a reference table is followed by a multi-shard UPDATE to a distributed table
BEGIN;
BEGIN;
SELECT count(*) FROM reference_table;
count
-------
@ -101,8 +155,17 @@ BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
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;
BEGIN;
SELECT count(*) FROM reference_table;
count
-------
@ -114,6 +177,18 @@ BEGIN;
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;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
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 reference_table;
@ -127,6 +202,18 @@ DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
-- case 1.6: SELECT to a reference table is followed by an unrelated DDL
BEGIN;
SELECT count(*) FROM reference_table;
@ -139,6 +226,17 @@ BEGIN;
DEBUG: switching to sequential query execution mode
DETAIL: cannot execute parallel DDL on relation "on_update_fkey_table" after SELECT command on reference relation "reference_table" because there is a foreign key between them and "reference_table" has been accessed in this transaction
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
DEBUG: switching to sequential query execution mode
DETAIL: cannot execute parallel DDL on relation "on_update_fkey_table" after SELECT command on reference relation "transitive_reference_table" because there is a foreign key between them and "transitive_reference_table" has been accessed in this transaction
ROLLBACK;
-- case 1.7.1: SELECT to a reference table is followed by a DDL that is on
-- the foreign key column
BEGIN;
@ -152,6 +250,17 @@ BEGIN;
SET LOCAL client_min_messages TO ERROR;
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
-- 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;
@ -172,6 +281,24 @@ ERROR: cannot modify table "on_update_fkey_table" because there was a parallel
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
count
-------
1001
(1 row)
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ERROR: cannot modify table "on_update_fkey_table" because there was a parallel operation on a distributed table in the transaction
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -192,6 +319,24 @@ ERROR: cannot execute parallel DDL on relation "on_update_fkey_table" after SEL
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
count
-------
1001
(1 row)
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
ERROR: cannot execute parallel DDL on relation "on_update_fkey_table" after SELECT command on reference relation "transitive_reference_table" because there is a foreign key between them and "transitive_reference_table" has been accessed in this transaction
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
-- case 1.8: SELECT to a reference table is followed by a COPY
BEGIN;
SELECT count(*) FROM reference_table;
@ -202,6 +347,15 @@ BEGIN;
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
-- case 2.1: UPDATE to a reference table is followed by a multi-shard SELECT
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -219,6 +373,23 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
10
(1 row)
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101;
count
-------
0
(1 row)
ROLLBACK;
-- case 2.2: UPDATE to a reference table is followed by multiple router SELECT
BEGIN;
@ -249,6 +420,35 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
1
(1 row)
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 99;
count
-------
0
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 199;
count
-------
0
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 299;
count
-------
0
(1 row)
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 399;
count
-------
0
(1 row)
ROLLBACK;
-- case 2.3: UPDATE to a reference table is followed by a multi-shard UPDATE
BEGIN;
@ -257,6 +457,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
UPDATE on_update_fkey_table SET value_1 = 15;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table SET id = 101 WHERE id = 99;
@ -267,6 +473,21 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
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;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 1;
ERROR: insert or update on table "on_update_fkey_table_2380002" violates foreign key constraint "fkey_2380002"
DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001".
CONTEXT: while executing command on localhost:57637
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 2;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 4;
ERROR: current transaction is aborted, commands ignored until end of transaction block
ROLLBACK;
-- case 2.5: UPDATE to a reference table is followed by a DDL that touches fkey column
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -277,6 +498,15 @@ DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
-- case 2.6: UPDATE to a reference table is followed by an unrelated DDL
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -284,6 +514,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table SET id = 101 WHERE id = 99;
@ -291,6 +527,14 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ERROR: insert or update on table "on_update_fkey_table_2380005" violates foreign key constraint "fkey_2380005"
DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001".
ROLLBACK;
-- case 2.8: UPDATE to a reference table is followed by TRUNCATE
BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
@ -299,6 +543,13 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "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 reference_table ALTER COLUMN id SET DEFAULT 1001;
@ -310,6 +561,17 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
1001
(1 row)
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
SELECT count(*) FROM on_update_fkey_table;
count
-------
1001
(1 row)
ROLLBACK;
-- case 3.2: DDL that touches fkey column to a reference table is followed by a real-time SELECT
BEGIN;
@ -320,6 +582,15 @@ BEGIN;
1001
(1 row)
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE int;
SELECT count(*) FROM on_update_fkey_table;
count
-------
1001
(1 row)
ROLLBACK;
-- case 3.3: DDL to a reference table followed by a multi shard UPDATE
BEGIN;
@ -328,6 +599,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table ALTER COLUMN id SET DEFAULT 1001;
@ -338,6 +615,15 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
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;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
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 reference_table ALTER COLUMN id SET DATA TYPE smallint;
@ -347,6 +633,14 @@ DEBUG: validating foreign key constraint "fkey"
CREATE INDEX fkey_test_index_1 ON on_update_fkey_table(value_1);
DEBUG: building index "fkey_test_index_1" on table "on_update_fkey_table"
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table"
DEBUG: validating foreign key constraint "fkey"
CREATE INDEX fkey_test_index_1 ON on_update_fkey_table(value_1);
DEBUG: building index "fkey_test_index_1" on table "on_update_fkey_table"
ROLLBACK;
-- case 4.6: DDL to reference table followed by a DDL to dist table, both touching fkey columns
BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint;
@ -358,6 +652,16 @@ DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table"
DEBUG: validating foreign key constraint "fkey"
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey"
ROLLBACK;
-- case 3.7: DDL to a reference table is followed by COPY
BEGIN;
ALTER TABLE reference_table ADD COLUMN X int;
@ -365,6 +669,12 @@ DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK;
-- case 3.8: DDL to a reference table is followed by TRUNCATE
BEGIN;
ALTER TABLE reference_table ADD COLUMN X int;
@ -373,6 +683,13 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
ROLLBACK;
-- case 3.9: DDL to a reference table is followed by TRUNCATE
BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint;
@ -382,6 +699,14 @@ DEBUG: validating foreign key constraint "fkey"
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table"
DEBUG: validating foreign key constraint "fkey"
TRUNCATE on_update_fkey_table;
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
ROLLBACK;
-----
--- Now, start testing the other way araound
-----
@ -399,6 +724,20 @@ BEGIN;
101
(1 row)
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ROLLBACK;
-- case 4.2: SELECT to a dist table is follwed by a DML to a reference table
BEGIN;
@ -413,6 +752,18 @@ ERROR: cannot modify reference table "reference_table" because there was a para
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify reference table "transitive_reference_table" because there was a parallel operation on a distributed table
DETAIL: 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.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -425,6 +776,17 @@ BEGIN;
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
ALTER TABLE transitive_reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -440,6 +802,20 @@ DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table"
DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -455,6 +831,21 @@ NOTICE: truncate cascades to table "on_update_fkey_table"
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count
-------
10
(1 row)
TRUNCATE transitive_reference_table CASCADE;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
NOTICE: truncate cascades to table "reference_table"
NOTICE: truncate cascades to table "on_update_fkey_table"
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel SELECT access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -467,14 +858,50 @@ BEGIN;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
NOTICE: truncate cascades to table "on_update_fkey_table"
DEBUG: truncate cascades to table "on_update_fkey_table_2380003"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380005"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380002"
DETAIL: NOTICE from localhost:57638
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380004"
DETAIL: NOTICE from localhost:57637
DEBUG: building index "reference_table_pkey" on table "reference_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE id = 9;
count
-------
1
(1 row)
TRUNCATE transitive_reference_table CASCADE;
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "transitive_reference_table" 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
NOTICE: truncate cascades to table "reference_table"
NOTICE: truncate cascades to table "on_update_fkey_table"
DEBUG: truncate cascades to table "reference_table_2380001"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380001"
DEBUG: truncate cascades to table "on_update_fkey_table_2380003"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380005"
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "reference_table_2380001"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380002"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380004"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380002"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380004"
DETAIL: NOTICE from localhost:57637
DEBUG: truncate cascades to table "on_update_fkey_table_2380003"
DETAIL: NOTICE from localhost:57637
DETAIL: NOTICE from localhost:57638
DEBUG: truncate cascades to table "on_update_fkey_table_2380005"
DETAIL: NOTICE from localhost:57638
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table"
DEBUG: building index "reference_table_pkey" on table "reference_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
ROLLBACK;
@ -487,6 +914,15 @@ BEGIN;
101
(1 row)
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
SELECT count(*) FROM transitive_reference_table;
count
-------
101
(1 row)
ROLLBACK;
-- case 5.2: Parallel UPDATE on distributed table follow by a UPDATE
BEGIN;
@ -495,6 +931,12 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
UPDATE transitive_reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -502,6 +944,12 @@ BEGIN;
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
-- case 5.4: Parallel UPDATE on distributed table follow by a related DDL on reference table
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
@ -512,6 +960,15 @@ DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table"
DEBUG: building index "transitive_reference_table_pkey" on table "transitive_reference_table"
DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -519,6 +976,12 @@ BEGIN;
ERROR: cannot execute SELECT on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_table;
ERROR: cannot execute SELECT on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -527,6 +990,13 @@ DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_tabl
DEBUG: validating foreign key constraint "fkey"
UPDATE reference_table SET id = 160 WHERE id = 15;
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
DEBUG: rewriting table "on_update_fkey_table"
DEBUG: building index "on_update_fkey_table_pkey" on table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey"
UPDATE transitive_reference_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;
@ -534,6 +1004,12 @@ BEGIN;
ERROR: cannot execute SELECT on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_table;
ERROR: cannot execute SELECT on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -541,6 +1017,12 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
UPDATE transitive_reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -548,6 +1030,12 @@ BEGIN;
ERROR: cannot execute DDL on reference relation "reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
ALTER TABLE transitive_reference_table ADD COLUMN X int;
ERROR: cannot execute DDL on reference relation "transitive_reference_table" because there was a parallel DDL access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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;
@ -564,6 +1052,12 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
DELETE FROM transitive_reference_table WHERE id = 99;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
-- an unrelated update followed by update on dist table and update
-- on reference table
BEGIN;
@ -573,6 +1067,13 @@ BEGIN;
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
ERROR: cannot execute DML on reference relation "transitive_reference_table" because there was a parallel DML access to distributed relation "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
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
@ -1076,8 +1577,8 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator
-- see https://github.com/citusdata/citus_docs/issues/664 for the discussion
WITH t1 AS (DELETE FROM reference_table RETURNING id)
DELETE FROM distributed_table USING t1 WHERE value_1 = t1.id RETURNING *;
DEBUG: generating subplan 92_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 92 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.distributed_table USING (SELECT intermediate_result.id FROM read_intermediate_result('92_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) RETURNING distributed_table.id, distributed_table.value_1, t1.id
DEBUG: generating subplan 170_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 170 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.distributed_table USING (SELECT intermediate_result.id FROM read_intermediate_result('170_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) RETURNING distributed_table.id, distributed_table.value_1, t1.id
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
id | value_1 | id
@ -1097,8 +1598,8 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator
-- see https://github.com/citusdata/citus_docs/issues/664 for the discussion
WITH t1 AS (DELETE FROM reference_table RETURNING id)
SELECT count(*) FROM distributed_table, t1 WHERE value_1 = t1.id;
DEBUG: generating subplan 96_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 96 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('96_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id)
DEBUG: generating subplan 174_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 174 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('174_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id)
DEBUG: switching to sequential query execution mode
DETAIL: Reference relation "reference_table" 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
count
@ -1111,17 +1612,17 @@ DETAIL: Reference relation "reference_table" is modified, which might lead to d
WITH t1 AS (DELETE FROM distributed_table RETURNING id),
t2 AS (DELETE FROM reference_table RETURNING id)
SELECT count(*) FROM distributed_table, t1, t2 WHERE value_1 = t1.id AND value_1 = t2.id;
DEBUG: generating subplan 98_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 98_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 98 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('98_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('98_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
DEBUG: generating subplan 176_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 176_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 176 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('176_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('176_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "distributed_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
-- similarly this should fail since we first access to a distributed
-- table via t1, and then access to the reference table in the main query
WITH t1 AS (DELETE FROM distributed_table RETURNING id)
DELETE FROM reference_table RETURNING id;
DEBUG: generating subplan 101_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 101 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: generating subplan 179_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 179 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
ERROR: cannot execute DML on reference relation "reference_table" because there was a parallel DML access to distributed relation "distributed_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
-- finally, make sure that we can execute the same queries
@ -1132,9 +1633,9 @@ BEGIN;
WITH t1 AS (DELETE FROM distributed_table RETURNING id),
t2 AS (DELETE FROM reference_table RETURNING id)
SELECT count(*) FROM distributed_table, t1, t2 WHERE value_1 = t1.id AND value_1 = t2.id;
DEBUG: generating subplan 103_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 103_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 103 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('103_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('103_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
DEBUG: generating subplan 181_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan 181_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan 181 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('181_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('181_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
count
-------
0
@ -1146,8 +1647,8 @@ BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
WITH t1 AS (DELETE FROM distributed_table RETURNING id)
DELETE FROM reference_table RETURNING id;
DEBUG: generating subplan 106_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 106 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: generating subplan 184_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan 184 query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
id
----
(0 rows)
@ -1155,8 +1656,9 @@ DEBUG: Plan 106 query after replacing subqueries and CTEs: DELETE FROM test_fke
ROLLBACK;
RESET client_min_messages;
DROP SCHEMA test_fkey_to_ref_in_tx CASCADE;
NOTICE: drop cascades to 4 other objects
DETAIL: drop cascades to table on_update_fkey_table
NOTICE: drop cascades to 5 other objects
DETAIL: drop cascades to table transitive_reference_table
drop cascades to table on_update_fkey_table
drop cascades to table unrelated_dist_table
drop cascades to table reference_table
drop cascades to table distributed_table

View File

@ -492,7 +492,7 @@ 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
-- self referencing foreign key on reference tables are 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);
@ -510,14 +510,10 @@ 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);
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
-- foreign Keys from reference table to reference table are not allowed
-- foreign Keys from reference table to reference table are allowed
ALTER TABLE referencing_table ADD CONSTRAINT fkey_ref FOREIGN KEY(id) REFERENCES referenced_table(id) ON UPDATE CASCADE;
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
DROP TABLE referenced_table;
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- cascades on delete with different schemas
CREATE SCHEMA referenced_schema;
CREATE SCHEMA referencing_schema;

View File

@ -0,0 +1,952 @@
Parsed test spec with 2 sessions
starting permutation: s2-begin s2-update-table-1 s1-begin s1-view-locks s1-rollback s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-update-table-1:
UPDATE ref_table_1 SET id = 2 WHERE id = 1;
step s1-begin:
BEGIN;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
ExclusiveLock 1
ShareLock 1
step s1-rollback:
ROLLBACK;
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-delete-table-1 s1-view-locks s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-delete-table-1:
DELETE FROM ref_table_1 WHERE id = 1;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
ExclusiveLock 1
ShareLock 1
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-update-table-2 s1-view-locks s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-update-table-2:
UPDATE ref_table_2 SET id = 2 WHERE id = 1;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
ExclusiveLock 2
ShareLock 1
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-delete-table-2 s1-view-locks s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-delete-table-2:
DELETE FROM ref_table_2 WHERE id = 1;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
ExclusiveLock 2
ShareLock 1
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-update-table-3 s1-begin s1-view-locks s1-rollback s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE id = 1;
step s1-begin:
BEGIN;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
ExclusiveLock 3
ShareLock 1
step s1-rollback:
ROLLBACK;
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-delete-table-3 s1-begin s1-view-locks s1-rollback s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-delete-table-3:
DELETE FROM ref_table_3 WHERE id = 1;
step s1-begin:
BEGIN;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
ExclusiveLock 3
ShareLock 1
step s1-rollback:
ROLLBACK;
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-insert-table-1 s1-view-locks s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
RowExclusiveLock1
ShareLock 1
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-insert-table-2 s1-view-locks s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-insert-table-2:
INSERT INTO ref_table_2 VALUES (7, 5);
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
RowExclusiveLock2
ShareLock 1
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s2-begin s2-insert-table-3 s1-view-locks s2-rollback s1-view-locks
step s2-begin:
BEGIN;
step s2-insert-table-3:
INSERT INTO ref_table_3 VALUES (7, 5);
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
RowExclusiveLock3
ShareLock 1
step s2-rollback:
ROLLBACK;
step s1-view-locks:
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
mode count
starting permutation: s1-begin s2-begin s2-update-table-1 s1-delete-table-2 s2-commit s1-commit s1-select-table-2
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-update-table-1:
UPDATE ref_table_1 SET id = 2 WHERE id = 1;
step s1-delete-table-2:
DELETE FROM ref_table_2 WHERE value = 2;
<waiting ...>
step s2-commit:
COMMIT;
step s1-delete-table-2: <... completed>
step s1-commit:
COMMIT;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
3 3
5 5
starting permutation: s1-begin s2-begin s2-update-table-1 s1-insert-table-2 s2-commit s1-commit s1-select-table-2
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-update-table-1:
UPDATE ref_table_1 SET id = 2 WHERE id = 1;
step s1-insert-table-2:
INSERT INTO ref_table_2 VALUES (7, 2);
<waiting ...>
step s2-commit:
COMMIT;
step s1-insert-table-2: <... completed>
step s1-commit:
COMMIT;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
1 2
3 3
5 5
7 2
starting permutation: s1-begin s2-begin s2-update-table-1 s1-update-table-2 s2-commit s1-commit s1-select-table-2
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-update-table-1:
UPDATE ref_table_1 SET id = 2 WHERE id = 1;
step s1-update-table-2:
UPDATE ref_table_2 SET id = 0 WHERE value = 2;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-table-2: <... completed>
step s1-commit:
COMMIT;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
0 2
3 3
5 5
starting permutation: s1-begin s2-begin s2-delete-table-1 s1-delete-table-2 s2-commit s1-commit s1-select-table-2
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-1:
DELETE FROM ref_table_1 WHERE id = 1;
step s1-delete-table-2:
DELETE FROM ref_table_2 WHERE value = 2;
<waiting ...>
step s2-commit:
COMMIT;
step s1-delete-table-2: <... completed>
step s1-commit:
COMMIT;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
3 3
5 5
starting permutation: s1-begin s2-begin s2-delete-table-1 s1-insert-table-2 s2-commit s1-commit s1-select-table-2
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-1:
DELETE FROM ref_table_1 WHERE id = 1;
step s1-insert-table-2:
INSERT INTO ref_table_2 VALUES (7, 2);
<waiting ...>
step s2-commit:
COMMIT;
step s1-insert-table-2: <... completed>
error in steps s2-commit s1-insert-table-2: ERROR: insert or update on table "ref_table_2_102048" violates foreign key constraint "ref_table_2_value_fkey_102048"
step s1-commit:
COMMIT;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
3 3
5 5
starting permutation: s1-begin s2-begin s2-delete-table-1 s1-update-table-2 s2-commit s1-commit s1-select-table-2
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-1:
DELETE FROM ref_table_1 WHERE id = 1;
step s1-update-table-2:
UPDATE ref_table_2 SET id = 0 WHERE value = 2;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-table-2: <... completed>
step s1-commit:
COMMIT;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
3 3
5 5
starting permutation: s1-begin s2-begin s2-delete-table-1 s1-delete-table-3 s2-commit s1-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-1:
DELETE FROM ref_table_1 WHERE id = 1;
step s1-delete-table-3:
DELETE FROM ref_table_3 WHERE value = 1 RETURNING id;
<waiting ...>
step s2-commit:
COMMIT;
step s1-delete-table-3: <... completed>
id
step s1-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
3 3
5 5
starting permutation: s1-begin s2-begin s2-delete-table-1 s1-insert-table-3 s2-commit s1-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-1:
DELETE FROM ref_table_1 WHERE id = 1;
step s1-insert-table-3:
INSERT INTO ref_table_3 VALUES (7, 1);
<waiting ...>
step s2-commit:
COMMIT;
step s1-insert-table-3: <... completed>
error in steps s2-commit s1-insert-table-3: ERROR: insert or update on table "ref_table_3_102058" violates foreign key constraint "ref_table_3_value_fkey_102058"
step s1-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
3 3
5 5
starting permutation: s1-begin s2-begin s2-delete-table-1 s1-update-table-3 s2-commit s1-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-1:
DELETE FROM ref_table_1 WHERE id = 1;
step s1-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE value = 1 RETURNING id;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-table-3: <... completed>
id
step s1-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
3 3
5 5
starting permutation: s1-begin s2-begin s2-insert-table-1 s1-update-table-3 s2-commit s1-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
step s1-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE value = 1 RETURNING id;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-table-3: <... completed>
id
2
step s1-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
2 1
3 3
5 5
starting permutation: s1-begin s2-begin s1-update-table-3 s2-insert-table-1 s1-commit s2-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE value = 1 RETURNING id;
id
2
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
<waiting ...>
step s1-commit:
COMMIT;
step s2-insert-table-1: <... completed>
step s2-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
2 1
3 3
5 5
starting permutation: s1-begin s2-begin s2-insert-table-1 s1-update-table-2 s2-commit s1-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
step s1-update-table-2:
UPDATE ref_table_2 SET id = 0 WHERE value = 2;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-table-2: <... completed>
step s1-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
1 1
3 3
5 5
starting permutation: s1-begin s2-begin s1-update-table-2 s2-insert-table-1 s1-commit s2-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-update-table-2:
UPDATE ref_table_2 SET id = 0 WHERE value = 2;
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
<waiting ...>
step s1-commit:
COMMIT;
step s2-insert-table-1: <... completed>
step s2-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
1 1
3 3
5 5
starting permutation: s1-begin s2-begin s2-insert-table-2 s1-update-table-3 s2-commit s1-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-insert-table-2:
INSERT INTO ref_table_2 VALUES (7, 5);
step s1-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE value = 1 RETURNING id;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-table-3: <... completed>
id
2
step s1-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
2 1
3 3
5 5
starting permutation: s1-begin s2-begin s1-update-table-3 s2-insert-table-2 s1-commit s2-commit s1-select-table-3
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE value = 1 RETURNING id;
id
2
step s2-insert-table-2:
INSERT INTO ref_table_2 VALUES (7, 5);
<waiting ...>
step s1-commit:
COMMIT;
step s2-insert-table-2: <... completed>
step s2-commit:
COMMIT;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
2 1
3 3
5 5
starting permutation: s1-begin s2-begin s2-insert-table-1 s1-select-table-1 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
step s1-select-table-1:
SELECT * FROM ref_table_1 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-insert-table-1 s1-select-table-2 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-insert-table-1 s1-select-table-3 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-insert-table-1:
INSERT INTO ref_table_1 VALUES (7, 7);
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-delete-table-2 s1-select-table-1 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-2:
DELETE FROM ref_table_2 WHERE id = 1;
step s1-select-table-1:
SELECT * FROM ref_table_1 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-delete-table-2 s1-select-table-2 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-2:
DELETE FROM ref_table_2 WHERE id = 1;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-delete-table-2 s1-select-table-3 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-delete-table-2:
DELETE FROM ref_table_2 WHERE id = 1;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-update-table-3 s1-select-table-1 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE id = 1;
step s1-select-table-1:
SELECT * FROM ref_table_1 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-update-table-3 s1-select-table-2 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE id = 1;
step s1-select-table-2:
SELECT * FROM ref_table_2 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
starting permutation: s1-begin s2-begin s2-update-table-3 s1-select-table-3 s2-commit s1-commit
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-update-table-3:
UPDATE ref_table_3 SET id = 2 WHERE id = 1;
step s1-select-table-3:
SELECT * FROM ref_table_3 ORDER BY id, value;
id value
1 1
3 3
5 5
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;

View File

@ -0,0 +1,658 @@
Parsed test spec with 2 sessions
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-update-table-1 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-update-table-1:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table_1 SET id = 2 WHERE id = 1');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(ExclusiveLock,1)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-delete-table-1 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-delete-table-1:
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table_1 WHERE id = 1');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(ExclusiveLock,1)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-update-table-2 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-update-table-2:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table_2 SET id = 2 WHERE id = 1');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(ExclusiveLock,2)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-delete-table-2 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-delete-table-2:
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table_2 WHERE id = 1');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(ExclusiveLock,2)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-update-table-3 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-update-table-3:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table_3 SET id = 2 WHERE id = 1');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(ExclusiveLock,3)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-delete-table-3 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-delete-table-3:
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table_3 WHERE id = 1');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(ExclusiveLock,3)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-insert-table-1 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-insert-table-1:
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table_1 VALUES (7, 7)');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(RowExclusiveLock,1)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-insert-table-2 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-insert-table-2:
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table_2 VALUES (7, 5)');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(RowExclusiveLock,2)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func
starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-insert-table-3 s1-start-session-level-connection s1-view-locks s2-rollback-worker s1-view-locks s1-stop-connection s2-stop-connection
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s2-begin-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
step s2-insert-table-3:
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table_3 VALUES (7, 5)');
run_commands_on_session_level_connection_to_node
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t {"(RowExclusiveLock,3)","(ShareLock,1)"}
step s2-rollback-worker:
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
run_commands_on_session_level_connection_to_node
step s1-view-locks:
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
node_name node_port success result
localhost 57637 t
step s1-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
restore_isolation_tester_func

View File

@ -835,8 +835,8 @@ 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 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
ERROR: cannot create foreign key constraint since foreign keys from reference tables to distributed tables are not supported
DETAIL: A reference table can only have reference keys to other reference tables
-- 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);
@ -848,16 +848,19 @@ SELECT create_reference_table('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 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
create_reference_table
------------------------
(1 row)
-- 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 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 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
ERROR: cannot create foreign key constraint
DETAIL: Referenced table must be a distributed table or a reference table.
-- test foreign key creation on CREATE TABLE on self referencing reference table
CREATE TABLE self_referencing_reference_table(
id int,
@ -867,8 +870,11 @@ 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 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
create_reference_table
------------------------
(1 row)
-- 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);
@ -879,8 +885,8 @@ 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 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
ERROR: cannot create foreign key constraint since foreign keys from reference tables to distributed tables are not supported
DETAIL: A reference table can only have reference keys to other reference tables
-- test foreign key creation on ALTER TABLE to reference table
CREATE TABLE references_to_reference_table(id int, referencing_column int);
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');
@ -900,11 +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 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 CASCADE;
NOTICE: drop cascades to constraint fk on table references_to_reference_table
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to constraint fk on table references_to_reference_table
drop cascades to constraint fk on table reference_table_second
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
SELECT create_reference_table('reference_table');
create_reference_table
@ -913,8 +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: 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
ERROR: cannot create foreign key constraint
DETAIL: Referenced table must be a distributed table or a reference 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(
@ -930,7 +936,5 @@ 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 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

@ -1,7 +1,7 @@
--
-- MULTI_REPLICATE_REFERENCE_TABLE
--
-- Tests that check the metadata returned by the master node.
-- Tests that check that reference tables are replicated when adding new nodes.
SET citus.next_shard_id TO 1370000;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1370000;
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1370000;
@ -640,6 +640,67 @@ WHERE colocationid IN
DROP TABLE replicate_reference_table_schema.table1;
DROP SCHEMA replicate_reference_table_schema CASCADE;
-- test adding a node when there are foreign keys between reference tables
SELECT master_remove_node('localhost', :worker_2_port);
master_remove_node
--------------------
(1 row)
CREATE TABLE ref_table_1(id int primary key, v int);
CREATE TABLE ref_table_2(id int primary key, v int references ref_table_1(id));
CREATE TABLE ref_table_3(id int primary key, v int references ref_table_2(id));
SELECT create_reference_table('ref_table_1'),
create_reference_table('ref_table_2'),
create_reference_table('ref_table_3');
create_reference_table | create_reference_table | create_reference_table
------------------------+------------------------+------------------------
| |
(1 row)
-- status before master_add_node
SELECT
shardid, shardstate, shardlength, nodename, nodeport
FROM
pg_dist_shard_placement
WHERE
nodeport = :worker_2_port;
shardid | shardstate | shardlength | nodename | nodeport
---------+------------+-------------+----------+----------
(0 rows)
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
NOTICE: Replicating reference table "ref_table_1" to the node localhost:57638
NOTICE: Replicating reference table "ref_table_2" to the node localhost:57638
NOTICE: Replicating reference table "ref_table_3" to the node localhost:57638
?column?
----------
1
(1 row)
-- status after master_add_node
SELECT
shardid, shardstate, shardlength, nodename, nodeport
FROM
pg_dist_shard_placement
WHERE
nodeport = :worker_2_port;
shardid | shardstate | shardlength | nodename | nodeport
---------+------------+-------------+-----------+----------
1370012 | 1 | 0 | localhost | 57638
1370013 | 1 | 0 | localhost | 57638
1370014 | 1 | 0 | localhost | 57638
(3 rows)
-- verify constraints have been created on the new node
SELECT run_command_on_workers('select count(*) from pg_constraint where contype=''f'' AND conname like ''ref_table%'';');
run_command_on_workers
------------------------
(localhost,57637,t,2)
(localhost,57638,t,2)
(2 rows)
DROP TABLE ref_table_1, ref_table_2, ref_table_3;
-- do some tests with inactive node
SELECT master_remove_node('localhost', :worker_2_port);
master_remove_node
@ -675,7 +736,7 @@ WHERE
ORDER BY 1,4,5;
shardid | shardstate | shardlength | nodename | nodeport
---------+------------+-------------+-----------+----------
1370012 | 1 | 0 | localhost | 57637
1370015 | 1 | 0 | localhost | 57637
(1 row)
-- we should see the two shard placements after activation
@ -700,8 +761,8 @@ WHERE
ORDER BY 1,4,5;
shardid | shardstate | shardlength | nodename | nodeport
---------+------------+-------------+-----------+----------
1370012 | 1 | 0 | localhost | 57637
1370012 | 1 | 0 | localhost | 57638
1370015 | 1 | 0 | localhost | 57637
1370015 | 1 | 0 | localhost | 57638
(2 rows)
-- this should have no effect

View File

@ -54,7 +54,9 @@ test: isolation_ddl_vs_all
test: isolation_get_all_active_transactions
test: isolation_validate_vs_insert
test: isolation_insert_select_conflict
test: isolation_ref2ref_foreign_keys
# MX tests
test: isolation_reference_on_mx
test: isolation_ref2ref_foreign_keys_on_mx
test: isolation_get_distributed_wait_queries

View File

@ -0,0 +1,213 @@
setup
{
CREATE TABLE ref_table_1(id int PRIMARY KEY, value int);
SELECT create_reference_table('ref_table_1');
CREATE TABLE ref_table_2(id int PRIMARY KEY, value int REFERENCES ref_table_1(id) ON DELETE CASCADE ON UPDATE CASCADE);
SELECT create_reference_table('ref_table_2');
CREATE TABLE ref_table_3(id int PRIMARY KEY, value int REFERENCES ref_table_2(id) ON DELETE CASCADE ON UPDATE CASCADE);
SELECT create_reference_table('ref_table_3');
INSERT INTO ref_table_1 VALUES (1, 1), (3, 3), (5, 5);
INSERT INTO ref_table_2 SELECT * FROM ref_table_1;
INSERT INTO ref_table_3 SELECT * FROM ref_table_2;
}
teardown
{
DROP TABLE ref_table_1, ref_table_2, ref_table_3;
}
session "s1"
step "s1-begin"
{
BEGIN;
}
step "s1-delete-table-2"
{
DELETE FROM ref_table_2 WHERE value = 2;
}
step "s1-insert-table-2"
{
INSERT INTO ref_table_2 VALUES (7, 2);
}
step "s1-update-table-2"
{
UPDATE ref_table_2 SET id = 0 WHERE value = 2;
}
step "s1-delete-table-3"
{
DELETE FROM ref_table_3 WHERE value = 1 RETURNING id;
}
step "s1-insert-table-3"
{
INSERT INTO ref_table_3 VALUES (7, 1);
}
step "s1-update-table-3"
{
UPDATE ref_table_3 SET id = 2 WHERE value = 1 RETURNING id;
}
step "s1-select-table-1"
{
SELECT * FROM ref_table_1 ORDER BY id, value;
}
step "s1-select-table-2"
{
SELECT * FROM ref_table_2 ORDER BY id, value;
}
step "s1-select-table-3"
{
SELECT * FROM ref_table_3 ORDER BY id, value;
}
step "s1-view-locks"
{
SELECT mode, count(*)
FROM pg_locks
WHERE locktype='advisory'
GROUP BY mode;
}
step "s1-rollback"
{
ROLLBACK;
}
step "s1-commit"
{
COMMIT;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-insert-table-1"
{
INSERT INTO ref_table_1 VALUES (7, 7);
}
step "s2-update-table-1"
{
UPDATE ref_table_1 SET id = 2 WHERE id = 1;
}
step "s2-delete-table-1"
{
DELETE FROM ref_table_1 WHERE id = 1;
}
step "s2-insert-table-2"
{
INSERT INTO ref_table_2 VALUES (7, 5);
}
step "s2-update-table-2"
{
UPDATE ref_table_2 SET id = 2 WHERE id = 1;
}
step "s2-delete-table-2"
{
DELETE FROM ref_table_2 WHERE id = 1;
}
step "s2-insert-table-3"
{
INSERT INTO ref_table_3 VALUES (7, 5);
}
step "s2-update-table-3"
{
UPDATE ref_table_3 SET id = 2 WHERE id = 1;
}
step "s2-delete-table-3"
{
DELETE FROM ref_table_3 WHERE id = 1;
}
step "s2-rollback"
{
ROLLBACK;
}
step "s2-commit"
{
COMMIT;
}
# Check that we get necessary resource locks
# Case 1. UPDATE/DELETE ref_table_1 should only lock its own shard in Exclusive mode.
permutation "s2-begin" "s2-update-table-1" "s1-begin" "s1-view-locks" "s1-rollback" "s2-rollback" "s1-view-locks"
permutation "s2-begin" "s2-delete-table-1" "s1-view-locks" "s2-rollback" "s1-view-locks"
# Case 2. Modifying ref_table_2 should also lock ref_table_1 shard in Exclusive mode.
permutation "s2-begin" "s2-update-table-2" "s1-view-locks" "s2-rollback" "s1-view-locks"
permutation "s2-begin" "s2-delete-table-2" "s1-view-locks" "s2-rollback" "s1-view-locks"
# Case 3. Modifying ref_table_3 should also lock ref_table_1 and ref_table_2 shards in Exclusive mode.
permutation "s2-begin" "s2-update-table-3" "s1-begin" "s1-view-locks" "s1-rollback" "s2-rollback" "s1-view-locks"
permutation "s2-begin" "s2-delete-table-3" "s1-begin" "s1-view-locks" "s1-rollback" "s2-rollback" "s1-view-locks"
# Case 4. Inserting into ref_table_1 should only lock its own shard in RowExclusive mode.
permutation "s2-begin" "s2-insert-table-1" "s1-view-locks" "s2-rollback" "s1-view-locks"
# Case 5. Modifying ref_table_2 should also lock ref_table_1 in RowExclusive mode.
permutation "s2-begin" "s2-insert-table-2" "s1-view-locks" "s2-rollback" "s1-view-locks"
# Case 6. Modifying ref_table_2 should also lock ref_table_1 in RowExclusive mode.
permutation "s2-begin" "s2-insert-table-3" "s1-view-locks" "s2-rollback" "s1-view-locks"
# Now some concurrent operations
# Updates/Deletes from ref_table_1 cascade to ref_table_2, so DML on ref_table_2 should block
# Case 1. UPDATE -> DELETE
permutation "s1-begin" "s2-begin" "s2-update-table-1" "s1-delete-table-2" "s2-commit" "s1-commit" "s1-select-table-2"
# Case 2. UPDATE -> INSERT
permutation "s1-begin" "s2-begin" "s2-update-table-1" "s1-insert-table-2" "s2-commit" "s1-commit" "s1-select-table-2"
# Case 3. UPDATE -> UPDATE
permutation "s1-begin" "s2-begin" "s2-update-table-1" "s1-update-table-2" "s2-commit" "s1-commit" "s1-select-table-2"
# Case 4. DELETE -> DELETE
permutation "s1-begin" "s2-begin" "s2-delete-table-1" "s1-delete-table-2" "s2-commit" "s1-commit" "s1-select-table-2"
# Case 5. DELETE -> INSERT
permutation "s1-begin" "s2-begin" "s2-delete-table-1" "s1-insert-table-2" "s2-commit" "s1-commit" "s1-select-table-2"
# Case 6. DELETE -> UPDATE
permutation "s1-begin" "s2-begin" "s2-delete-table-1" "s1-update-table-2" "s2-commit" "s1-commit" "s1-select-table-2"
# Deletes from ref_table_1 can transitively cascade to ref_table_3, so DML on ref_table_3 should block
# Case 1. DELETE -> DELETE
permutation "s1-begin" "s2-begin" "s2-delete-table-1" "s1-delete-table-3" "s2-commit" "s1-commit" "s1-select-table-3"
# Case 2. DELETE -> INSERT, should error out
permutation "s1-begin" "s2-begin" "s2-delete-table-1" "s1-insert-table-3" "s2-commit" "s1-commit" "s1-select-table-3"
# Case 3. DELETE -> UPDATE
permutation "s1-begin" "s2-begin" "s2-delete-table-1" "s1-update-table-3" "s2-commit" "s1-commit" "s1-select-table-3"
# Any DML on any of ref_table_{1,2,3} should block others from DML in the foreign constraint graph ...
permutation "s1-begin" "s2-begin" "s2-insert-table-1" "s1-update-table-3" "s2-commit" "s1-commit" "s1-select-table-3"
permutation "s1-begin" "s2-begin" "s1-update-table-3" "s2-insert-table-1" "s1-commit" "s2-commit" "s1-select-table-3"
permutation "s1-begin" "s2-begin" "s2-insert-table-1" "s1-update-table-2" "s2-commit" "s1-commit" "s1-select-table-3"
permutation "s1-begin" "s2-begin" "s1-update-table-2" "s2-insert-table-1" "s1-commit" "s2-commit" "s1-select-table-3"
permutation "s1-begin" "s2-begin" "s2-insert-table-2" "s1-update-table-3" "s2-commit" "s1-commit" "s1-select-table-3"
permutation "s1-begin" "s2-begin" "s1-update-table-3" "s2-insert-table-2" "s1-commit" "s2-commit" "s1-select-table-3"
# DMLs shouldn't block select on tables in the same foreign constraint graph
permutation "s1-begin" "s2-begin" "s2-insert-table-1" "s1-select-table-1" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-insert-table-1" "s1-select-table-2" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-insert-table-1" "s1-select-table-3" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-delete-table-2" "s1-select-table-1" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-delete-table-2" "s1-select-table-2" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-delete-table-2" "s1-select-table-3" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-update-table-3" "s1-select-table-1" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-update-table-3" "s1-select-table-2" "s2-commit" "s1-commit"
permutation "s1-begin" "s2-begin" "s2-update-table-3" "s1-select-table-3" "s2-commit" "s1-commit"

View File

@ -0,0 +1,153 @@
setup
{
SELECT citus.replace_isolation_tester_func();
SELECT citus.refresh_isolation_tester_prepared_statement();
-- start_metadata_sync_to_node can not be run inside a transaction block.
-- Following is a workaround to overcome that. Port numbers are hard coded
-- at the moment.
SELECT master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57636]::int[],
ARRAY[format('SELECT start_metadata_sync_to_node(''%s'', %s)', nodename, nodeport)]::text[],
false)
FROM pg_dist_node;
SET citus.replication_model to streaming;
CREATE TABLE ref_table_1(id int PRIMARY KEY, value int);
SELECT create_reference_table('ref_table_1');
CREATE TABLE ref_table_2(id int PRIMARY KEY, value int REFERENCES ref_table_1(id) ON DELETE CASCADE ON UPDATE CASCADE);
SELECT create_reference_table('ref_table_2');
CREATE TABLE ref_table_3(id int PRIMARY KEY, value int REFERENCES ref_table_2(id) ON DELETE CASCADE ON UPDATE CASCADE);
SELECT create_reference_table('ref_table_3');
INSERT INTO ref_table_1 VALUES (1, 1), (3, 3), (5, 5);
INSERT INTO ref_table_2 SELECT * FROM ref_table_1;
INSERT INTO ref_table_3 SELECT * FROM ref_table_2;
}
teardown
{
DROP TABLE ref_table_1, ref_table_2, ref_table_3;
SELECT citus.restore_isolation_tester_func();
}
session "s1"
step "s1-start-session-level-connection"
{
SELECT start_session_level_connection_to_node('localhost', 57637);
}
step "s1-begin-on-worker"
{
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
}
step "s1-view-locks"
{
SELECT * FROM master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[57637]::int[],
ARRAY[$$
SELECT array_agg(ROW(t.mode, t.count) ORDER BY t.mode) FROM
(SELECT mode, count(*) count FROM pg_locks
WHERE locktype='advisory' GROUP BY mode) t$$]::text[],
false);
}
step "s1-rollback-worker"
{
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
}
step "s1-stop-connection"
{
SELECT stop_session_level_connection_to_node();
}
session "s2"
step "s2-start-session-level-connection"
{
SELECT start_session_level_connection_to_node('localhost', 57637);
}
step "s2-begin-on-worker"
{
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
}
step "s2-insert-table-1"
{
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table_1 VALUES (7, 7)');
}
step "s2-update-table-1"
{
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table_1 SET id = 2 WHERE id = 1');
}
step "s2-delete-table-1"
{
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table_1 WHERE id = 1');
}
step "s2-insert-table-2"
{
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table_2 VALUES (7, 5)');
}
step "s2-update-table-2"
{
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table_2 SET id = 2 WHERE id = 1');
}
step "s2-delete-table-2"
{
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table_2 WHERE id = 1');
}
step "s2-insert-table-3"
{
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table_3 VALUES (7, 5)');
}
step "s2-update-table-3"
{
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table_3 SET id = 2 WHERE id = 1');
}
step "s2-delete-table-3"
{
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table_3 WHERE id = 1');
}
step "s2-rollback-worker"
{
SELECT run_commands_on_session_level_connection_to_node('ROLLBACK');
}
step "s2-stop-connection"
{
SELECT stop_session_level_connection_to_node();
}
# Case 1. UPDATE/DELETE ref_table_1 should only lock its own shard in Exclusive mode.
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-table-1" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-delete-table-1" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
# Case 2. Modifying ref_table_2 should also lock ref_table_1 shard in Exclusive mode.
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-table-2" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-delete-table-2" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
# Case 3. Modifying ref_table_3 should also lock ref_table_1 and ref_table_2 shards in Exclusive mode.
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-table-3" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-delete-table-3" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
# Case 4. Inserting into ref_table_1 should only lock its own shard in RowExclusive mode.
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-table-1" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
# Case 5. Modifying ref_table_2 should also lock ref_table_1 in RowExclusive mode.
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-table-2" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"
# Case 6. Modifying ref_table_2 should also lock ref_table_1 in RowExclusive mode.
permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-table-3" "s1-start-session-level-connection" "s1-view-locks" "s2-rollback-worker" "s1-view-locks" "s1-stop-connection" "s2-stop-connection"

View File

@ -14,7 +14,10 @@ SET citus.next_placement_id TO 2380000;
SET citus.shard_replication_factor TO 1;
CREATE TABLE reference_table(id int PRIMARY KEY);
CREATE TABLE transitive_reference_table(id int PRIMARY KEY);
SELECT create_reference_table('transitive_reference_table');
CREATE TABLE reference_table(id int PRIMARY KEY, value_1 int);
SELECT create_reference_table('reference_table');
CREATE TABLE on_update_fkey_table(id int PRIMARY KEY, value_1 int);
@ -24,8 +27,10 @@ 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 reference_table(id) ON UPDATE CASCADE;
ALTER TABLE reference_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES transitive_reference_table(id) ON UPDATE CASCADE;
INSERT INTO reference_table SELECT i FROM generate_series(0, 100) i;
INSERT INTO transitive_reference_table SELECT i FROM generate_series(0, 100) i;
INSERT INTO reference_table SELECT i, 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;
@ -39,6 +44,11 @@ BEGIN;
SELECT count(*) FROM on_update_fkey_table;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_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 reference_table;
@ -46,17 +56,29 @@ BEGIN;
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;
BEGIN;
SELECT count(*) FROM transitive_reference_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;
BEGIN;
SELECT count(*) FROM reference_table;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_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;
BEGIN;
SELECT count(*) FROM reference_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;
@ -64,18 +86,36 @@ BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE id = 18;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_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 reference_table;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_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 reference_table;
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_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;
@ -86,6 +126,14 @@ BEGIN;
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_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;
@ -95,6 +143,13 @@ BEGIN;
ALTER TABLE on_update_fkey_table DROP COLUMN value_1 CASCADE;
ROLLBACK;
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
SELECT count(*) FROM transitive_reference_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;
@ -103,6 +158,12 @@ BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X INT;
ROLLBACK;
BEGIN;
SELECT count(*) FROM unrelated_dist_table;
SELECT count(*) FROM transitive_reference_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 reference_table;
@ -115,6 +176,17 @@ BEGIN;
\.
ROLLBACK;
BEGIN;
SELECT count(*) FROM transitive_reference_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 reference_table SET id = 101 WHERE id = 99;
@ -122,6 +194,12 @@ BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_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 reference_table SET id = 101 WHERE id = 99;
@ -131,6 +209,13 @@ BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 399;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_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;
@ -138,6 +223,11 @@ BEGIN;
UPDATE on_update_fkey_table SET value_1 = 15;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_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 reference_table SET id = 101 WHERE id = 99;
@ -147,18 +237,36 @@ BEGIN;
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 4;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_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 reference_table SET id = 101 WHERE id = 99;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_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 reference_table SET id = 101 WHERE id = 99;
ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_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 reference_table SET id = 101 WHERE id = 99;
@ -171,6 +279,16 @@ BEGIN;
\.
ROLLBACK;
BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
COPY on_update_fkey_table FROM STDIN WITH CSV;
1001,101
1002,101
1003,101
1004,101
1005,101
\.
ROLLBACK;
-- case 2.8: UPDATE to a reference table is followed by TRUNCATE
BEGIN;
@ -178,18 +296,32 @@ BEGIN;
TRUNCATE on_update_fkey_table;
ROLLBACK;
BEGIN;
UPDATE transitive_reference_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 reference_table ALTER COLUMN id SET DEFAULT 1001;
SELECT count(*) FROM on_update_fkey_table;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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 reference_table ALTER COLUMN id SET DATA TYPE int;
SELECT count(*) FROM on_update_fkey_table;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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;
@ -197,6 +329,11 @@ BEGIN;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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 reference_table ALTER COLUMN id SET DEFAULT 1001;
@ -206,6 +343,13 @@ BEGIN;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 4;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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;
@ -213,12 +357,22 @@ BEGIN;
CREATE INDEX fkey_test_index_1 ON on_update_fkey_table(value_1);
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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 reference_table ALTER COLUMN id SET DATA TYPE smallint;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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 reference_table ADD COLUMN X int;
@ -231,18 +385,38 @@ BEGIN;
\.
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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 reference_table ADD COLUMN X int;
TRUNCATE on_update_fkey_table;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_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 reference_table ALTER COLUMN id SET DATA TYPE smallint;
TRUNCATE on_update_fkey_table;
ROLLBACK;
BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
TRUNCATE on_update_fkey_table;
ROLLBACK;
-----
--- Now, start testing the other way araound
@ -254,97 +428,171 @@ BEGIN;
SELECT count(*) FROM reference_table;
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
SELECT count(*) FROM transitive_reference_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 reference_table SET id = 101 WHERE id = 99;
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
UPDATE transitive_reference_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 reference_table ADD COLUMN X INT;
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
ALTER TABLE transitive_reference_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 reference_table ALTER COLUMN id SET DATA TYPE smallint;
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
ALTER TABLE transitive_reference_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 reference_table CASCADE;
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
TRUNCATE transitive_reference_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 reference_table CASCADE;
ROLLBACK;
BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE id = 9;
TRUNCATE transitive_reference_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 reference_table;
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
SELECT count(*) FROM transitive_reference_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 reference_table SET id = 160 WHERE id = 15;
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
UPDATE transitive_reference_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 reference_table ADD COLUMN X INT;
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_table ADD COLUMN X INT;
ROLLBACK;
-- case 5.4: Parallel UPDATE on distributed table follow by a related DDL on reference table
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint;
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_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 reference_table;
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_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 reference_table SET id = 160 WHERE id = 15;
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE smallint;
UPDATE transitive_reference_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 reference_table;
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_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 reference_table SET id = 160 WHERE id = 15;
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
UPDATE transitive_reference_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 reference_table ADD COLUMN X int;
ROLLBACK;
BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int;
ALTER TABLE transitive_reference_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
@ -353,6 +601,11 @@ BEGIN;
DELETE FROM reference_table WHERE id = 99;
ROLLBACK;
BEGIN;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
DELETE FROM transitive_reference_table WHERE id = 99;
ROLLBACK;
-- an unrelated update followed by update on dist table and update
-- on reference table
BEGIN;
@ -361,6 +614,12 @@ BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99;
ROLLBACK;
BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
UPDATE transitive_reference_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

View File

@ -223,7 +223,7 @@ SELECT count(*) FROM referencing_table;
DROP TABLE referencing_table;
DROP TABLE referenced_table;
-- self referencing foreign key on reference tables are not allowed
-- self referencing foreign key on reference tables are 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);
@ -231,11 +231,11 @@ 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
-- foreign Keys from reference table to reference table are 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;
DROP TABLE referenced_table;
-- cascades on delete with different schemas
CREATE SCHEMA referenced_schema;

View File

@ -1,7 +1,7 @@
--
-- MULTI_REPLICATE_REFERENCE_TABLE
--
-- Tests that check the metadata returned by the master node.
-- Tests that check that reference tables are replicated when adding new nodes.
SET citus.next_shard_id TO 1370000;
@ -416,6 +416,41 @@ WHERE colocationid IN
DROP TABLE replicate_reference_table_schema.table1;
DROP SCHEMA replicate_reference_table_schema CASCADE;
-- test adding a node when there are foreign keys between reference tables
SELECT master_remove_node('localhost', :worker_2_port);
CREATE TABLE ref_table_1(id int primary key, v int);
CREATE TABLE ref_table_2(id int primary key, v int references ref_table_1(id));
CREATE TABLE ref_table_3(id int primary key, v int references ref_table_2(id));
SELECT create_reference_table('ref_table_1'),
create_reference_table('ref_table_2'),
create_reference_table('ref_table_3');
-- status before master_add_node
SELECT
shardid, shardstate, shardlength, nodename, nodeport
FROM
pg_dist_shard_placement
WHERE
nodeport = :worker_2_port;
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
-- status after master_add_node
SELECT
shardid, shardstate, shardlength, nodename, nodeport
FROM
pg_dist_shard_placement
WHERE
nodeport = :worker_2_port;
-- verify constraints have been created on the new node
SELECT run_command_on_workers('select count(*) from pg_constraint where contype=''f'' AND conname like ''ref_table%'';');
DROP TABLE ref_table_1, ref_table_2, ref_table_3;
-- do some tests with inactive node
SELECT master_remove_node('localhost', :worker_2_port);