mirror of https://github.com/citusdata/citus.git
Create foreign key relation graph and functions to query on it
parent
89a8d6ab95
commit
6be6911ed9
2
Makefile
2
Makefile
|
@ -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 \
|
||||
|
|
|
@ -759,6 +759,11 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
|
|||
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn,
|
||||
colocationId);
|
||||
|
||||
if (TableReferenced(relationId) || TableReferencing(relationId))
|
||||
{
|
||||
InvalidateForeignKeyGraph();
|
||||
}
|
||||
|
||||
relation_close(relation, NoLock);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -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);
|
||||
|
|
|
@ -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
|
|
@ -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
|
||||
|
|
|
@ -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;
|
Loading…
Reference in New Issue