Create foreign key relation graph and functions to query on it

pull/2240/head
velioglu 2018-06-12 12:44:35 +03:00 committed by mehmet furkan şahin
parent 89a8d6ab95
commit 6be6911ed9
13 changed files with 1839 additions and 15 deletions

View File

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

View File

@ -759,6 +759,11 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn,
colocationId);
if (TableReferenced(relationId) || TableReferencing(relationId))
{
InvalidateForeignKeyGraph();
}
relation_close(relation, NoLock);
}

View File

@ -649,3 +649,97 @@ HeapTupleOfForeignConstraintIncludesColumn(HeapTuple heapTuple, Oid relationId,
return false;
}
/*
* TableReferencing function checks whether given table is referencing by another table
* via foreign constraints. If it is referencing, this function returns true. To check
* that, this function searches given relation at pg_constraints system catalog. However
* since there is no index for the column we searched, this function performs sequential
* search, therefore call this function with caution.
*/
bool
TableReferencing(Oid relationId)
{
Relation pgConstraint = NULL;
HeapTuple heapTuple = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Oid scanIndexId = InvalidOid;
bool useIndex = false;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
relationId);
scanDescriptor = systable_beginscan(pgConstraint, scanIndexId, useIndex, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
if (constraintForm->contype == CONSTRAINT_FOREIGN)
{
systable_endscan(scanDescriptor);
heap_close(pgConstraint, NoLock);
return true;
}
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
heap_close(pgConstraint, NoLock);
return false;
}
/*
* ConstraintIsAForeignKey returns true if the given constraint name
* is a foreign key to defined on the relation.
*/
bool
ConstraintIsAForeignKey(char *constraintNameInput, Oid relationId)
{
Relation pgConstraint = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
HeapTuple heapTuple = NULL;
pgConstraint = heap_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_contype, BTEqualStrategyNumber, F_CHAREQ,
CharGetDatum(CONSTRAINT_FOREIGN));
scanDescriptor = systable_beginscan(pgConstraint, InvalidOid, false,
NULL, scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
char *constraintName = (constraintForm->conname).data;
if (strncmp(constraintName, constraintNameInput, NAMEDATALEN) == 0 &&
constraintForm->conrelid == relationId)
{
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
return true;
}
heapTuple = systable_getnext(scanDescriptor);
}
/* clean up scan and close system catalog */
systable_endscan(scanDescriptor);
heap_close(pgConstraint, AccessShareLock);
return false;
}

View File

@ -75,6 +75,7 @@
#include "utils/builtins.h"
#include "utils/elog.h"
#include "utils/errcodes.h"
#include "utils/fmgroids.h"
#include "utils/guc.h"
#include "utils/hsearch.h"
#include "utils/inval.h"
@ -86,6 +87,10 @@
bool EnableDDLPropagation = true; /* ddl propagation is enabled */
static bool shouldInvalidateForeignKeyGraph = false;
/*
* This struct defines the state for the callback for drop statements.
* It is copied as it is from commands/tablecmds.c in Postgres source.
@ -162,7 +167,11 @@ static void CheckCopyPermissions(CopyStmt *copyStatement);
static List * CopyGetAttnums(TupleDesc tupDesc, Relation rel, List *attnamelist);
static void PostProcessUtility(Node *parsetree);
static List * CollectGrantTableIdList(GrantStmt *grantStmt);
static char * GetSchemaNameFromDropObject(ListCell *dropSchemaCell);
static void ProcessDropTableStmt(DropStmt *dropTableStatement);
static void ProcessDropSchemaStmt(DropStmt *dropSchemaStatement);
static void InvalidateForeignKeyGraphForDDL(void);
/*
* We need to run some of the commands sequentially if there is a foreign constraint
@ -371,6 +380,11 @@ multi_ProcessUtility(PlannedStmt *pstmt,
{
ProcessDropTableStmt(dropStatement);
}
if (dropStatement->removeType == OBJECT_SCHEMA)
{
ProcessDropSchemaStmt(dropStatement);
}
}
if (IsA(parsetree, AlterTableStmt))
@ -530,6 +544,16 @@ multi_ProcessUtility(PlannedStmt *pstmt,
SetUserIdAndSecContext(savedUserId, savedSecurityContext);
}
/*
* Re-forming the foreign key graph relies on the command being executed
* on the local table first. However, in order to decide whether the
* command leads to an invalidation, we need to check before the command
* is being executed since we read pg_constraint table. Thus, we maintain a
* local flag and do the invalidation after multi_ProcessUtility,
* before ExecuteDistributedDDLJob().
*/
InvalidateForeignKeyGraphForDDL();
/* after local command has completed, finish by executing worker DDLJobs, if any */
if (ddlJobs != NIL)
{
@ -556,8 +580,27 @@ multi_ProcessUtility(PlannedStmt *pstmt,
if (alterTableType == AT_AddConstraint)
{
LOCKMODE lockmode = NoLock;
Oid relationId = InvalidOid;
Constraint *constraint = NULL;
Assert(list_length(commandList) == 1);
ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement);
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId))
{
continue;
}
constraint = (Constraint *) command->def;
if (ConstraintIsAForeignKey(constraint->conname, relationId))
{
InvalidateForeignKeyGraph();
}
}
}
}
@ -617,6 +660,22 @@ multi_ProcessUtility(PlannedStmt *pstmt,
}
/*
* InvalidateForeignKeyGraphForDDL simply keeps track of whether
* the foreign key graph should be invalidated due to a DDL.
*/
static void
InvalidateForeignKeyGraphForDDL(void)
{
if (shouldInvalidateForeignKeyGraph)
{
InvalidateForeignKeyGraph();
shouldInvalidateForeignKeyGraph = false;
}
}
/*
* IsCitusExtensionStmt returns whether a given utility is a CREATE or ALTER
* EXTENSION statement which references the citus extension. This function
@ -2244,8 +2303,25 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
}
#endif
case AT_SetNotNull:
case AT_DropConstraint:
{
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId))
{
return;
}
if (ConstraintIsAForeignKey(command->name, relationId))
{
shouldInvalidateForeignKeyGraph = true;
}
break;
}
case AT_SetNotNull:
case AT_EnableTrigAll:
case AT_DisableTrigAll:
case AT_ReplicaIdentity:
@ -3747,6 +3823,99 @@ RoleSpecString(RoleSpec *spec)
}
/*
* ProcessDropSchemaStmt invalidates the foreign key cache if any table created
* under dropped schema involved in any foreign key relationship.
*/
static void
ProcessDropSchemaStmt(DropStmt *dropStatement)
{
Relation pgClass = NULL;
HeapTuple heapTuple = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Oid scanIndexId = InvalidOid;
bool useIndex = false;
ListCell *dropSchemaCell;
if (dropStatement->behavior != DROP_CASCADE)
{
return;
}
foreach(dropSchemaCell, dropStatement->objects)
{
char *schemaString = GetSchemaNameFromDropObject(dropSchemaCell);
Oid namespaceOid = get_namespace_oid(schemaString, true);
if (namespaceOid == InvalidOid)
{
continue;
}
pgClass = heap_open(RelationRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_class_relnamespace, BTEqualStrategyNumber,
F_OIDEQ, namespaceOid);
scanDescriptor = systable_beginscan(pgClass, scanIndexId, useIndex, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_class relationForm = (Form_pg_class) GETSTRUCT(heapTuple);
char *relationName = NameStr(relationForm->relname);
Oid relationId = get_relname_relid(relationName, namespaceOid);
/* we're not interested in non-valid, non-distributed relations */
if (relationId == InvalidOid || !IsDistributedTable(relationId))
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
/* invalidate foreign key cache if the table involved in any foreign key */
if (TableReferenced(relationId) || TableReferencing(relationId))
{
shouldInvalidateForeignKeyGraph = true;
systable_endscan(scanDescriptor);
heap_close(pgClass, NoLock);
return;
}
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
heap_close(pgClass, NoLock);
}
}
/*
* GetSchemaNameFromDropObject gets the name of the drop schema from given
* list cell. This function is defined due to API change between PG 9.6 and
* PG 10.
*/
static char *
GetSchemaNameFromDropObject(ListCell *dropSchemaCell)
{
char *schemaString = NULL;
#if (PG_VERSION_NUM >= 100000)
Value *schemaValue = (Value *) lfirst(dropSchemaCell);
schemaString = strVal(schemaValue);
#else
List *schemaNameList = (List *) lfirst(dropSchemaCell);
schemaString = NameListToString(schemaNameList);
#endif
return schemaString;
}
/*
* ProcessDropTableStmt processes DROP TABLE commands for partitioned tables.
* If we are trying to DROP partitioned tables, we first need to go to MX nodes
@ -3774,10 +3943,20 @@ ProcessDropTableStmt(DropStmt *dropTableStatement)
Oid relationId = RangeVarGetRelid(tableRangeVar, AccessShareLock, missingOK);
if (relationId == InvalidOid ||
!IsDistributedTable(relationId) ||
!ShouldSyncTableMetadata(relationId) ||
!PartitionedTable(relationId))
/* we're not interested in non-valid, non-distributed relations */
if (relationId == InvalidOid || !IsDistributedTable(relationId))
{
continue;
}
/* invalidate foreign key cache if the table involved in any foreign key */
if ((TableReferenced(relationId) || TableReferencing(relationId)))
{
shouldInvalidateForeignKeyGraph = true;
}
/* we're only interested in partitioned and mx tables */
if (!ShouldSyncTableMetadata(relationId) || !PartitionedTable(relationId))
{
continue;
}

View File

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

View File

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

View File

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

View File

@ -12,6 +12,7 @@
#include "postgres.h"
#include "postgres_ext.h"
#include "utils/relcache.h"
#include "utils/hsearch.h"
#include "nodes/primnodes.h"
extern bool ConstraintIsAForeignKeyToReferenceTable(char *constraintName,
@ -25,5 +26,7 @@ extern bool ColumnAppearsInForeignKeyToReferenceTable(char *columnName, Oid
extern List * GetTableForeignConstraintCommands(Oid relationId);
extern bool HasForeignKeyToReferenceTable(Oid relationId);
extern bool TableReferenced(Oid relationId);
extern bool TableReferencing(Oid relationId);
extern bool ConstraintIsAForeignKey(char *constraintName, Oid relationId);
#endif

View File

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

View File

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

View File

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

View File

@ -228,7 +228,7 @@ test: multi_citus_tools
# ----------
# multi_foreign_key tests foreign key push down on distributed tables
# ----------
test: multi_foreign_key
test: multi_foreign_key multi_foreign_key_relation_graph
# ----------
# multi_upgrade_reference_table tests for upgrade_reference_table UDF

View File

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